Andres Freund <and...@anarazel.de> writes:
> I wonder if we shouldn't just expose a 'which pid is process X waiting
> for' API, implemented serverside. That's generally really useful, and
> looks like it's actually going to be less complicated than that
> query... And it's surely going to be faster.

Attached is a draft patch for a new function that reports the set of PIDs
directly blocking a given PID (that is, holding or awaiting conflicting
locks on the lockable object it's waiting for).

I replaced isolationtester's pg_locks query with this, and found that
it's about 9X faster in a normal build, and 3X faster with
CLOBBER_CACHE_ALWAYS turned on.  That would give us some nice headroom
for the isolation tests with CLOBBER_CACHE_ALWAYS animals.  (Note that
in view of
http://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=markhor&dt=2016-02-14%2007%3A38%3A37
we still need to do *something* about the speed of the new deadlock-hard
test; this patch could avoid the need to dumb down or slow down that test
even further.)

Not to be neglected also is that (I believe) this gives the right answer,
whereas isolationtester's existing query is currently completely broken by
parallel queries, and it doesn't understand non-conflicting lock modes
either.  (It did, at least partially, before commit 38f8bdcac4982215;
I am not sure that taking out the mode checks was a good idea.  But
putting them back would make the query slower yet.)

This is WIP, in part because I've written no user docs for the new
function, and in part because I think people might want to bikeshed the
API.  What is here is:

"pg_blocker_pids(integer) returns integer[]" returns a possibly-empty
array of the PIDs of backend processes that block the backend with
specified PID.  You get an empty array, not an error, if the argument
isn't a valid backend PID or that backend isn't waiting.  In parallel
query situations, the output includes PIDs that are blocking any PID
in the given process's lock group, and what is reported is always
the PID of the lock group leader for whichever process is kdoing the
blocking.  Also, in parallel query situations, the same PID might appear
multiple times in the output; it didn't seem worth trying to eliminate
duplicates.

Reasonable API change requests might include returning a rowset rather
than an array and returning more data per lock conflict.  I've not
bothered with either thing here because isolationtester doesn't care
and it would make the query somewhat slower for isolationtester's
usage (though, probably, not enough slower to really matter).

It should also be noted that I've not really tested the parallel
query aspects of this, because I'm not sure how to create a conflicting
lock request in a parallel worker.  However, if I'm not still
misunderstanding the new semantics of the lock data structures, that
aspect of it seems unlikely to be wrong.

Comments?

                        regards, tom lane

diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 91218d0..97e8962 100644
*** a/src/backend/storage/ipc/procarray.c
--- b/src/backend/storage/ipc/procarray.c
*************** HaveVirtualXIDsDelayingChkpt(VirtualTran
*** 2313,2318 ****
--- 2313,2341 ----
  PGPROC *
  BackendPidGetProc(int pid)
  {
+ 	PGPROC	   *result;
+ 
+ 	if (pid == 0)				/* never match dummy PGPROCs */
+ 		return NULL;
+ 
+ 	LWLockAcquire(ProcArrayLock, LW_SHARED);
+ 
+ 	result = BackendPidGetProcWithLock(pid);
+ 
+ 	LWLockRelease(ProcArrayLock);
+ 
+ 	return result;
+ }
+ 
+ /*
+  * BackendPidGetProcWithLock -- get a backend's PGPROC given its PID
+  *
+  * Same as above, except caller must be holding ProcArrayLock.  The found
+  * entry, if any, can be assumed to be valid as long as the lock remains held.
+  */
+ PGPROC *
+ BackendPidGetProcWithLock(int pid)
+ {
  	PGPROC	   *result = NULL;
  	ProcArrayStruct *arrayP = procArray;
  	int			index;
*************** BackendPidGetProc(int pid)
*** 2320,2327 ****
  	if (pid == 0)				/* never match dummy PGPROCs */
  		return NULL;
  
- 	LWLockAcquire(ProcArrayLock, LW_SHARED);
- 
  	for (index = 0; index < arrayP->numProcs; index++)
  	{
  		PGPROC	   *proc = &allProcs[arrayP->pgprocnos[index]];
--- 2343,2348 ----
*************** BackendPidGetProc(int pid)
*** 2333,2340 ****
  		}
  	}
  
- 	LWLockRelease(ProcArrayLock);
- 
  	return result;
  }
  
--- 2354,2359 ----
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index fef59a2..fb32769 100644
*** a/src/backend/storage/lmgr/lock.c
--- b/src/backend/storage/lmgr/lock.c
***************
*** 21,27 ****
   *
   *	Interface:
   *
!  *	InitLocks(), GetLocksMethodTable(),
   *	LockAcquire(), LockRelease(), LockReleaseAll(),
   *	LockCheckConflicts(), GrantLock()
   *
--- 21,27 ----
   *
   *	Interface:
   *
!  *	InitLocks(), GetLocksMethodTable(), GetLockTagsMethodTable(),
   *	LockAcquire(), LockRelease(), LockReleaseAll(),
   *	LockCheckConflicts(), GrantLock()
   *
***************
*** 41,46 ****
--- 41,47 ----
  #include "pg_trace.h"
  #include "pgstat.h"
  #include "storage/proc.h"
+ #include "storage/procarray.h"
  #include "storage/sinvaladt.h"
  #include "storage/spin.h"
  #include "storage/standby.h"
*************** static void CleanUpLock(LOCK *lock, PROC
*** 356,361 ****
--- 357,364 ----
  static void LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
  					 LOCKTAG *locktag, LOCKMODE lockmode,
  					 bool decrement_strong_lock_count);
+ static void GetSingleProcBlockerStatusData(PGPROC *blocked_proc,
+ 							   BlockedProcsData *data);
  
  
  /*
*************** GetLocksMethodTable(const LOCK *lock)
*** 462,467 ****
--- 465,482 ----
  	return LockMethods[lockmethodid];
  }
  
+ /*
+  * Fetch the lock method table associated with a given locktag
+  */
+ LockMethod
+ GetLockTagsMethodTable(const LOCKTAG *locktag)
+ {
+ 	LOCKMETHODID lockmethodid = (LOCKMETHODID) locktag->locktag_lockmethodid;
+ 
+ 	Assert(0 < lockmethodid && lockmethodid < lengthof(LockMethods));
+ 	return LockMethods[lockmethodid];
+ }
+ 
  
  /*
   * Compute the hash code associated with a LOCKTAG.
*************** GetLockStatusData(void)
*** 3406,3412 ****
  	 * impractical (in particular, note MAX_SIMUL_LWLOCKS).  It shouldn't
  	 * matter too much, because none of these locks can be involved in lock
  	 * conflicts anyway - anything that might must be present in the main lock
! 	 * table.
  	 */
  	for (i = 0; i < ProcGlobal->allProcCount; ++i)
  	{
--- 3421,3430 ----
  	 * impractical (in particular, note MAX_SIMUL_LWLOCKS).  It shouldn't
  	 * matter too much, because none of these locks can be involved in lock
  	 * conflicts anyway - anything that might must be present in the main lock
! 	 * table.  (For the same reason, we don't sweat about making leaderPid
! 	 * completely valid.  We cannot safely dereference another backend's
! 	 * lockGroupLeader field without holding all lock partition locks, and
! 	 * it's not worth that.)
  	 */
  	for (i = 0; i < ProcGlobal->allProcCount; ++i)
  	{
*************** GetLockStatusData(void)
*** 3439,3444 ****
--- 3457,3463 ----
  			instance->backend = proc->backendId;
  			instance->lxid = proc->lxid;
  			instance->pid = proc->pid;
+ 			instance->leaderPid = proc->pid;
  			instance->fastpath = true;
  
  			el++;
*************** GetLockStatusData(void)
*** 3466,3471 ****
--- 3485,3491 ----
  			instance->backend = proc->backendId;
  			instance->lxid = proc->lxid;
  			instance->pid = proc->pid;
+ 			instance->leaderPid = proc->pid;
  			instance->fastpath = true;
  
  			el++;
*************** GetLockStatusData(void)
*** 3517,3522 ****
--- 3537,3543 ----
  		instance->backend = proc->backendId;
  		instance->lxid = proc->lxid;
  		instance->pid = proc->pid;
+ 		instance->leaderPid = proclock->groupLeader->pid;
  		instance->fastpath = false;
  
  		el++;
*************** GetLockStatusData(void)
*** 3538,3543 ****
--- 3559,3754 ----
  }
  
  /*
+  * GetBlockerStatusData - Return a summary of the lock manager's state
+  * concerning locks that are blocking the specified PID or any member of
+  * the PID's lock group, for use in a user-level reporting function.
+  *
+  * For each PID within the lock group that is awaiting some heavyweight lock,
+  * the return data includes an array of LockInstanceData objects, which are
+  * the same data structure used by GetLockStatusData; but unlike that function,
+  * this one reports only the PROCLOCKs associated with the lock that that pid
+  * is blocked on.  (Hence, all the locktags should be the same for any one
+  * blocked PID.)  In addition, we return an array of the PIDs of those backends
+  * that are ahead of the blocked PID in the lock's wait queue.  These can be
+  * compared with the pids in the LockInstanceData objects to determine which
+  * waiters are ahead of or behind the blocked PID in the queue.
+  *
+  * If blocked_pid isn't a valid backend PID or nothing in its lock group is
+  * waiting on any heavyweight lock, return empty arrays.
+  *
+  * The design goal is to hold the LWLocks for as short a time as possible;
+  * thus, this function simply makes a copy of the necessary data and releases
+  * the locks, allowing the caller to contemplate and format the data for as
+  * long as it pleases.
+  */
+ BlockedProcsData *
+ GetBlockerStatusData(int blocked_pid)
+ {
+ 	BlockedProcsData *data;
+ 	PGPROC	   *proc;
+ 	int			i;
+ 
+ 	data = (BlockedProcsData *) palloc(sizeof(BlockedProcsData));
+ 
+ 	/*
+ 	 * Guess how much space we'll need, and preallocate.  Most of the time
+ 	 * this will avoid needing to do repalloc while holding the LWLocks.  (We
+ 	 * assume, but check with an Assert, that MaxBackends is enough entries
+ 	 * for the procs[] array; the other two could need enlargement, though.)
+ 	 */
+ 	data->nprocs = data->nlocks = data->npids = 0;
+ 	data->maxprocs = data->maxlocks = data->maxpids = MaxBackends;
+ 	data->procs = (BlockedProcData *) palloc(sizeof(BlockedProcData) * data->maxprocs);
+ 	data->locks = (LockInstanceData *) palloc(sizeof(LockInstanceData) * data->maxlocks);
+ 	data->waiter_pids = (int *) palloc(sizeof(int) * data->maxpids);
+ 
+ 	/*
+ 	 * In order to search the ProcArray for blocked_pid and assume that that
+ 	 * entry won't immediately disappear under us, we must hold ProcArrayLock.
+ 	 * In addition, to examine the lock grouping fields of any other backend,
+ 	 * we must hold all the hash partition locks.  (Only one of those locks is
+ 	 * actually relevant for any one lock group, but we can't know which one
+ 	 * ahead of time.)	It's fairly annoying to hold all those locks
+ 	 * throughout this, but it's no worse than GetLockStatusData(), and it
+ 	 * does have the advantage that we're guaranteed to return a
+ 	 * self-consistent instantaneous state.
+ 	 */
+ 	LWLockAcquire(ProcArrayLock, LW_SHARED);
+ 
+ 	proc = BackendPidGetProcWithLock(blocked_pid);
+ 
+ 	/* Nothing to do if it's gone */
+ 	if (proc != NULL)
+ 	{
+ 		/*
+ 		 * Acquire lock on the entire shared lock data structure.  See notes
+ 		 * in GetLockStatusData().
+ 		 */
+ 		for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
+ 			LWLockAcquire(LockHashPartitionLockByIndex(i), LW_SHARED);
+ 
+ 		if (proc->lockGroupLeader == NULL)
+ 		{
+ 			/* Easy case, proc is not a lock group member */
+ 			GetSingleProcBlockerStatusData(proc, data);
+ 		}
+ 		else
+ 		{
+ 			/* Examine all procs in proc's lock group */
+ 			dlist_iter	iter;
+ 
+ 			dlist_foreach(iter, &proc->lockGroupLeader->lockGroupMembers)
+ 			{
+ 				PGPROC	   *memberProc;
+ 
+ 				memberProc = dlist_container(PGPROC, lockGroupLink, iter.cur);
+ 				GetSingleProcBlockerStatusData(memberProc, data);
+ 			}
+ 		}
+ 
+ 		/*
+ 		 * And release locks.  See notes in GetLockStatusData().
+ 		 */
+ 		for (i = NUM_LOCK_PARTITIONS; --i >= 0;)
+ 			LWLockRelease(LockHashPartitionLockByIndex(i));
+ 
+ 		Assert(data->nprocs <= data->maxprocs);
+ 	}
+ 
+ 	LWLockRelease(ProcArrayLock);
+ 
+ 	return data;
+ }
+ 
+ /* Accumulate data about one possibly-blocked proc for GetBlockerStatusData */
+ static void
+ GetSingleProcBlockerStatusData(PGPROC *blocked_proc, BlockedProcsData *data)
+ {
+ 	LOCK	   *theLock = blocked_proc->waitLock;
+ 	BlockedProcData *bproc;
+ 	SHM_QUEUE  *procLocks;
+ 	PROCLOCK   *proclock;
+ 	PROC_QUEUE *waitQueue;
+ 	PGPROC	   *proc;
+ 	int			queue_size;
+ 	int			i;
+ 
+ 	/* Nothing to do if this proc is not blocked */
+ 	if (theLock == NULL)
+ 		return;
+ 
+ 	/* Set up a procs[] element */
+ 	bproc = &data->procs[data->nprocs++];
+ 	bproc->pid = blocked_proc->pid;
+ 	bproc->first_lock = data->nlocks;
+ 	bproc->first_waiter = data->npids;
+ 
+ 	/*
+ 	 * We may ignore the proc's fast-path arrays, since nothing in those could
+ 	 * be related to a contended lock.
+ 	 */
+ 
+ 	/* Scan all PROCLOCKs associated with theLock. */
+ 	procLocks = &(theLock->procLocks);
+ 	proclock = (PROCLOCK *) SHMQueueNext(procLocks, procLocks,
+ 										 offsetof(PROCLOCK, lockLink));
+ 	while (proclock)
+ 	{
+ 		PGPROC	   *proc = proclock->tag.myProc;
+ 		LOCK	   *lock = proclock->tag.myLock;
+ 		LockInstanceData *instance;
+ 
+ 		if (data->nlocks >= data->maxlocks)
+ 		{
+ 			data->maxlocks += MaxBackends;
+ 			data->locks = (LockInstanceData *)
+ 				repalloc(data->locks, sizeof(LockInstanceData) * data->maxlocks);
+ 		}
+ 
+ 		instance = &data->locks[data->nlocks];
+ 		memcpy(&instance->locktag, &lock->tag, sizeof(LOCKTAG));
+ 		instance->holdMask = proclock->holdMask;
+ 		if (proc->waitLock == proclock->tag.myLock)
+ 			instance->waitLockMode = proc->waitLockMode;
+ 		else
+ 			instance->waitLockMode = NoLock;
+ 		instance->backend = proc->backendId;
+ 		instance->lxid = proc->lxid;
+ 		instance->pid = proc->pid;
+ 		instance->leaderPid = proclock->groupLeader->pid;
+ 		instance->fastpath = false;
+ 		data->nlocks++;
+ 
+ 		proclock = (PROCLOCK *) SHMQueueNext(procLocks, &proclock->lockLink,
+ 											 offsetof(PROCLOCK, lockLink));
+ 	}
+ 
+ 	/* Now scan the lock's wait queue, stopping at blocked_proc */
+ 	waitQueue = &(theLock->waitProcs);
+ 	queue_size = waitQueue->size;
+ 
+ 	if (queue_size > data->maxpids - data->npids)
+ 	{
+ 		data->maxpids = Max(data->maxpids + MaxBackends,
+ 							data->npids + queue_size);
+ 		data->waiter_pids = (int *) repalloc(data->waiter_pids,
+ 											 sizeof(int) * data->maxpids);
+ 	}
+ 
+ 	proc = (PGPROC *) waitQueue->links.next;
+ 	for (i = 0; i < queue_size; i++)
+ 	{
+ 		if (proc == blocked_proc)
+ 			break;
+ 		data->waiter_pids[data->npids++] = proc->pid;
+ 		proc = (PGPROC *) proc->links.next;
+ 	}
+ 
+ 	bproc->num_locks = data->nlocks - bproc->first_lock;
+ 	bproc->num_waiters = data->npids - bproc->first_waiter;
+ }
+ 
+ /*
   * Returns a list of currently held AccessExclusiveLocks, for use by
   * LogStandbySnapshot().  The result is a palloc'd array,
   * with the number of elements returned into *nlocks.
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index 73c78e9..ad9072d 100644
*** a/src/backend/utils/adt/lockfuncs.c
--- b/src/backend/utils/adt/lockfuncs.c
***************
*** 18,23 ****
--- 18,24 ----
  #include "funcapi.h"
  #include "miscadmin.h"
  #include "storage/predicate_internals.h"
+ #include "utils/array.h"
  #include "utils/builtins.h"
  
  
*************** pg_lock_status(PG_FUNCTION_ARGS)
*** 99,105 ****
  		oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx);
  
  		/* build tupdesc for result tuples */
! 		/* this had better match pg_locks view in system_views.sql */
  		tupdesc = CreateTemplateTupleDesc(NUM_LOCK_STATUS_COLUMNS, false);
  		TupleDescInitEntry(tupdesc, (AttrNumber) 1, "locktype",
  						   TEXTOID, -1, 0);
--- 100,106 ----
  		oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx);
  
  		/* build tupdesc for result tuples */
! 		/* this had better match function's declaration in pg_proc.h */
  		tupdesc = CreateTemplateTupleDesc(NUM_LOCK_STATUS_COLUMNS, false);
  		TupleDescInitEntry(tupdesc, (AttrNumber) 1, "locktype",
  						   TEXTOID, -1, 0);
*************** pg_lock_status(PG_FUNCTION_ARGS)
*** 395,400 ****
--- 396,511 ----
  
  
  /*
+  * pg_blocker_pids - produce an array of the PIDs blocking given PID
+  *
+  * In parallel-query cases, we report all PIDs blocking any member of the
+  * given PID's lock group, and the reported PIDs are always those of lock
+  * group leaders.  We do not bother eliminating duplicates from the result.
+  *
+  * We need not consider predicate locks here, since those don't block anything.
+  */
+ Datum
+ pg_blocker_pids(PG_FUNCTION_ARGS)
+ {
+ 	int			blocked_pid = PG_GETARG_INT32(0);
+ 	Datum	   *arrayelems;
+ 	int			narrayelems;
+ 	BlockedProcsData *lockData; /* state data from lmgr */
+ 	int			i,
+ 				j;
+ 
+ 	/* Collect a snapshot of lock manager state */
+ 	lockData = GetBlockerStatusData(blocked_pid);
+ 
+ 	/* We can't need more output entries than there are reported PROCLOCKs */
+ 	arrayelems = (Datum *) palloc(lockData->nlocks * sizeof(Datum));
+ 	narrayelems = 0;
+ 
+ 	/* For each blocked proc in the lock group ... */
+ 	for (i = 0; i < lockData->nprocs; i++)
+ 	{
+ 		BlockedProcData *bproc = &lockData->procs[i];
+ 		LockInstanceData *instances = &lockData->locks[bproc->first_lock];
+ 		int		   *preceding_waiters = &lockData->waiter_pids[bproc->first_waiter];
+ 		LockInstanceData *blocked_instance;
+ 		LockMethod	lockMethodTable;
+ 		int			conflictMask;
+ 
+ 		/*
+ 		 * Locate the blocked proc's own entry in the LockInstanceData array.
+ 		 * There should be exactly one matching entry.
+ 		 */
+ 		blocked_instance = NULL;
+ 		for (j = 0; j < bproc->num_locks; j++)
+ 		{
+ 			LockInstanceData *instance = &(instances[j]);
+ 
+ 			if (instance->pid == bproc->pid)
+ 			{
+ 				Assert(blocked_instance == NULL);
+ 				blocked_instance = instance;
+ 			}
+ 		}
+ 		Assert(blocked_instance != NULL);
+ 
+ 		lockMethodTable = GetLockTagsMethodTable(&(blocked_instance->locktag));
+ 		conflictMask = lockMethodTable->conflictTab[blocked_instance->waitLockMode];
+ 
+ 		/* Now scan the PROCLOCK data for conflicting procs */
+ 		for (j = 0; j < bproc->num_locks; j++)
+ 		{
+ 			LockInstanceData *instance = &(instances[j]);
+ 
+ 			/* A proc never blocks itself, so ignore that entry */
+ 			if (instance == blocked_instance)
+ 				continue;
+ 			/* Members of same lock group never block each other, either */
+ 			if (instance->leaderPid == blocked_instance->leaderPid)
+ 				continue;
+ 
+ 			if (conflictMask & instance->holdMask)
+ 				 /* blocked by lock already held by this entry */ ;
+ 			else if (instance->waitLockMode != NoLock &&
+ 					 (conflictMask & LOCKBIT_ON(instance->waitLockMode)))
+ 			{
+ 				/* conflict with lock requested by this entry; who's in front? */
+ 				bool		ahead = false;
+ 				int			k;
+ 
+ 				for (k = 0; k < bproc->num_waiters; k++)
+ 				{
+ 					if (preceding_waiters[k] == instance->pid)
+ 					{
+ 						/* this entry is waiting ahead of blocked proc */
+ 						ahead = true;
+ 						break;
+ 					}
+ 				}
+ 				if (!ahead)
+ 					continue;	/* not blocked by this entry */
+ 			}
+ 			else
+ 			{
+ 				/* not blocked by this entry */
+ 				continue;
+ 			}
+ 
+ 			/* blocked by this entry, so emit a record */
+ 			arrayelems[narrayelems++] = Int32GetDatum(instance->leaderPid);
+ 		}
+ 	}
+ 
+ 	/* Assert we didn't overrun arrayelems[] */
+ 	Assert(narrayelems <= lockData->nlocks);
+ 
+ 	/* Construct array, using hardwired knowledge about int4 type */
+ 	PG_RETURN_ARRAYTYPE_P(construct_array(arrayelems, narrayelems,
+ 										  INT4OID,
+ 										  sizeof(int32), true, 'i'));
+ }
+ 
+ 
+ /*
   * Functions for manipulating advisory locks
   *
   * We make use of the locktag fields as follows:
diff --git a/src/include/catalog/pg_proc.h b/src/include/catalog/pg_proc.h
index 1c0ef9a..90a02b3 100644
*** a/src/include/catalog/pg_proc.h
--- b/src/include/catalog/pg_proc.h
*************** DATA(insert OID = 3329 (  pg_show_all_fi
*** 3012,3017 ****
--- 3012,3019 ----
  DESCR("show config file settings");
  DATA(insert OID = 1371 (  pg_lock_status   PGNSP PGUID 12 1 1000 0 0 f f f f t t v s 0 0 2249 "" "{25,26,26,23,21,25,28,26,26,21,25,23,25,16,16}" "{o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}" "{locktype,database,relation,page,tuple,virtualxid,transactionid,classid,objid,objsubid,virtualtransaction,pid,mode,granted,fastpath}" _null_ _null_ pg_lock_status _null_ _null_ _null_ ));
  DESCR("view system lock information");
+ DATA(insert OID = 2561 (  pg_blocker_pids  PGNSP PGUID 12 1 0 0 0 f f f f t f v s 1 0 1007 "23" _null_ _null_ _null_ _null_ _null_ pg_blocker_pids _null_ _null_ _null_ ));
+ DESCR("report PIDs of blockers of specified backend PID, as array");
  DATA(insert OID = 1065 (  pg_prepared_xact PGNSP PGUID 12 1 1000 0 0 f f f f t t v s 0 0 2249 "" "{28,25,1184,26,26}" "{o,o,o,o,o}" "{transaction,gid,prepared,ownerid,dbid}" _null_ _null_ pg_prepared_xact _null_ _null_ _null_ ));
  DESCR("view two-phase transactions");
  DATA(insert OID = 3819 (  pg_get_multixact_members PGNSP PGUID 12 1 1000 0 0 f f f f t t v s 1 0 2249 "28" "{28,28,25}" "{i,o,o}" "{multixid,xid,mode}" _null_ _null_ pg_get_multixact_members _null_ _null_ _null_ ));
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 6b4e365..2ab5aa9 100644
*** a/src/include/storage/lock.h
--- b/src/include/storage/lock.h
*************** typedef struct PROCLOCK
*** 346,352 ****
  	PROCLOCKTAG tag;			/* unique identifier of proclock object */
  
  	/* data */
! 	PGPROC	   *groupLeader;	/* group leader, or NULL if no lock group */
  	LOCKMASK	holdMask;		/* bitmask for lock types currently held */
  	LOCKMASK	releaseMask;	/* bitmask for lock types to be released */
  	SHM_QUEUE	lockLink;		/* list link in LOCK's list of proclocks */
--- 346,352 ----
  	PROCLOCKTAG tag;			/* unique identifier of proclock object */
  
  	/* data */
! 	PGPROC	   *groupLeader;	/* proc's lock group leader, or proc itself */
  	LOCKMASK	holdMask;		/* bitmask for lock types currently held */
  	LOCKMASK	releaseMask;	/* bitmask for lock types to be released */
  	SHM_QUEUE	lockLink;		/* list link in LOCK's list of proclocks */
*************** typedef struct LOCALLOCK
*** 423,443 ****
  
  typedef struct LockInstanceData
  {
! 	LOCKTAG		locktag;		/* locked object */
  	LOCKMASK	holdMask;		/* locks held by this PGPROC */
  	LOCKMODE	waitLockMode;	/* lock awaited by this PGPROC, if any */
  	BackendId	backend;		/* backend ID of this PGPROC */
  	LocalTransactionId lxid;	/* local transaction ID of this PGPROC */
  	int			pid;			/* pid of this PGPROC */
  	bool		fastpath;		/* taken via fastpath? */
  } LockInstanceData;
  
  typedef struct LockData
  {
  	int			nelements;		/* The length of the array */
! 	LockInstanceData *locks;
  } LockData;
  
  
  /* Result codes for LockAcquire() */
  typedef enum
--- 423,470 ----
  
  typedef struct LockInstanceData
  {
! 	LOCKTAG		locktag;		/* tag for locked object */
  	LOCKMASK	holdMask;		/* locks held by this PGPROC */
  	LOCKMODE	waitLockMode;	/* lock awaited by this PGPROC, if any */
  	BackendId	backend;		/* backend ID of this PGPROC */
  	LocalTransactionId lxid;	/* local transaction ID of this PGPROC */
  	int			pid;			/* pid of this PGPROC */
+ 	int			leaderPid;		/* pid of group leader; = pid if no group */
  	bool		fastpath;		/* taken via fastpath? */
  } LockInstanceData;
  
  typedef struct LockData
  {
  	int			nelements;		/* The length of the array */
! 	LockInstanceData *locks;	/* Array of per-PROCLOCK information */
  } LockData;
  
+ typedef struct BlockedProcData
+ {
+ 	int			pid;			/* pid of a blocked PGPROC */
+ 	/* Per-PROCLOCK information about PROCLOCKs of the lock the pid awaits */
+ 	/* (these fields refer to indexes in BlockedProcsData.locks[]) */
+ 	int			first_lock;		/* index of first relevant LockInstanceData */
+ 	int			num_locks;		/* number of relevant LockInstanceDatas */
+ 	/* PIDs of PGPROCs that are ahead of "pid" in the lock's wait queue */
+ 	/* (these fields refer to indexes in BlockedProcsData.waiter_pids[]) */
+ 	int			first_waiter;	/* index of first preceding waiter */
+ 	int			num_waiters;	/* number of preceding waiters */
+ } BlockedProcData;
+ 
+ typedef struct BlockedProcsData
+ {
+ 	BlockedProcData *procs;		/* Array of per-blocked-proc information */
+ 	LockInstanceData *locks;	/* Array of per-PROCLOCK information */
+ 	int		   *waiter_pids;	/* Array of PIDs of other blocked PGPROCs */
+ 	int			nprocs;			/* # of valid entries in procs[] array */
+ 	int			maxprocs;		/* Allocated length of procs[] array */
+ 	int			nlocks;			/* # of valid entries in locks[] array */
+ 	int			maxlocks;		/* Allocated length of locks[] array */
+ 	int			npids;			/* # of valid entries in waiter_pids[] array */
+ 	int			maxpids;		/* Allocated length of waiter_pids[] array */
+ } BlockedProcsData;
+ 
  
  /* Result codes for LockAcquire() */
  typedef enum
*************** typedef enum
*** 488,493 ****
--- 515,521 ----
   */
  extern void InitLocks(void);
  extern LockMethod GetLocksMethodTable(const LOCK *lock);
+ extern LockMethod GetLockTagsMethodTable(const LOCKTAG *locktag);
  extern uint32 LockTagHashCode(const LOCKTAG *locktag);
  extern bool DoLockModesConflict(LOCKMODE mode1, LOCKMODE mode2);
  extern LockAcquireResult LockAcquire(const LOCKTAG *locktag,
*************** extern void GrantAwaitedLock(void);
*** 520,525 ****
--- 548,554 ----
  extern void RemoveFromWaitQueue(PGPROC *proc, uint32 hashcode);
  extern Size LockShmemSize(void);
  extern LockData *GetLockStatusData(void);
+ extern BlockedProcsData *GetBlockerStatusData(int blocked_pid);
  
  extern xl_standby_lock *GetRunningTransactionLocks(int *nlocks);
  extern const char *GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 1fbf4f3..dd37c0c 100644
*** a/src/include/storage/procarray.h
--- b/src/include/storage/procarray.h
*************** extern VirtualTransactionId *GetVirtualX
*** 61,66 ****
--- 61,67 ----
  extern bool HaveVirtualXIDsDelayingChkpt(VirtualTransactionId *vxids, int nvxids);
  
  extern PGPROC *BackendPidGetProc(int pid);
+ extern PGPROC *BackendPidGetProcWithLock(int pid);
  extern int	BackendXidGetPid(TransactionId xid);
  extern bool IsBackendPid(int pid);
  
diff --git a/src/include/utils/builtins.h b/src/include/utils/builtins.h
index affcc01..4848cbd 100644
*** a/src/include/utils/builtins.h
--- b/src/include/utils/builtins.h
*************** extern Datum row_security_active_name(PG
*** 1153,1158 ****
--- 1153,1159 ----
  
  /* lockfuncs.c */
  extern Datum pg_lock_status(PG_FUNCTION_ARGS);
+ extern Datum pg_blocker_pids(PG_FUNCTION_ARGS);
  extern Datum pg_advisory_lock_int8(PG_FUNCTION_ARGS);
  extern Datum pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS);
  extern Datum pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS);
diff --git a/src/test/isolation/isolationtester.c b/src/test/isolation/isolationtester.c
index 0a9d25c..a85d24e 100644
*** a/src/test/isolation/isolationtester.c
--- b/src/test/isolation/isolationtester.c
*************** main(int argc, char **argv)
*** 227,253 ****
  	 */
  	initPQExpBuffer(&wait_query);
  	appendPQExpBufferStr(&wait_query,
! 						 "SELECT 1 FROM pg_locks holder, pg_locks waiter "
! 						 "WHERE NOT waiter.granted AND waiter.pid = $1 "
! 						 "AND holder.granted "
! 						 "AND holder.pid <> $1 AND holder.pid IN (");
  	/* The spec syntax requires at least one session; assume that here. */
  	appendPQExpBufferStr(&wait_query, backend_pids[1]);
  	for (i = 2; i < nconns; i++)
! 		appendPQExpBuffer(&wait_query, ", %s", backend_pids[i]);
! 	appendPQExpBufferStr(&wait_query,
! 						 ") "
! 
! 				  "AND holder.locktype IS NOT DISTINCT FROM waiter.locktype "
! 				  "AND holder.database IS NOT DISTINCT FROM waiter.database "
! 				  "AND holder.relation IS NOT DISTINCT FROM waiter.relation "
! 						 "AND holder.page IS NOT DISTINCT FROM waiter.page "
! 						 "AND holder.tuple IS NOT DISTINCT FROM waiter.tuple "
! 			  "AND holder.virtualxid IS NOT DISTINCT FROM waiter.virtualxid "
! 		"AND holder.transactionid IS NOT DISTINCT FROM waiter.transactionid "
! 					"AND holder.classid IS NOT DISTINCT FROM waiter.classid "
! 						 "AND holder.objid IS NOT DISTINCT FROM waiter.objid "
! 				"AND holder.objsubid IS NOT DISTINCT FROM waiter.objsubid ");
  
  	res = PQprepare(conns[0], PREP_WAITING, wait_query.data, 0, NULL);
  	if (PQresultStatus(res) != PGRES_COMMAND_OK)
--- 227,238 ----
  	 */
  	initPQExpBuffer(&wait_query);
  	appendPQExpBufferStr(&wait_query,
! 						 "SELECT pg_catalog.pg_blocker_pids($1) && '{");
  	/* The spec syntax requires at least one session; assume that here. */
  	appendPQExpBufferStr(&wait_query, backend_pids[1]);
  	for (i = 2; i < nconns; i++)
! 		appendPQExpBuffer(&wait_query, ",%s", backend_pids[i]);
! 	appendPQExpBufferStr(&wait_query, "}'::integer[]");
  
  	res = PQprepare(conns[0], PREP_WAITING, wait_query.data, 0, NULL);
  	if (PQresultStatus(res) != PGRES_COMMAND_OK)
*************** try_complete_step(Step *step, int flags)
*** 745,765 ****
  			/* If it's OK for the step to block, check whether it has. */
  			if (flags & STEP_NONBLOCK)
  			{
! 				int			ntuples;
  
  				res = PQexecPrepared(conns[0], PREP_WAITING, 1,
  									 &backend_pids[step->session + 1],
  									 NULL, NULL, 0);
! 				if (PQresultStatus(res) != PGRES_TUPLES_OK)
  				{
  					fprintf(stderr, "lock wait query failed: %s",
  							PQerrorMessage(conn));
  					exit_nicely();
  				}
! 				ntuples = PQntuples(res);
  				PQclear(res);
  
! 				if (ntuples >= 1)		/* waiting to acquire a lock */
  				{
  					if (!(flags & STEP_RETRY))
  						printf("step %s: %s <waiting ...>\n",
--- 730,751 ----
  			/* If it's OK for the step to block, check whether it has. */
  			if (flags & STEP_NONBLOCK)
  			{
! 				bool		waiting;
  
  				res = PQexecPrepared(conns[0], PREP_WAITING, 1,
  									 &backend_pids[step->session + 1],
  									 NULL, NULL, 0);
! 				if (PQresultStatus(res) != PGRES_TUPLES_OK ||
! 					PQntuples(res) != 1)
  				{
  					fprintf(stderr, "lock wait query failed: %s",
  							PQerrorMessage(conn));
  					exit_nicely();
  				}
! 				waiting = ((PQgetvalue(res, 0, 0))[0] == 't');
  				PQclear(res);
  
! 				if (waiting)	/* waiting to acquire a lock */
  				{
  					if (!(flags & STEP_RETRY))
  						printf("step %s: %s <waiting ...>\n",
-- 
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Reply via email to