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:
Tom Lane 2005-12-11 21:02:18 +00:00
parent be8100d64e
commit ec0baf949e
10 changed files with 643 additions and 415 deletions

View File

@ -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;

View File

@ -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 $
*
*-------------------------------------------------------------------------
*/

View File

@ -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

View File

@ -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

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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);

View File

@ -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;

View File

@ -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);