Commit a1c1af2a authored by Robert Haas's avatar Robert Haas

Introduce group locking to prevent parallel processes from deadlocking.

For locking purposes, we now regard heavyweight locks as mutually
non-conflicting between cooperating parallel processes.  There are some
possible pitfalls to this approach that are not to be taken lightly,
but it works OK for now and can be changed later if we find a better
approach.  Without this, it's very easy for parallel queries to
silently self-deadlock if the user backend holds strong relation locks.

Robert Haas, with help from Amit Kapila.  Thanks to Noah Misch and
Andres Freund for extensive discussion of possible issues with this
approach.
parent aa2387e2
......@@ -432,6 +432,9 @@ LaunchParallelWorkers(ParallelContext *pcxt)
if (pcxt->nworkers == 0)
return;
/* We need to be a lock group leader. */
BecomeLockGroupLeader();
/* If we do have workers, we'd better have a DSM segment. */
Assert(pcxt->seg != NULL);
......@@ -951,6 +954,19 @@ ParallelWorkerMain(Datum main_arg)
* backend-local state to match the original backend.
*/
/*
* Join locking group. We must do this before anything that could try
* to acquire a heavyweight lock, because any heavyweight locks acquired
* to this point could block either directly against the parallel group
* leader or against some process which in turn waits for a lock that
* conflicts with the parallel group leader, causing an undetected
* deadlock. (If we can't join the lock group, the leader has gone away,
* so just exit quietly.)
*/
if (!BecomeLockGroupMember(fps->parallel_master_pgproc,
fps->parallel_master_pid))
return;
/*
* Load libraries that were loaded by original backend. We want to do
* this before restoring GUCs, because the libraries might define custom
......
......@@ -586,6 +586,69 @@ The caller can then send a cancellation signal. This implements the
principle that autovacuum has a low locking priority (eg it must not block
DDL on the table).
Group Locking
-------------
As if all of that weren't already complicated enough, PostgreSQL now supports
parallelism (see src/backend/access/transam/README.parallel), which means that
we might need to resolve deadlocks that occur between gangs of related processes
rather than individual processes. This doesn't change the basic deadlock
detection algorithm very much, but it makes the bookkeeping more complicated.
We choose to regard locks held by processes in the same parallel group as
non-conflicting. This means that two processes in a parallel group can hold
a self-exclusive lock on the same relation at the same time, or one process
can acquire an AccessShareLock while the other already holds AccessExclusiveLock.
This might seem dangerous and could be in some cases (more on that below), but
if we didn't do this then parallel query would be extremely prone to
self-deadlock. For example, a parallel query against a relation on which the
leader had already AccessExclusiveLock would hang, because the workers would
try to lock the same relation and be blocked by the leader; yet the leader can't
finish until it receives completion indications from all workers. An undetected
deadlock results. This is far from the only scenario where such a problem
happens. The same thing will occur if the leader holds only AccessShareLock,
the worker seeks AccessShareLock, but between the time the leader attempts to
acquire the lock and the time the worker attempts to acquire it, some other
process queues up waiting for an AccessExclusiveLock. In this case, too, an
indefinite hang results.
It might seem that we could predict which locks the workers will attempt to
acquire and ensure before going parallel that those locks would be acquired
successfully. But this is very difficult to make work in a general way. For
example, a parallel worker's portion of the query plan could involve an
SQL-callable function which generates a query dynamically, and that query
might happen to hit a table on which the leader happens to hold
AccessExcusiveLock. By imposing enough restrictions on what workers can do,
we could eventually create a situation where their behavior can be adequately
restricted, but these restrictions would be fairly onerous, and even then, the
system required to decide whether the workers will succeed at acquiring the
necessary locks would be complex and possibly buggy.
So, instead, we take the approach of deciding that locks within a lock group
do not conflict. This eliminates the possibility of an undetected deadlock,
but also opens up some problem cases: if the leader and worker try to do some
operation at the same time which would ordinarily be prevented by the heavyweight
lock mechanism, undefined behavior might result. In practice, the dangers are
modest. The leader and worker share the same transaction, snapshot, and combo
CID hash, and neither can perform any DDL or, indeed, write any data at all.
Thus, for either to read a table locked exclusively by the other is safe enough.
Problems would occur if the leader initiated parallelism from a point in the
code at which it had some backend-private state that made table access from
another process unsafe, for example after calling SetReindexProcessing and
before calling ResetReindexProcessing, catastrophe could ensue, because the
worker won't have that state. Similarly, problems could occur with certain
kinds of non-relation locks, such as relation extension locks. It's no safer
for two related processes to extend the same relation at the time than for
unrelated processes to do the same. However, since parallel mode is strictly
read-only at present, neither this nor most of the similar cases can arise at
present. To allow parallel writes, we'll either need to (1) further enhance
the deadlock detector to handle those types of locks in a different way than
other types; or (2) have parallel workers use some other mutual exclusion
method for such cases; or (3) revise those cases so that they no longer use
heavyweight locking in the first place (which is not a crazy idea, given that
such lock acquisitions are not expected to deadlock and that heavyweight lock
acquisition is fairly slow anyway).
User Locks (Advisory Locks)
---------------------------
......
This diff is collapsed.
......@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
......@@ -1136,6 +1137,18 @@ SetupLockInTable(LockMethod lockMethodTable, PGPROC *proc,
{
uint32 partition = LockHashPartition(hashcode);
/*
* It might seem unsafe to access proclock->groupLeader without a lock,
* but it's not really. Either we are initializing a proclock on our
* own behalf, in which case our group leader isn't changing because
* the group leader for a process can only ever be changed by the
* process itself; or else we are transferring a fast-path lock to the
* main lock table, in which case that process can't change it's lock
* group leader without first releasing all of its locks (and in
* particular the one we are currently transferring).
*/
proclock->groupLeader = proc->lockGroupLeader != NULL ?
proc->lockGroupLeader : proc;
proclock->holdMask = 0;
proclock->releaseMask = 0;
/* Add proclock to appropriate lists */
......@@ -1255,9 +1268,10 @@ RemoveLocalLock(LOCALLOCK *locallock)
* NOTES:
* Here's what makes this complicated: one process's locks don't
* conflict with one another, no matter what purpose they are held for
* (eg, session and transaction locks do not conflict).
* So, we must subtract off our own locks when determining whether the
* requested new lock conflicts with those already held.
* (eg, session and transaction locks do not conflict). Nor do the locks
* of one process in a lock group conflict with those of another process in
* the same group. So, we must subtract off these locks when determining
* whether the requested new lock conflicts with those already held.
*/
int
LockCheckConflicts(LockMethod lockMethodTable,
......@@ -1267,8 +1281,12 @@ LockCheckConflicts(LockMethod lockMethodTable,
{
int numLockModes = lockMethodTable->numLockModes;
LOCKMASK myLocks;
LOCKMASK otherLocks;
int conflictMask = lockMethodTable->conflictTab[lockmode];
int conflictsRemaining[MAX_LOCKMODES];
int totalConflictsRemaining = 0;
int i;
SHM_QUEUE *procLocks;
PROCLOCK *otherproclock;
/*
* first check for global conflicts: If no locks conflict with my request,
......@@ -1279,40 +1297,91 @@ LockCheckConflicts(LockMethod lockMethodTable,
* type of lock that conflicts with request. Bitwise compare tells if
* there is a conflict.
*/
if (!(lockMethodTable->conflictTab[lockmode] & lock->grantMask))
if (!(conflictMask & lock->grantMask))
{
PROCLOCK_PRINT("LockCheckConflicts: no conflict", proclock);
return STATUS_OK;
}
/*
* Rats. Something conflicts. But it could still be my own lock. We have
* to construct a conflict mask that does not reflect our own locks, but
* only lock types held by other processes.
* Rats. Something conflicts. But it could still be my own lock, or
* a lock held by another member of my locking group. First, figure out
* how many conflicts remain after subtracting out any locks I hold
* myself.
*/
myLocks = proclock->holdMask;
otherLocks = 0;
for (i = 1; i <= numLockModes; i++)
{
int myHolding = (myLocks & LOCKBIT_ON(i)) ? 1 : 0;
if ((conflictMask & LOCKBIT_ON(i)) == 0)
{
conflictsRemaining[i] = 0;
continue;
}
conflictsRemaining[i] = lock->granted[i];
if (myLocks & LOCKBIT_ON(i))
--conflictsRemaining[i];
totalConflictsRemaining += conflictsRemaining[i];
}
if (lock->granted[i] > myHolding)
otherLocks |= LOCKBIT_ON(i);
/* If no conflicts remain, we get the lock. */
if (totalConflictsRemaining == 0)
{
PROCLOCK_PRINT("LockCheckConflicts: resolved (simple)", proclock);
return STATUS_OK;
}
/* If no group locking, it's definitely a conflict. */
if (proclock->groupLeader == MyProc && MyProc->lockGroupLeader == NULL)
{
Assert(proclock->tag.myProc == MyProc);
PROCLOCK_PRINT("LockCheckConflicts: conflicting (simple)",
proclock);
return STATUS_FOUND;
}
/*
* now check again for conflicts. 'otherLocks' describes the types of
* locks held by other processes. If one of these conflicts with the kind
* of lock that I want, there is a conflict and I have to sleep.
* Locks held in conflicting modes by members of our own lock group are
* not real conflicts; we can subtract those out and see if we still have
* a conflict. This is O(N) in the number of processes holding or awaiting
* locks on this object. We could improve that by making the shared memory
* state more complex (and larger) but it doesn't seem worth it.
*/
if (!(lockMethodTable->conflictTab[lockmode] & otherLocks))
procLocks = &(lock->procLocks);
otherproclock = (PROCLOCK *)
SHMQueueNext(procLocks, procLocks, offsetof(PROCLOCK, lockLink));
while (otherproclock != NULL)
{
/* no conflict. OK to get the lock */
PROCLOCK_PRINT("LockCheckConflicts: resolved", proclock);
return STATUS_OK;
if (proclock != otherproclock &&
proclock->groupLeader == otherproclock->groupLeader &&
(otherproclock->holdMask & conflictMask) != 0)
{
int intersectMask = otherproclock->holdMask & conflictMask;
for (i = 1; i <= numLockModes; i++)
{
if ((intersectMask & LOCKBIT_ON(i)) != 0)
{
if (conflictsRemaining[i] <= 0)
elog(PANIC, "proclocks held do not match lock");
conflictsRemaining[i]--;
totalConflictsRemaining--;
}
}
if (totalConflictsRemaining == 0)
{
PROCLOCK_PRINT("LockCheckConflicts: resolved (group)",
proclock);
return STATUS_OK;
}
}
otherproclock = (PROCLOCK *)
SHMQueueNext(procLocks, &otherproclock->lockLink,
offsetof(PROCLOCK, lockLink));
}
PROCLOCK_PRINT("LockCheckConflicts: conflicting", proclock);
/* Nope, it's a real conflict. */
PROCLOCK_PRINT("LockCheckConflicts: conflicting (group)", proclock);
return STATUS_FOUND;
}
......@@ -3095,6 +3164,10 @@ PostPrepare_Locks(TransactionId xid)
PROCLOCKTAG proclocktag;
int partition;
/* Can't prepare a lock group follower. */
Assert(MyProc->lockGroupLeader == NULL ||
MyProc->lockGroupLeader == MyProc);
/* This is a critical section: any error means big trouble */
START_CRIT_SECTION();
......@@ -3238,6 +3311,13 @@ PostPrepare_Locks(TransactionId xid)
proclocktag.myLock = lock;
proclocktag.myProc = newproc;
/*
* Update groupLeader pointer to point to the new proc. (We'd
* better not be a member of somebody else's lock group!)
*/
Assert(proclock->groupLeader == proclock->tag.myProc);
proclock->groupLeader = newproc;
/*
* Update the proclock. We should not find any existing entry for
* the same hash key, since there can be only one entry for any
......@@ -3785,6 +3865,8 @@ lock_twophase_recover(TransactionId xid, uint16 info,
*/
if (!found)
{
Assert(proc->lockGroupLeader == NULL);
proclock->groupLeader = proc;
proclock->holdMask = 0;
proclock->releaseMask = 0;
/* Add proclock to appropriate lists */
......
......@@ -263,6 +263,9 @@ InitProcGlobal(void)
/* Initialize myProcLocks[] shared memory queues. */
for (j = 0; j < NUM_LOCK_PARTITIONS; j++)
SHMQueueInit(&(procs[i].myProcLocks[j]));
/* Initialize lockGroupMembers list. */
dlist_init(&procs[i].lockGroupMembers);
}
/*
......@@ -397,6 +400,11 @@ InitProcess(void)
MyProc->backendLatestXid = InvalidTransactionId;
pg_atomic_init_u32(&MyProc->nextClearXidElem, INVALID_PGPROCNO);
/* Check that group locking fields are in a proper initial state. */
Assert(MyProc->lockGroupLeaderIdentifier == 0);
Assert(MyProc->lockGroupLeader == NULL);
Assert(dlist_is_empty(&MyProc->lockGroupMembers));
/*
* Acquire ownership of the PGPROC's latch, so that we can use WaitLatch
* on it. That allows us to repoint the process latch, which so far
......@@ -556,6 +564,11 @@ InitAuxiliaryProcess(void)
OwnLatch(&MyProc->procLatch);
SwitchToSharedLatch();
/* Check that group locking fields are in a proper initial state. */
Assert(MyProc->lockGroupLeaderIdentifier == 0);
Assert(MyProc->lockGroupLeader == NULL);
Assert(dlist_is_empty(&MyProc->lockGroupMembers));
/*
* We might be reusing a semaphore that belonged to a failed process. So
* be careful and reinitialize its value here. (This is not strictly
......@@ -793,6 +806,40 @@ ProcKill(int code, Datum arg)
if (MyReplicationSlot != NULL)
ReplicationSlotRelease();
/*
* Detach from any lock group of which we are a member. If the leader
* exist before all other group members, it's PGPROC will remain allocated
* until the last group process exits; that process must return the
* leader's PGPROC to the appropriate list.
*/
if (MyProc->lockGroupLeader != NULL)
{
PGPROC *leader = MyProc->lockGroupLeader;
LWLock *leader_lwlock = LockHashPartitionLockByProc(leader);
LWLockAcquire(leader_lwlock, LW_EXCLUSIVE);
Assert(!dlist_is_empty(&leader->lockGroupMembers));
dlist_delete(&MyProc->lockGroupLink);
if (dlist_is_empty(&leader->lockGroupMembers))
{
leader->lockGroupLeaderIdentifier = 0;
leader->lockGroupLeader = NULL;
if (leader != MyProc)
{
procgloballist = leader->procgloballist;
/* Leader exited first; return its PGPROC. */
SpinLockAcquire(ProcStructLock);
leader->links.next = (SHM_QUEUE *) *procgloballist;
*procgloballist = leader;
SpinLockRelease(ProcStructLock);
}
}
else if (leader != MyProc)
MyProc->lockGroupLeader = NULL;
LWLockRelease(leader_lwlock);
}
/*
* Reset MyLatch to the process local one. This is so that signal
* handlers et al can continue using the latch after the shared latch
......@@ -807,9 +854,20 @@ ProcKill(int code, Datum arg)
procgloballist = proc->procgloballist;
SpinLockAcquire(ProcStructLock);
/* Return PGPROC structure (and semaphore) to appropriate freelist */
proc->links.next = (SHM_QUEUE *) *procgloballist;
*procgloballist = proc;
/*
* If we're still a member of a locking group, that means we're a leader
* which has somehow exited before its children. The last remaining child
* will release our PGPROC. Otherwise, release it now.
*/
if (proc->lockGroupLeader == NULL)
{
/* Since lockGroupLeader is NULL, lockGroupMembers should be empty. */
Assert(dlist_is_empty(&proc->lockGroupMembers));
/* Return PGPROC structure (and semaphore) to appropriate freelist */
proc->links.next = (SHM_QUEUE *) *procgloballist;
*procgloballist = proc;
}
/* Update shared estimate of spins_per_delay */
ProcGlobal->spins_per_delay = update_spins_per_delay(ProcGlobal->spins_per_delay);
......@@ -942,8 +1000,30 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
bool allow_autovacuum_cancel = true;
int myWaitStatus;
PGPROC *proc;
PGPROC *leader = MyProc->lockGroupLeader;
int i;
/*
* If group locking is in use, locks held my members of my locking group
* need to be included in myHeldLocks.
*/
if (leader != NULL)
{
SHM_QUEUE *procLocks = &(lock->procLocks);
PROCLOCK *otherproclock;
otherproclock = (PROCLOCK *)
SHMQueueNext(procLocks, procLocks, offsetof(PROCLOCK, lockLink));
while (otherproclock != NULL)
{
if (otherproclock->groupLeader == leader)
myHeldLocks |= otherproclock->holdMask;
otherproclock = (PROCLOCK *)
SHMQueueNext(procLocks, &otherproclock->lockLink,
offsetof(PROCLOCK, lockLink));
}
}
/*
* Determine where to add myself in the wait queue.
*
......@@ -968,6 +1048,15 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
proc = (PGPROC *) waitQueue->links.next;
for (i = 0; i < waitQueue->size; i++)
{
/*
* If we're part of the same locking group as this waiter, its
* locks neither conflict with ours nor contribute to aheadRequsts.
*/
if (leader != NULL && leader == proc->lockGroupLeader)
{
proc = (PGPROC *) proc->links.next;
continue;
}
/* Must he wait for me? */
if (lockMethodTable->conflictTab[proc->waitLockMode] & myHeldLocks)
{
......@@ -1658,3 +1747,66 @@ ProcSendSignal(int pid)
SetLatch(&proc->procLatch);
}
}
/*
* BecomeLockGroupLeader - designate process as lock group leader
*
* Once this function has returned, other processes can join the lock group
* by calling BecomeLockGroupMember.
*/
void
BecomeLockGroupLeader(void)
{
LWLock *leader_lwlock;
/* If we already did it, we don't need to do it again. */
if (MyProc->lockGroupLeader == MyProc)
return;
/* We had better not be a follower. */
Assert(MyProc->lockGroupLeader == NULL);
/* Create single-member group, containing only ourselves. */
leader_lwlock = LockHashPartitionLockByProc(MyProc);
LWLockAcquire(leader_lwlock, LW_EXCLUSIVE);
MyProc->lockGroupLeader = MyProc;
MyProc->lockGroupLeaderIdentifier = MyProcPid;
dlist_push_head(&MyProc->lockGroupMembers, &MyProc->lockGroupLink);
LWLockRelease(leader_lwlock);
}
/*
* BecomeLockGroupMember - designate process as lock group member
*
* This is pretty straightforward except for the possibility that the leader
* whose group we're trying to join might exit before we manage to do so;
* and the PGPROC might get recycled for an unrelated process. To avoid
* that, we require the caller to pass the PID of the intended PGPROC as
* an interlock. Returns true if we successfully join the intended lock
* group, and false if not.
*/
bool
BecomeLockGroupMember(PGPROC *leader, int pid)
{
LWLock *leader_lwlock;
bool ok = false;
/* Group leader can't become member of group */
Assert(MyProc != leader);
/* PID must be valid. */
Assert(pid != 0);
/* Try to join the group. */
leader_lwlock = LockHashPartitionLockByProc(MyProc);
LWLockAcquire(leader_lwlock, LW_EXCLUSIVE);
if (leader->lockGroupLeaderIdentifier == pid)
{
ok = true;
MyProc->lockGroupLeader = leader;
dlist_push_tail(&leader->lockGroupMembers, &MyProc->lockGroupLink);
}
LWLockRelease(leader_lwlock);
return ok;
}
......@@ -346,6 +346,7 @@ typedef struct PROCLOCK
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 */
......@@ -457,7 +458,6 @@ typedef enum
* worker */
} DeadLockState;
/*
* The lockmgr's shared hash tables are partitioned to reduce contention.
* To determine which partition a given locktag belongs to, compute the tag's
......@@ -472,6 +472,17 @@ typedef enum
#define LockHashPartitionLockByIndex(i) \
(&MainLWLockArray[LOCK_MANAGER_LWLOCK_OFFSET + (i)].lock)
/*
* The deadlock detector needs to be able to access lockGroupLeader and
* related fields in the PGPROC, so we arrange for those fields to be protected
* by one of the lock hash partition locks. Since the deadlock detector
* acquires all such locks anyway, this makes it safe for it to access these
* fields without doing anything extra. To avoid contention as much as
* possible, we map different PGPROCs to different partition locks.
*/
#define LockHashPartitionLockByProc(p) \
LockHashPartitionLock((p)->pgprocno)
/*
* function prototypes
*/
......
......@@ -155,6 +155,15 @@ struct PGPROC
bool fpVXIDLock; /* are we holding a fast-path VXID lock? */
LocalTransactionId fpLocalTransactionId; /* lxid for fast-path VXID
* lock */
/*
* Support for lock groups. Use LockHashPartitionLockByProc to get the
* LWLock protecting these fields.
*/
int lockGroupLeaderIdentifier; /* MyProcPid, if I'm a leader */
PGPROC *lockGroupLeader; /* lock group leader, if I'm a follower */
dlist_head lockGroupMembers; /* list of members, if I'm a leader */
dlist_node lockGroupLink; /* my member link, if I'm a member */
};
/* NOTE: "typedef struct PGPROC PGPROC" appears in storage/lock.h. */
......@@ -272,4 +281,7 @@ extern void LockErrorCleanup(void);
extern void ProcWaitForSignal(void);
extern void ProcSendSignal(int pid);
extern void BecomeLockGroupLeader(void);
extern bool BecomeLockGroupMember(PGPROC *leader, int pid);
#endif /* PROC_H */
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment