mirror of
https://github.com/postgres/postgres.git
synced 2025-04-24 10:47:04 +03:00
Divide the lock manager's shared state into 'partitions', so as to
reduce contention for the former single LockMgrLock. Per my recent proposal. I set it up for 16 partitions, but on a pgbench test this gives only a marginal further improvement over 4 partitions --- we need to test more scenarios to choose the number of partitions.
This commit is contained in:
parent
be8100d64e
commit
ec0baf949e
@ -7,7 +7,7 @@
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/access/transam/twophase.c,v 1.17 2005/11/22 18:17:07 momjian Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/access/transam/twophase.c,v 1.18 2005/12/11 21:02:17 tgl Exp $
|
||||
*
|
||||
* NOTES
|
||||
* Each global transaction is associated with a global transaction
|
||||
@ -284,7 +284,8 @@ MarkAsPreparing(TransactionId xid, const char *gid,
|
||||
gxact->proc.lwWaitLink = NULL;
|
||||
gxact->proc.waitLock = NULL;
|
||||
gxact->proc.waitProcLock = NULL;
|
||||
SHMQueueInit(&(gxact->proc.procLocks));
|
||||
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
|
||||
SHMQueueInit(&(gxact->proc.myProcLocks[i]));
|
||||
/* subxid data must be filled later by GXactLoadSubxactData */
|
||||
gxact->proc.subxids.overflowed = false;
|
||||
gxact->proc.subxids.nxids = 0;
|
||||
|
@ -14,8 +14,8 @@
|
||||
*
|
||||
* The process array now also includes PGPROC structures representing
|
||||
* prepared transactions. The xid and subxids fields of these are valid,
|
||||
* as is the procLocks list. They can be distinguished from regular backend
|
||||
* PGPROCs at need by checking for pid == 0.
|
||||
* as are the myProcLocks lists. They can be distinguished from regular
|
||||
* backend PGPROCs at need by checking for pid == 0.
|
||||
*
|
||||
*
|
||||
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
|
||||
@ -23,7 +23,7 @@
|
||||
*
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.8 2005/11/22 18:17:20 momjian Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.9 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
@ -1,4 +1,4 @@
|
||||
$PostgreSQL: pgsql/src/backend/storage/lmgr/README,v 1.18 2005/12/09 01:22:04 tgl Exp $
|
||||
$PostgreSQL: pgsql/src/backend/storage/lmgr/README,v 1.19 2005/12/11 21:02:18 tgl Exp $
|
||||
|
||||
|
||||
LOCKING OVERVIEW
|
||||
@ -50,9 +50,12 @@ LOCK DATA STRUCTURES
|
||||
Lock methods describe the overall locking behavior. Currently there are
|
||||
two lock methods: DEFAULT and USER. (USER locks are non-blocking.)
|
||||
|
||||
Lock modes describe the type of the lock (read/write or shared/exclusive).
|
||||
See src/tools/backend/index.html and src/include/storage/lock.h for more
|
||||
details.
|
||||
Lock modes describe the type of the lock (read/write or shared/exclusive).
|
||||
In principle, each lock method can have its own set of lock modes with
|
||||
different conflict rules, but currently DEFAULT and USER methods use
|
||||
identical lock mode sets. See src/tools/backend/index.html and
|
||||
src/include/storage/lock.h for more details. (Lock modes are also called
|
||||
lock types in some places in the code and documentation.)
|
||||
|
||||
There are two fundamental lock structures in shared memory: the
|
||||
per-lockable-object LOCK struct, and the per-lock-and-requestor PROCLOCK
|
||||
@ -67,7 +70,7 @@ be made per lockable object/lock mode/backend. Internally to a backend,
|
||||
however, the same lock may be requested and perhaps released multiple times
|
||||
in a transaction, and it can also be held both transactionally and session-
|
||||
wide. The internal request counts are held in LOCALLOCK so that the shared
|
||||
LockMgrLock need not be obtained to alter them.
|
||||
data structures need not be accessed to alter them.
|
||||
|
||||
---------------------------------------------------------------------------
|
||||
|
||||
@ -103,10 +106,10 @@ procLocks -
|
||||
be waiting for more!).
|
||||
|
||||
waitProcs -
|
||||
This is a shared memory queue of all process structures corresponding to
|
||||
a backend that is waiting (sleeping) until another backend releases this
|
||||
This is a shared memory queue of all PGPROC structures corresponding to
|
||||
backends that are waiting (sleeping) until another backend releases this
|
||||
lock. The process structure holds the information needed to determine
|
||||
if it should be woken up when this lock is released.
|
||||
if it should be woken up when the lock is released.
|
||||
|
||||
nRequested -
|
||||
Keeps a count of how many times this lock has been attempted to be
|
||||
@ -131,12 +134,12 @@ nGranted -
|
||||
granted -
|
||||
Keeps count of how many locks of each type are currently held. Once again
|
||||
only elements 1 through MAX_LOCKMODES-1 are used (0 is not). Also, like
|
||||
requested, summing the values of granted should total to the value
|
||||
requested[], summing the values of granted[] should total to the value
|
||||
of nGranted.
|
||||
|
||||
We should always have 0 <= nGranted <= nRequested, and
|
||||
0 <= granted[i] <= requested[i] for each i. If the request counts go to
|
||||
zero, the lock object is no longer needed and can be freed.
|
||||
0 <= granted[i] <= requested[i] for each i. When all the request counts
|
||||
go to zero, the LOCK object is no longer needed and can be freed.
|
||||
|
||||
---------------------------------------------------------------------------
|
||||
|
||||
@ -154,15 +157,16 @@ tag -
|
||||
SHMEM offset of PGPROC of backend process that owns this PROCLOCK.
|
||||
|
||||
holdMask -
|
||||
A bitmask for the lock types successfully acquired by this PROCLOCK.
|
||||
A bitmask for the lock modes successfully acquired by this PROCLOCK.
|
||||
This should be a subset of the LOCK object's grantMask, and also a
|
||||
subset of the PGPROC object's heldLocks mask.
|
||||
subset of the PGPROC object's heldLocks mask (if the PGPROC is
|
||||
currently waiting for another lock mode on this lock).
|
||||
|
||||
releaseMask -
|
||||
A bitmask for the lock types due to be released during LockReleaseAll.
|
||||
A bitmask for the lock modes due to be released during LockReleaseAll.
|
||||
This must be a subset of the holdMask. Note that it is modified without
|
||||
taking the LockMgrLock, and therefore it is unsafe for any backend except
|
||||
the one owning the PROCLOCK to examine/change it.
|
||||
taking the partition LWLock, and therefore it is unsafe for any
|
||||
backend except the one owning the PROCLOCK to examine/change it.
|
||||
|
||||
lockLink -
|
||||
List link for shared memory queue of all the PROCLOCK objects for the
|
||||
@ -174,7 +178,60 @@ procLink -
|
||||
|
||||
---------------------------------------------------------------------------
|
||||
|
||||
The deadlock detection algorithm:
|
||||
|
||||
LOCK MANAGER INTERNAL LOCKING
|
||||
|
||||
Before PostgreSQL 8.2, all of the shared-memory data structures used by
|
||||
the lock manager were protected by a single LWLock, the LockMgrLock;
|
||||
any operation involving these data structures had to exclusively lock
|
||||
LockMgrLock. Not too surprisingly, this became a contention bottleneck.
|
||||
To reduce contention, the lock manager's data structures have been split
|
||||
into multiple "partitions", each protected by an independent LWLock.
|
||||
Most operations only need to lock the single partition they are working in.
|
||||
Here are the details:
|
||||
|
||||
* Each possible lock is assigned to one partition according to a hash of
|
||||
its LOCKTAG value (see LockTagToPartition()). The partition's LWLock is
|
||||
considered to protect all the LOCK objects of that partition as well as
|
||||
their subsidiary PROCLOCKs. The shared-memory hash tables for LOCKs and
|
||||
PROCLOCKs are divided into separate hash tables for each partition, and
|
||||
operations on each hash table are likewise protected by the partition
|
||||
lock.
|
||||
|
||||
* Formerly, each PGPROC had a single list of PROCLOCKs belonging to it.
|
||||
This has now been split into per-partition lists, so that access to a
|
||||
particular PROCLOCK list can be protected by the associated partition's
|
||||
LWLock. (This is not strictly necessary at the moment, because at this
|
||||
writing a PGPROC's PROCLOCK list is only accessed by the owning backend
|
||||
anyway. But it seems forward-looking to maintain a convention for how
|
||||
other backends could access it. In any case LockReleaseAll needs to be
|
||||
able to quickly determine which partition each LOCK belongs to, and
|
||||
for the currently contemplated number of partitions, this way takes less
|
||||
shared memory than explicitly storing a partition number in LOCK structs
|
||||
would require.)
|
||||
|
||||
* The other lock-related fields of a PGPROC are only interesting when
|
||||
the PGPROC is waiting for a lock, so we consider that they are protected
|
||||
by the partition LWLock of the awaited lock.
|
||||
|
||||
For normal lock acquisition and release, it is sufficient to lock the
|
||||
partition containing the desired lock. Deadlock checking needs to touch
|
||||
multiple partitions in general; for simplicity, we just make it lock all
|
||||
the partitions in partition-number order. (To prevent LWLock deadlock,
|
||||
we establish the rule that any backend needing to lock more than one
|
||||
partition at once must lock them in partition-number order.) It's
|
||||
possible that deadlock checking could be done without touching every
|
||||
partition in typical cases, but since in a properly functioning system
|
||||
deadlock checking should not occur often enough to be performance-critical,
|
||||
trying to make this work does not seem a productive use of effort.
|
||||
|
||||
A backend's internal LOCALLOCK hash table is not partitioned. We do store
|
||||
the partition number in LOCALLOCK table entries, but this is a straight
|
||||
speed-for-space tradeoff: we could instead recalculate the partition
|
||||
number from the LOCKTAG when needed.
|
||||
|
||||
|
||||
THE DEADLOCK DETECTION ALGORITHM
|
||||
|
||||
Since we allow user transactions to request locks in any order, deadlock
|
||||
is possible. We use a deadlock detection/breaking algorithm that is
|
||||
|
@ -12,7 +12,7 @@
|
||||
*
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/deadlock.c,v 1.37 2005/12/09 01:22:04 tgl Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/deadlock.c,v 1.38 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
* Interface:
|
||||
*
|
||||
@ -53,9 +53,9 @@ typedef struct
|
||||
* Information saved about each edge in a detected deadlock cycle. This
|
||||
* is used to print a diagnostic message upon failure.
|
||||
*
|
||||
* Note: because we want to examine this info after releasing the LockMgrLock,
|
||||
* we can't just store LOCK and PGPROC pointers; we must extract out all the
|
||||
* info we want to be able to print.
|
||||
* Note: because we want to examine this info after releasing the lock
|
||||
* manager's partition locks, we can't just store LOCK and PGPROC pointers;
|
||||
* we must extract out all the info we want to be able to print.
|
||||
*/
|
||||
typedef struct
|
||||
{
|
||||
@ -188,19 +188,11 @@ InitDeadLockChecking(void)
|
||||
* deadlock. If resolution is impossible, return TRUE --- the caller
|
||||
* is then expected to abort the given proc's transaction.
|
||||
*
|
||||
* We can't block on user locks, so no sense testing for deadlock
|
||||
* because there is no blocking, and no timer for the block. So,
|
||||
* only look at regular locks.
|
||||
*
|
||||
* We must have already locked the master lock before being called.
|
||||
* NOTE: although the lockmethod structure appears to allow each lock
|
||||
* table to have a different masterLock, all locks that can block had
|
||||
* better use the same LWLock, else this code will not be adequately
|
||||
* interlocked!
|
||||
* Caller must already have locked all partitions of the lock tables.
|
||||
*
|
||||
* On failure, deadlock details are recorded in deadlockDetails[] for
|
||||
* subsequent printing by DeadLockReport(). That activity is separate
|
||||
* because we don't want to do it while holding the master lock.
|
||||
* because we don't want to do it while holding all those LWLocks.
|
||||
*/
|
||||
bool
|
||||
DeadLockCheck(PGPROC *proc)
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -8,14 +8,14 @@
|
||||
* exclusive and shared lock modes (to support read/write and read-only
|
||||
* access to a shared object). There are few other frammishes. User-level
|
||||
* locking should be done with the full lock manager --- which depends on
|
||||
* an LWLock to protect its shared state.
|
||||
* LWLocks to protect its shared state.
|
||||
*
|
||||
*
|
||||
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lwlock.c,v 1.35 2005/12/06 23:08:33 tgl Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lwlock.c,v 1.36 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@ -125,7 +125,10 @@ NumLWLocks(void)
|
||||
*/
|
||||
|
||||
/* Predefined LWLocks */
|
||||
numLocks = (int) NumFixedLWLocks;
|
||||
numLocks = (int) FirstLockMgrLock;
|
||||
|
||||
/* lock.c gets the ones starting at FirstLockMgrLock */
|
||||
numLocks += NUM_LOCK_PARTITIONS;
|
||||
|
||||
/* bufmgr.c needs two for each shared buffer */
|
||||
numLocks += 2 * NBuffers;
|
||||
@ -204,10 +207,11 @@ CreateLWLocks(void)
|
||||
|
||||
/*
|
||||
* Initialize the dynamic-allocation counter, which is stored just before
|
||||
* the first LWLock.
|
||||
* the first LWLock. The LWLocks used by lock.c are not dynamically
|
||||
* allocated, it just assumes it has them.
|
||||
*/
|
||||
LWLockCounter = (int *) ((char *) LWLockArray - 2 * sizeof(int));
|
||||
LWLockCounter[0] = (int) NumFixedLWLocks;
|
||||
LWLockCounter[0] = (int) FirstLockMgrLock + NUM_LOCK_PARTITIONS;
|
||||
LWLockCounter[1] = numLocks;
|
||||
}
|
||||
|
||||
|
@ -8,7 +8,7 @@
|
||||
*
|
||||
*
|
||||
* IDENTIFICATION
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.169 2005/12/09 01:22:04 tgl Exp $
|
||||
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.170 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@ -18,9 +18,8 @@
|
||||
* ProcQueueAlloc() -- create a shm queue for sleeping processes
|
||||
* ProcQueueInit() -- create a queue without allocing memory
|
||||
*
|
||||
* Locking and waiting for buffers can cause the backend to be
|
||||
* put to sleep. Whoever releases the lock, etc. wakes the
|
||||
* process up again (and gives it an error code so it knows
|
||||
* Waiting for a lock causes the backend to be put to sleep. Whoever releases
|
||||
* the lock wakes the process up again (and gives it an error code so it knows
|
||||
* whether it was awoken on an error condition).
|
||||
*
|
||||
* Interface (b):
|
||||
@ -28,7 +27,7 @@
|
||||
* ProcReleaseLocks -- frees the locks associated with current transaction
|
||||
*
|
||||
* ProcKill -- destroys the shared memory state (and locks)
|
||||
* associated with the process.
|
||||
* associated with the process.
|
||||
*/
|
||||
#include "postgres.h"
|
||||
|
||||
@ -65,7 +64,8 @@ NON_EXEC_STATIC slock_t *ProcStructLock = NULL;
|
||||
static PROC_HDR *ProcGlobal = NULL;
|
||||
static PGPROC *DummyProcs = NULL;
|
||||
|
||||
static bool waitingForLock = false;
|
||||
/* If we are waiting for a lock, this points to the associated LOCALLOCK */
|
||||
static LOCALLOCK *lockAwaited = NULL;
|
||||
|
||||
/* Mark these volatile because they can be changed by signal handler */
|
||||
static volatile bool statement_timeout_active = false;
|
||||
@ -200,10 +200,10 @@ InitProcGlobal(void)
|
||||
void
|
||||
InitProcess(void)
|
||||
{
|
||||
SHMEM_OFFSET myOffset;
|
||||
|
||||
/* use volatile pointer to prevent code rearrangement */
|
||||
volatile PROC_HDR *procglobal = ProcGlobal;
|
||||
SHMEM_OFFSET myOffset;
|
||||
int i;
|
||||
|
||||
/*
|
||||
* ProcGlobal should be set by a previous call to InitProcGlobal (if we
|
||||
@ -264,7 +264,8 @@ InitProcess(void)
|
||||
MyProc->lwWaitLink = NULL;
|
||||
MyProc->waitLock = NULL;
|
||||
MyProc->waitProcLock = NULL;
|
||||
SHMQueueInit(&(MyProc->procLocks));
|
||||
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
|
||||
SHMQueueInit(&(MyProc->myProcLocks[i]));
|
||||
|
||||
/*
|
||||
* Add our PGPROC to the PGPROC array in shared memory.
|
||||
@ -304,6 +305,7 @@ void
|
||||
InitDummyProcess(int proctype)
|
||||
{
|
||||
PGPROC *dummyproc;
|
||||
int i;
|
||||
|
||||
/*
|
||||
* ProcGlobal should be set by a previous call to InitProcGlobal (we
|
||||
@ -360,7 +362,8 @@ InitDummyProcess(int proctype)
|
||||
MyProc->lwWaitLink = NULL;
|
||||
MyProc->waitLock = NULL;
|
||||
MyProc->waitProcLock = NULL;
|
||||
SHMQueueInit(&(MyProc->procLocks));
|
||||
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
|
||||
SHMQueueInit(&(MyProc->myProcLocks[i]));
|
||||
|
||||
/*
|
||||
* Arrange to clean up at process exit.
|
||||
@ -416,21 +419,24 @@ HaveNFreeProcs(int n)
|
||||
bool
|
||||
LockWaitCancel(void)
|
||||
{
|
||||
LWLockId partitionLock;
|
||||
|
||||
/* Nothing to do if we weren't waiting for a lock */
|
||||
if (!waitingForLock)
|
||||
if (lockAwaited == NULL)
|
||||
return false;
|
||||
|
||||
/* Turn off the deadlock timer, if it's still running (see ProcSleep) */
|
||||
disable_sig_alarm(false);
|
||||
|
||||
/* Unlink myself from the wait queue, if on it (might not be anymore!) */
|
||||
LWLockAcquire(LockMgrLock, LW_EXCLUSIVE);
|
||||
partitionLock = FirstLockMgrLock + lockAwaited->partition;
|
||||
LWLockAcquire(partitionLock, LW_EXCLUSIVE);
|
||||
|
||||
if (MyProc->links.next != INVALID_OFFSET)
|
||||
{
|
||||
/* We could not have been granted the lock yet */
|
||||
Assert(MyProc->waitStatus == STATUS_ERROR);
|
||||
RemoveFromWaitQueue(MyProc);
|
||||
RemoveFromWaitQueue(MyProc, lockAwaited->partition);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -444,9 +450,9 @@ LockWaitCancel(void)
|
||||
GrantAwaitedLock();
|
||||
}
|
||||
|
||||
waitingForLock = false;
|
||||
lockAwaited = NULL;
|
||||
|
||||
LWLockRelease(LockMgrLock);
|
||||
LWLockRelease(partitionLock);
|
||||
|
||||
/*
|
||||
* Reset the proc wait semaphore to zero. This is necessary in the
|
||||
@ -606,18 +612,18 @@ ProcQueueInit(PROC_QUEUE *queue)
|
||||
|
||||
|
||||
/*
|
||||
* ProcSleep -- put a process to sleep
|
||||
* ProcSleep -- put a process to sleep on the specified lock
|
||||
*
|
||||
* Caller must have set MyProc->heldLocks to reflect locks already held
|
||||
* on the lockable object by this process (under all XIDs).
|
||||
*
|
||||
* Locktable's masterLock must be held at entry, and will be held
|
||||
* The lock table's partition lock must be held at entry, and will be held
|
||||
* at exit.
|
||||
*
|
||||
* Result: STATUS_OK if we acquired the lock, STATUS_ERROR if not (deadlock).
|
||||
*
|
||||
* ASSUME: that no one will fiddle with the queue until after
|
||||
* we release the masterLock.
|
||||
* we release the partition lock.
|
||||
*
|
||||
* NOTES: The process queue is now a priority queue for locking.
|
||||
*
|
||||
@ -625,12 +631,13 @@ ProcQueueInit(PROC_QUEUE *queue)
|
||||
* semaphore is normally zero, so when we try to acquire it, we sleep.
|
||||
*/
|
||||
int
|
||||
ProcSleep(LockMethod lockMethodTable,
|
||||
LOCKMODE lockmode,
|
||||
LOCK *lock,
|
||||
PROCLOCK *proclock)
|
||||
ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
|
||||
{
|
||||
LWLockId masterLock = LockMgrLock;
|
||||
LOCKMODE lockmode = locallock->tag.mode;
|
||||
LOCK *lock = locallock->lock;
|
||||
PROCLOCK *proclock = locallock->proclock;
|
||||
int partition = locallock->partition;
|
||||
LWLockId partitionLock = FirstLockMgrLock + partition;
|
||||
PROC_QUEUE *waitQueue = &(lock->waitProcs);
|
||||
LOCKMASK myHeldLocks = MyProc->heldLocks;
|
||||
bool early_deadlock = false;
|
||||
@ -732,22 +739,22 @@ ProcSleep(LockMethod lockMethodTable,
|
||||
*/
|
||||
if (early_deadlock)
|
||||
{
|
||||
RemoveFromWaitQueue(MyProc);
|
||||
RemoveFromWaitQueue(MyProc, partition);
|
||||
return STATUS_ERROR;
|
||||
}
|
||||
|
||||
/* mark that we are waiting for a lock */
|
||||
waitingForLock = true;
|
||||
lockAwaited = locallock;
|
||||
|
||||
/*
|
||||
* Release the locktable's masterLock.
|
||||
* Release the lock table's partition lock.
|
||||
*
|
||||
* NOTE: this may also cause us to exit critical-section state, possibly
|
||||
* allowing a cancel/die interrupt to be accepted. This is OK because we
|
||||
* have recorded the fact that we are waiting for a lock, and so
|
||||
* LockWaitCancel will clean up if cancel/die happens.
|
||||
*/
|
||||
LWLockRelease(masterLock);
|
||||
LWLockRelease(partitionLock);
|
||||
|
||||
/*
|
||||
* Set timer so we can wake up after awhile and check for a deadlock. If a
|
||||
@ -785,16 +792,16 @@ ProcSleep(LockMethod lockMethodTable,
|
||||
elog(FATAL, "could not disable timer for process wakeup");
|
||||
|
||||
/*
|
||||
* Re-acquire the locktable's masterLock. We have to do this to hold off
|
||||
* cancel/die interrupts before we can mess with waitingForLock (else we
|
||||
* might have a missed or duplicated locallock update).
|
||||
* Re-acquire the lock table's partition lock. We have to do this to
|
||||
* hold off cancel/die interrupts before we can mess with lockAwaited
|
||||
* (else we might have a missed or duplicated locallock update).
|
||||
*/
|
||||
LWLockAcquire(masterLock, LW_EXCLUSIVE);
|
||||
LWLockAcquire(partitionLock, LW_EXCLUSIVE);
|
||||
|
||||
/*
|
||||
* We no longer want LockWaitCancel to do anything.
|
||||
*/
|
||||
waitingForLock = false;
|
||||
lockAwaited = NULL;
|
||||
|
||||
/*
|
||||
* If we got the lock, be sure to remember it in the locallock table.
|
||||
@ -816,6 +823,8 @@ ProcSleep(LockMethod lockMethodTable,
|
||||
* Also remove the process from the wait queue and set its links invalid.
|
||||
* RETURN: the next process in the wait queue.
|
||||
*
|
||||
* The appropriate lock partition lock must be held by caller.
|
||||
*
|
||||
* XXX: presently, this code is only used for the "success" case, and only
|
||||
* works correctly for that case. To clean up in failure case, would need
|
||||
* to twiddle the lock's request counts too --- see RemoveFromWaitQueue.
|
||||
@ -825,8 +834,6 @@ ProcWakeup(PGPROC *proc, int waitStatus)
|
||||
{
|
||||
PGPROC *retProc;
|
||||
|
||||
/* assume that masterLock has been acquired */
|
||||
|
||||
/* Proc should be sleeping ... */
|
||||
if (proc->links.prev == INVALID_OFFSET ||
|
||||
proc->links.next == INVALID_OFFSET)
|
||||
@ -854,6 +861,8 @@ ProcWakeup(PGPROC *proc, int waitStatus)
|
||||
* ProcLockWakeup -- routine for waking up processes when a lock is
|
||||
* released (or a prior waiter is aborted). Scan all waiters
|
||||
* for lock, waken any that are no longer blocked.
|
||||
*
|
||||
* The appropriate lock partition lock must be held by caller.
|
||||
*/
|
||||
void
|
||||
ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock)
|
||||
@ -908,25 +917,32 @@ ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock)
|
||||
Assert(waitQueue->size >= 0);
|
||||
}
|
||||
|
||||
/* --------------------
|
||||
/*
|
||||
* CheckDeadLock
|
||||
*
|
||||
* We only get to this routine if we got SIGALRM after DeadlockTimeout
|
||||
* while waiting for a lock to be released by some other process. Look
|
||||
* to see if there's a deadlock; if not, just return and continue waiting.
|
||||
* If we have a real deadlock, remove ourselves from the lock's wait queue
|
||||
* and signal an error to ProcSleep.
|
||||
* --------------------
|
||||
*/
|
||||
static void
|
||||
CheckDeadLock(void)
|
||||
{
|
||||
int i;
|
||||
|
||||
/*
|
||||
* Acquire locktable lock. Note that the deadlock check interrupt had
|
||||
* better not be enabled anywhere that this process itself holds the
|
||||
* locktable lock, else this will wait forever. Also note that
|
||||
* LWLockAcquire creates a critical section, so that this routine cannot
|
||||
* be interrupted by cancel/die interrupts.
|
||||
* Acquire exclusive lock on the entire shared lock data structures.
|
||||
* Must grab LWLocks in partition-number order to avoid LWLock deadlock.
|
||||
*
|
||||
* Note that the deadlock check interrupt had better not be enabled
|
||||
* anywhere that this process itself holds lock partition locks, else this
|
||||
* will wait forever. Also note that LWLockAcquire creates a critical
|
||||
* section, so that this routine cannot be interrupted by cancel/die
|
||||
* interrupts.
|
||||
*/
|
||||
LWLockAcquire(LockMgrLock, LW_EXCLUSIVE);
|
||||
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
|
||||
LWLockAcquire(FirstLockMgrLock + i, LW_EXCLUSIVE);
|
||||
|
||||
/*
|
||||
* Check to see if we've been awoken by anyone in the interim.
|
||||
@ -937,14 +953,11 @@ CheckDeadLock(void)
|
||||
*
|
||||
* We check by looking to see if we've been unlinked from the wait queue.
|
||||
* This is quicker than checking our semaphore's state, since no kernel
|
||||
* call is needed, and it is safe because we hold the locktable lock.
|
||||
* call is needed, and it is safe because we hold the lock partition lock.
|
||||
*/
|
||||
if (MyProc->links.prev == INVALID_OFFSET ||
|
||||
MyProc->links.next == INVALID_OFFSET)
|
||||
{
|
||||
LWLockRelease(LockMgrLock);
|
||||
return;
|
||||
}
|
||||
goto check_done;
|
||||
|
||||
#ifdef LOCK_DEBUG
|
||||
if (Debug_deadlocks)
|
||||
@ -954,16 +967,19 @@ CheckDeadLock(void)
|
||||
if (!DeadLockCheck(MyProc))
|
||||
{
|
||||
/* No deadlock, so keep waiting */
|
||||
LWLockRelease(LockMgrLock);
|
||||
return;
|
||||
goto check_done;
|
||||
}
|
||||
|
||||
/*
|
||||
* Oops. We have a deadlock.
|
||||
*
|
||||
* Get this process out of wait state.
|
||||
* Get this process out of wait state. (Note: we could do this more
|
||||
* efficiently by relying on lockAwaited, but use this coding to preserve
|
||||
* the flexibility to kill some other transaction than the one detecting
|
||||
* the deadlock.)
|
||||
*/
|
||||
RemoveFromWaitQueue(MyProc);
|
||||
Assert(MyProc->waitLock != NULL);
|
||||
RemoveFromWaitQueue(MyProc, LockTagToPartition(&(MyProc->waitLock->tag)));
|
||||
|
||||
/*
|
||||
* Set MyProc->waitStatus to STATUS_ERROR so that ProcSleep will report an
|
||||
@ -987,7 +1003,15 @@ CheckDeadLock(void)
|
||||
* them anymore. However, RemoveFromWaitQueue took care of waking up any
|
||||
* such processes.
|
||||
*/
|
||||
LWLockRelease(LockMgrLock);
|
||||
|
||||
/*
|
||||
* Release locks acquired at head of routine. Order is not critical,
|
||||
* so do it back-to-front to avoid waking another CheckDeadLock instance
|
||||
* before it can get all the locks.
|
||||
*/
|
||||
check_done:
|
||||
for (i = NUM_LOCK_PARTITIONS; --i >= 0; )
|
||||
LWLockRelease(FirstLockMgrLock + i);
|
||||
}
|
||||
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
* $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.92 2005/12/09 01:22:04 tgl Exp $
|
||||
* $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.93 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@ -19,6 +19,13 @@
|
||||
#include "storage/shmem.h"
|
||||
|
||||
|
||||
/*
|
||||
* Number of partitions the shared lock tables are divided into.
|
||||
*
|
||||
* See LockTagToPartition() if you change this.
|
||||
*/
|
||||
#define NUM_LOCK_PARTITIONS 16
|
||||
|
||||
/* originally in procq.h */
|
||||
typedef struct PROC_QUEUE
|
||||
{
|
||||
@ -348,6 +355,7 @@ typedef struct LOCALLOCK
|
||||
LOCK *lock; /* associated LOCK object in shared mem */
|
||||
PROCLOCK *proclock; /* associated PROCLOCK object in shmem */
|
||||
bool isTempObject; /* true if lock is on a temporary object */
|
||||
int partition; /* ID of partition containing this lock */
|
||||
int nLocks; /* total number of times lock is held */
|
||||
int numLockOwners; /* # of relevant ResourceOwners */
|
||||
int maxLockOwners; /* allocated size of array */
|
||||
@ -389,6 +397,7 @@ typedef enum
|
||||
*/
|
||||
extern void InitLocks(void);
|
||||
extern LockMethod GetLocksMethodTable(const LOCK *lock);
|
||||
extern int LockTagToPartition(const LOCKTAG *locktag);
|
||||
extern LockAcquireResult LockAcquire(const LOCKTAG *locktag,
|
||||
bool isTempObject,
|
||||
LOCKMODE lockmode,
|
||||
@ -406,7 +415,7 @@ extern int LockCheckConflicts(LockMethod lockMethodTable,
|
||||
LOCK *lock, PROCLOCK *proclock, PGPROC *proc);
|
||||
extern void GrantLock(LOCK *lock, PROCLOCK *proclock, LOCKMODE lockmode);
|
||||
extern void GrantAwaitedLock(void);
|
||||
extern void RemoveFromWaitQueue(PGPROC *proc);
|
||||
extern void RemoveFromWaitQueue(PGPROC *proc, int partition);
|
||||
extern Size LockShmemSize(void);
|
||||
extern bool DeadLockCheck(PGPROC *proc);
|
||||
extern void DeadLockReport(void);
|
||||
|
@ -7,7 +7,7 @@
|
||||
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
* $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.23 2005/10/15 02:49:46 momjian Exp $
|
||||
* $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.24 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@ -16,9 +16,9 @@
|
||||
|
||||
/*
|
||||
* We have a number of predefined LWLocks, plus a bunch of LWLocks that are
|
||||
* dynamically assigned (for shared buffers). The LWLock structures live
|
||||
* in shared memory (since they contain shared data) and are identified by
|
||||
* values of this enumerated type. We abuse the notion of an enum somewhat
|
||||
* dynamically assigned (e.g., for shared buffers). The LWLock structures
|
||||
* live in shared memory (since they contain shared data) and are identified
|
||||
* by values of this enumerated type. We abuse the notion of an enum somewhat
|
||||
* by allowing values not listed in the enum declaration to be assigned.
|
||||
* The extra value MaxDynamicLWLock is there to keep the compiler from
|
||||
* deciding that the enum can be represented as char or short ...
|
||||
@ -27,7 +27,6 @@ typedef enum LWLockId
|
||||
{
|
||||
BufMappingLock,
|
||||
BufFreelistLock,
|
||||
LockMgrLock,
|
||||
OidGenLock,
|
||||
XidGenLock,
|
||||
ProcArrayLock,
|
||||
@ -46,8 +45,7 @@ typedef enum LWLockId
|
||||
RelCacheInitLock,
|
||||
BgWriterCommLock,
|
||||
TwoPhaseStateLock,
|
||||
|
||||
NumFixedLWLocks, /* must be last except for MaxDynamicLWLock */
|
||||
FirstLockMgrLock, /* must be last except for MaxDynamicLWLock */
|
||||
|
||||
MaxDynamicLWLock = 1000000000
|
||||
} LWLockId;
|
||||
|
@ -7,7 +7,7 @@
|
||||
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
* $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.84 2005/10/15 02:49:46 momjian Exp $
|
||||
* $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.85 2005/12/11 21:02:18 tgl Exp $
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
@ -52,7 +52,8 @@ struct XidCache
|
||||
* so that the prepared transactions appear to be still running and are
|
||||
* correctly shown as holding locks. A prepared transaction PGPROC can be
|
||||
* distinguished from a real one at need by the fact that it has pid == 0.
|
||||
* The semaphore and lock-related fields in a prepared-xact PGPROC are unused.
|
||||
* The semaphore and lock-activity fields in a prepared-xact PGPROC are unused,
|
||||
* but its myProcLocks[] lists are valid.
|
||||
*/
|
||||
struct PGPROC
|
||||
{
|
||||
@ -86,8 +87,12 @@ struct PGPROC
|
||||
LOCKMASK heldLocks; /* bitmask for lock types already held on this
|
||||
* lock object by this backend */
|
||||
|
||||
SHM_QUEUE procLocks; /* list of PROCLOCK objects for locks held or
|
||||
* awaited by this backend */
|
||||
/*
|
||||
* All PROCLOCK objects for locks held or awaited by this backend are
|
||||
* linked into one of these lists, according to the partition number of
|
||||
* their lock.
|
||||
*/
|
||||
SHM_QUEUE myProcLocks[NUM_LOCK_PARTITIONS];
|
||||
|
||||
struct XidCache subxids; /* cache for subtransaction XIDs */
|
||||
};
|
||||
@ -99,7 +104,7 @@ extern DLLIMPORT PGPROC *MyProc;
|
||||
|
||||
|
||||
/*
|
||||
* There is one ProcGlobal struct for the whole installation.
|
||||
* There is one ProcGlobal struct for the whole database cluster.
|
||||
*/
|
||||
typedef struct PROC_HDR
|
||||
{
|
||||
@ -134,8 +139,7 @@ extern bool HaveNFreeProcs(int n);
|
||||
extern void ProcReleaseLocks(bool isCommit);
|
||||
|
||||
extern void ProcQueueInit(PROC_QUEUE *queue);
|
||||
extern int ProcSleep(LockMethod lockMethodTable, LOCKMODE lockmode,
|
||||
LOCK *lock, PROCLOCK *proclock);
|
||||
extern int ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable);
|
||||
extern PGPROC *ProcWakeup(PGPROC *proc, int waitStatus);
|
||||
extern void ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock);
|
||||
extern bool LockWaitCancel(void);
|
||||
|
Loading…
x
Reference in New Issue
Block a user