Re: Re: [COMMITTERS] pgsql: Add some isolation tests for deadlock detection and resolution.
| От | Tom Lane |
|---|---|
| Тема | Re: Re: [COMMITTERS] pgsql: Add some isolation tests for deadlock detection and resolution. |
| Дата | |
| Msg-id | 11103.1456089966@sss.pgh.pa.us обсуждение исходный текст |
| Ответ на | Re: Re: [COMMITTERS] pgsql: Add some isolation tests for deadlock detection and resolution. (Tom Lane <tgl@sss.pgh.pa.us>) |
| Ответы |
Re: Re: [COMMITTERS] pgsql: Add some isolation tests for
deadlock detection and resolution.
|
| Список | pgsql-hackers |
I wrote:
> Robert Haas <robertmhaas@gmail.com> writes:
>> As for the patch itself, I'm having trouble grokking what it's trying
>> to do. I think it might be worth having a comment defining precisely
>> what we mean by "A blocks B". I would define "A blocks B" in general
>> as either A holds a lock which conflicts with one sought by B
>> (hard-blocked) or A awaits a lock which conflicts with one sought by B
>> and precedes it in the wait queue (soft-blocked).
> Yes, that is exactly what I implemented ... and it's something you can't
> find out from pg_locks. I'm not sure how that view could be made to
> expose wait-queue ordering.
Here's an updated version of this patch, now with user-facing docs.
I decided that "pg_blocking_pids()" is a better function name than
"pg_blocker_pids()". The code's otherwise the same, although I
revisited some of the comments.
I also changed quite a few references to "transaction" into "process"
in the discussion of pg_locks. The previous choice to conflate
processes with transactions was never terribly wise in my view, and
it's certainly completely broken by parallel query.
regards, tom lane
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index d77e999..d3270e4 100644
*** a/doc/src/sgml/catalogs.sgml
--- b/doc/src/sgml/catalogs.sgml
***************
*** 8015,8030 ****
<para>
The view <structname>pg_locks</structname> provides access to
! information about the locks held by open transactions within the
database server. See <xref linkend="mvcc"> for more discussion
of locking.
</para>
<para>
<structname>pg_locks</structname> contains one row per active lockable
! object, requested lock mode, and relevant transaction. Thus, the same
lockable object might
! appear many times, if multiple transactions are holding or waiting
for locks on it. However, an object that currently has no locks on it
will not appear at all.
</para>
--- 8015,8030 ----
<para>
The view <structname>pg_locks</structname> provides access to
! information about the locks held by active processes within the
database server. See <xref linkend="mvcc"> for more discussion
of locking.
</para>
<para>
<structname>pg_locks</structname> contains one row per active lockable
! object, requested lock mode, and relevant process. Thus, the same
lockable object might
! appear many times, if multiple processs are holding or waiting
for locks on it. However, an object that currently has no locks on it
will not appear at all.
</para>
***************
*** 8200,8210 ****
<para>
<structfield>granted</structfield> is true in a row representing a lock
! held by the indicated transaction. False indicates that this transaction is
! currently waiting to acquire this lock, which implies that some other
! transaction is holding a conflicting lock mode on the same lockable object.
! The waiting transaction will sleep until the other lock is released (or a
! deadlock situation is detected). A single transaction can be waiting to
acquire at most one lock at a time.
</para>
--- 8200,8210 ----
<para>
<structfield>granted</structfield> is true in a row representing a lock
! held by the indicated process. False indicates that this process is
! currently waiting to acquire this lock, which implies that at least one other
! process is holding a conflicting lock mode on the same lockable object.
! The waiting process will sleep until the other lock is released (or a
! deadlock situation is detected). A single process can be waiting to
acquire at most one lock at a time.
</para>
***************
*** 8224,8230 ****
Although tuples are a lockable type of object,
information about row-level locks is stored on disk, not in memory,
and therefore row-level locks normally do not appear in this view.
! If a transaction is waiting for a
row-level lock, it will usually appear in the view as waiting for the
permanent transaction ID of the current holder of that row lock.
</para>
--- 8224,8230 ----
Although tuples are a lockable type of object,
information about row-level locks is stored on disk, not in memory,
and therefore row-level locks normally do not appear in this view.
! If a process is waiting for a
row-level lock, it will usually appear in the view as waiting for the
permanent transaction ID of the current holder of that row lock.
</para>
*************** SELECT * FROM pg_locks pl LEFT JOIN pg_p
*** 8281,8286 ****
--- 8281,8300 ----
</para>
<para>
+ While it is possible to obtain information about which processes block
+ which other processes by joining <structname>pg_locks</structname> against
+ itself, this is very difficult to get right in detail. Such a query would
+ have to encode knowledge about which lock modes conflict with which
+ others. Worse, the <structname>pg_locks</structname> view does not expose
+ information about which processes are ahead of which others in lock wait
+ queues, nor information about which processes are parallel workers running
+ on behalf of which other client sessions. It is better to use
+ the <function>pg_blocking_pids()</> function
+ (see <xref linkend="functions-info-session-table">) to identify which
+ process(es) a waiting process is blocked behind.
+ </para>
+
+ <para>
The <structname>pg_locks</structname> view displays data from both the
regular lock manager and the predicate lock manager, which are
separate systems; in addition, the regular lock manager subdivides its
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index b001ce5..2d9fe7f 100644
*** a/doc/src/sgml/func.sgml
--- b/doc/src/sgml/func.sgml
*************** SELECT * FROM pg_ls_dir('.') WITH ORDINA
*** 14997,15002 ****
--- 14997,15008 ----
</row>
<row>
+ <entry><literal><function>pg_blocking_pids(<type>int</type>)</function></literal></entry>
+ <entry><type>int[]</type></entry>
+ <entry>Process ID(s) that are blocking specified server process ID</entry>
+ </row>
+
+ <row>
<entry><literal><function>pg_conf_load_time()</function></literal></entry>
<entry><type>timestamp with time zone</type></entry>
<entry>configuration load time</entry>
*************** SET search_path TO <replaceable>schema</
*** 15184,15189 ****
--- 15190,15215 ----
</para>
<indexterm>
+ <primary>pg_blocking_pids</primary>
+ </indexterm>
+
+ <para>
+ <function>pg_blocking_pids</function> returns an array of the process IDs
+ of the sessions that are blocking the server process with the specified
+ process ID, or an empty array if there is no such server process or it is
+ not blocked. One server process blocks another if it either holds a lock
+ that conflicts with the blocked process's lock request (hard block), or is
+ waiting for a lock that would conflict with the blocked process's lock
+ request and is ahead of it in the wait queue (soft block). When using
+ parallel queries the result always lists client-visible process IDs (that
+ is, <function>pg_backend_pid</> results) even if the actual lock is held
+ or awaited by a child worker process. As a result of that, there may be
+ duplicated PIDs in the result. Also note that when a prepared transaction
+ holds a conflicting lock, it will be represented by a zero process ID in
+ the result of this function.
+ </para>
+
+ <indexterm>
<primary>pg_conf_load_time</primary>
</indexterm>
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..9eb4dfb 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,3755 ----
}
/*
+ * 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.
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index 73c78e9..6bcab81 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,523 ----
/*
+ * pg_blocking_pids - produce an array of the PIDs blocking given PID
+ *
+ * The reported PIDs are those that hold a lock conflicting with blocked_pid's
+ * current request (hard block), or are requesting such a lock and are ahead
+ * of blocked_pid in the lock's wait queue (soft block).
+ *
+ * In parallel-query cases, we report all PIDs blocking any member of the
+ * given PID's lock group, and the reported PIDs are those of the blocking
+ * PIDs' lock group leaders. This allows callers to compare the result to
+ * lists of clients' pg_backend_pid() results even during a parallel query.
+ *
+ * Parallel query makes it possible for there to be duplicate PIDs in the
+ * result (either because multiple waiters are blocked by same PID, or
+ * because multiple blockers have same group leader PID). We do not bother
+ * to eliminate such duplicates from the result.
+ *
+ * We need not consider predicate locks here, since those don't block anything.
+ */
+ Datum
+ pg_blocking_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)
+ {
+ /* hard block: blocked by lock already held by this entry */
+ }
+ else if (instance->waitLockMode != NoLock &&
+ (conflictMask & LOCKBIT_ON(instance->waitLockMode)))
+ {
+ /* conflict in lock requests; who's in front in wait queue? */
+ bool ahead = false;
+ int k;
+
+ for (k = 0; k < bproc->num_waiters; k++)
+ {
+ if (preceding_waiters[k] == instance->pid)
+ {
+ /* soft block: this entry is 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 59c50d9..62b9125 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_blocking_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_blocking_pids _null_ _null_ _null_ ));
+ DESCR("get array of PIDs of sessions blocking specified backend PID");
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 703eaf2..788d50a 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
*** 489,494 ****
--- 516,522 ----
*/
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);
*** 521,526 ****
--- 549,555 ----
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 94c1881..7ec93c9 100644
*** a/src/include/utils/builtins.h
--- b/src/include/utils/builtins.h
*************** extern Datum row_security_active_name(PG
*** 1157,1162 ****
--- 1157,1163 ----
/* lockfuncs.c */
extern Datum pg_lock_status(PG_FUNCTION_ARGS);
+ extern Datum pg_blocking_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..6461ae8 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_blocking_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",
В списке pgsql-hackers по дате отправления: