aboutsummaryrefslogtreecommitdiff
path: root/src/backend/storage/lmgr/lock.c
diff options
context:
space:
mode:
authorTom Lane <tgl@sss.pgh.pa.us>2016-02-22 14:31:43 -0500
committerTom Lane <tgl@sss.pgh.pa.us>2016-02-22 14:31:43 -0500
commit52f5d578d6c29bf254e93c69043b817d4047ca67 (patch)
tree5c983046681537c41773cf3e81a7461b9dc8551d /src/backend/storage/lmgr/lock.c
parent73bf8715aa7430bd003516bde448507fbe789c05 (diff)
downloadpostgresql-52f5d578d6c29bf254e93c69043b817d4047ca67.tar.gz
postgresql-52f5d578d6c29bf254e93c69043b817d4047ca67.zip
Create a function to reliably identify which sessions block which others.
This patch introduces "pg_blocking_pids(int) returns int[]", which returns the PIDs of any sessions that are blocking the session with the given PID. Historically people have obtained such information using a self-join on the pg_locks view, but it's unreasonably tedious to do it that way with any modicum of correctness, and the addition of parallel queries has pretty much broken that approach altogether. (Given some more columns in the view than there are today, you could imagine handling parallel-query cases with a 4-way join; but ugh.) The new function has the following behaviors that are painful or impossible to get right via pg_locks: 1. Correctly understands which lock modes block which other ones. 2. In soft-block situations (two processes both waiting for conflicting lock modes), only the one that's in front in the wait queue is reported to block the other. 3. In parallel-query cases, reports all sessions blocking any member of the given PID's lock group, and reports a session by naming its leader process's PID, which will be the pg_backend_pid() value visible to clients. The motivation for doing this right now is mostly to fix the isolation tests. Commit 38f8bdcac4982215beb9f65a19debecaf22fd470 lobotomized isolationtester's is-it-waiting query by removing its ability to recognize nonconflicting lock modes, as a crude workaround for the inability to handle soft-block situations properly. But even without the lock mode tests, the old query was excessively slow, particularly in CLOBBER_CACHE_ALWAYS builds; some of our buildfarm animals fail the new deadlock-hard test because the deadlock timeout elapses before they can probe the waiting status of all eight sessions. Replacing the pg_locks self-join with use of pg_blocking_pids() is not only much more correct, but a lot faster: I measure it at about 9X faster in a typical dev build with Asserts, and 3X faster in CLOBBER_CACHE_ALWAYS builds. That should provide enough headroom for the slower CLOBBER_CACHE_ALWAYS animals to pass the test, without having to lengthen deadlock_timeout yet more and thus slow down the test for everyone else.
Diffstat (limited to 'src/backend/storage/lmgr/lock.c')
-rw-r--r--src/backend/storage/lmgr/lock.c216
1 files changed, 214 insertions, 2 deletions
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index fef59a280a6..a458c68b9e9 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -21,7 +21,7 @@
*
* Interface:
*
- * InitLocks(), GetLocksMethodTable(),
+ * InitLocks(), GetLocksMethodTable(), GetLockTagsMethodTable(),
* LockAcquire(), LockRelease(), LockReleaseAll(),
* LockCheckConflicts(), GrantLock()
*
@@ -41,6 +41,7 @@
#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"
@@ -356,6 +357,8 @@ static void CleanUpLock(LOCK *lock, PROCLOCK *proclock,
static void LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
LOCKTAG *locktag, LOCKMODE lockmode,
bool decrement_strong_lock_count);
+static void GetSingleProcBlockerStatusData(PGPROC *blocked_proc,
+ BlockedProcsData *data);
/*
@@ -462,6 +465,18 @@ GetLocksMethodTable(const LOCK *lock)
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.
@@ -3406,7 +3421,10 @@ GetLockStatusData(void)
* 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.
+ * 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)
{
@@ -3439,6 +3457,7 @@ GetLockStatusData(void)
instance->backend = proc->backendId;
instance->lxid = proc->lxid;
instance->pid = proc->pid;
+ instance->leaderPid = proc->pid;
instance->fastpath = true;
el++;
@@ -3466,6 +3485,7 @@ GetLockStatusData(void)
instance->backend = proc->backendId;
instance->lxid = proc->lxid;
instance->pid = proc->pid;
+ instance->leaderPid = proc->pid;
instance->fastpath = true;
el++;
@@ -3517,6 +3537,7 @@ GetLockStatusData(void)
instance->backend = proc->backendId;
instance->lxid = proc->lxid;
instance->pid = proc->pid;
+ instance->leaderPid = proclock->groupLeader->pid;
instance->fastpath = false;
el++;
@@ -3538,6 +3559,197 @@ GetLockStatusData(void)
}
/*
+ * 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.
+ */
+
+ /* Collect 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 == lock)
+ 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));
+ }
+
+ /* Enlarge waiter_pids[] if it's too small to hold all wait queue PIDs */
+ 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);
+ }
+
+ /* Collect PIDs from the lock's wait queue, stopping at blocked_proc */
+ 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.