Commit 8563ccae authored by Tom Lane's avatar Tom Lane

Simplify shared-memory lock data structures as per recent discussion:

it is sufficient to track whether a backend holds a lock or not, and
store information about transaction vs. session locks only in the
inside-the-backend LocalLockTable.  Since there can now be but one
PROCLOCK per lock per backend, LockCountMyLocks() is no longer needed,
thus eliminating some O(N^2) behavior when a backend holds many locks.
Also simplify the LockAcquire/LockRelease API by passing just a
'sessionLock' boolean instead of a transaction ID.  The previous API
was designed with the idea that per-transaction lock holding would be
important for subtransactions, but now that we have subtransactions we
know that this is unwanted.  While at it, add an 'isTempObject' parameter
to LockAcquire to indicate whether the lock is being taken on a temp
table.  This is not used just yet, but will be needed shortly for
two-phase commit.
parent f5835b4b
...@@ -33,8 +33,8 @@ user_lock(uint32 id1, uint32 id2, LOCKMODE lockmode) ...@@ -33,8 +33,8 @@ user_lock(uint32 id1, uint32 id2, LOCKMODE lockmode)
SET_LOCKTAG_USERLOCK(tag, id1, id2); SET_LOCKTAG_USERLOCK(tag, id1, id2);
return (LockAcquire(USER_LOCKMETHOD, &tag, InvalidTransactionId, return (LockAcquire(USER_LOCKMETHOD, &tag, false,
lockmode, true) != LOCKACQUIRE_NOT_AVAIL); lockmode, true, true) != LOCKACQUIRE_NOT_AVAIL);
} }
int int
...@@ -44,7 +44,7 @@ user_unlock(uint32 id1, uint32 id2, LOCKMODE lockmode) ...@@ -44,7 +44,7 @@ user_unlock(uint32 id1, uint32 id2, LOCKMODE lockmode)
SET_LOCKTAG_USERLOCK(tag, id1, id2); SET_LOCKTAG_USERLOCK(tag, id1, id2);
return LockRelease(USER_LOCKMETHOD, &tag, InvalidTransactionId, lockmode); return LockRelease(USER_LOCKMETHOD, &tag, lockmode, true);
} }
int int
......
...@@ -13,7 +13,7 @@ ...@@ -13,7 +13,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.309 2005/05/19 21:35:45 tgl Exp $ * $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.310 2005/06/14 22:15:32 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -1009,7 +1009,7 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, char expected_relkind) ...@@ -1009,7 +1009,7 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, char expected_relkind)
* same process. * same process.
*/ */
onerelid = onerel->rd_lockInfo.lockRelId; onerelid = onerel->rd_lockInfo.lockRelId;
LockRelationForSession(&onerelid, lmode); LockRelationForSession(&onerelid, onerel->rd_istemp, lmode);
/* /*
* Remember the relation's TOAST relation for later * Remember the relation's TOAST relation for later
......
$PostgreSQL: pgsql/src/backend/storage/lmgr/README,v 1.16 2005/04/29 22:28:24 tgl Exp $ $PostgreSQL: pgsql/src/backend/storage/lmgr/README,v 1.17 2005/06/14 22:15:32 tgl Exp $
LOCKING OVERVIEW LOCKING OVERVIEW
...@@ -57,16 +57,17 @@ details. ...@@ -57,16 +57,17 @@ details.
There are two fundamental lock structures in shared memory: the There are two fundamental lock structures in shared memory: the
per-lockable-object LOCK struct, and the per-lock-and-requestor PROCLOCK per-lockable-object LOCK struct, and the per-lock-and-requestor PROCLOCK
struct. A LOCK object exists for each lockable object that currently has struct. A LOCK object exists for each lockable object that currently has
locks held or requested on it. A PROCLOCK struct exists for each transaction locks held or requested on it. A PROCLOCK struct exists for each backend
that is holding or requesting lock(s) on each LOCK object. that is holding or requesting lock(s) on each LOCK object.
In addition to these, each backend maintains an unshared LOCALLOCK structure In addition to these, each backend maintains an unshared LOCALLOCK structure
for each lockable object and lock mode that it is currently holding or for each lockable object and lock mode that it is currently holding or
requesting. The shared lock structures only allow a single lock grant to requesting. The shared lock structures only allow a single lock grant to
be made per lockable object/lock mode/transaction. Internally to a backend, be made per lockable object/lock mode/backend. Internally to a backend,
however, the same lock may be requested and perhaps released multiple times however, the same lock may be requested and perhaps released multiple times
in a transaction. The internal request counts are held in LOCALLOCK so that in a transaction, and it can also be held both transactionally and session-
the shared LockMgrLock need not be obtained to alter them. wide. The internal request counts are held in LOCALLOCK so that the shared
LockMgrLock need not be obtained to alter them.
--------------------------------------------------------------------------- ---------------------------------------------------------------------------
...@@ -112,9 +113,8 @@ nRequested - ...@@ -112,9 +113,8 @@ nRequested -
acquired. The count includes attempts by processes which were put acquired. The count includes attempts by processes which were put
to sleep due to conflicts. It also counts the same backend twice to sleep due to conflicts. It also counts the same backend twice
if, for example, a backend process first acquires a read and then if, for example, a backend process first acquires a read and then
acquires a write, or acquires the lock under two different transaction acquires a write. (But multiple acquisitions of the same lock/lock mode
IDs. (But multiple acquisitions of the same lock/lock mode under the within a backend are not multiply counted here; they are recorded
same transaction ID are not multiply counted here; they are recorded
only in the backend's LOCALLOCK structure.) only in the backend's LOCALLOCK structure.)
requested - requested -
...@@ -153,23 +153,17 @@ tag - ...@@ -153,23 +153,17 @@ tag -
tag.proc tag.proc
SHMEM offset of PROC of backend process that owns this PROCLOCK. SHMEM offset of PROC of backend process that owns this PROCLOCK.
tag.xid
XID of transaction this PROCLOCK is for, or InvalidTransactionId
if the PROCLOCK is for session-level locking.
Note that this structure will support multiple transactions running
concurrently in one backend. Currently we do not use it for that
purpose: subtransactions acquire locks in the name of their top parent
transaction, to simplify reassigning lock ownership at subtransaction end.
So the XID field is really only needed to distinguish per-transaction
locks from session locks. User locks are always session locks, and we
also use session locks for multi-transaction operations like VACUUM.
holdMask - holdMask -
A bitmask for the lock types successfully acquired by this PROCLOCK. A bitmask for the lock types successfully acquired by this PROCLOCK.
This should be a subset of the LOCK object's grantMask, and also a 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.
releaseMask -
A bitmask for the lock types 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.
lockLink - lockLink -
List link for shared memory queue of all the PROCLOCK objects for the List link for shared memory queue of all the PROCLOCK objects for the
same LOCK. same LOCK.
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/lmgr/lmgr.c,v 1.75 2005/05/29 22:45:02 tgl Exp $ * $PostgreSQL: pgsql/src/backend/storage/lmgr/lmgr.c,v 1.76 2005/06/14 22:15:32 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -142,8 +142,8 @@ LockRelation(Relation relation, LOCKMODE lockmode) ...@@ -142,8 +142,8 @@ LockRelation(Relation relation, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.dbId, relation->rd_lockInfo.lockRelId.dbId,
relation->rd_lockInfo.lockRelId.relId); relation->rd_lockInfo.lockRelId.relId);
res = LockAcquire(LockTableId, &tag, GetTopTransactionId(), res = LockAcquire(LockTableId, &tag, relation->rd_istemp,
lockmode, false); lockmode, false, false);
/* /*
* Check to see if the relcache entry has been invalidated while we * Check to see if the relcache entry has been invalidated while we
...@@ -179,8 +179,8 @@ ConditionalLockRelation(Relation relation, LOCKMODE lockmode) ...@@ -179,8 +179,8 @@ ConditionalLockRelation(Relation relation, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.dbId, relation->rd_lockInfo.lockRelId.dbId,
relation->rd_lockInfo.lockRelId.relId); relation->rd_lockInfo.lockRelId.relId);
res = LockAcquire(LockTableId, &tag, GetTopTransactionId(), res = LockAcquire(LockTableId, &tag, relation->rd_istemp,
lockmode, true); lockmode, false, true);
if (res == LOCKACQUIRE_NOT_AVAIL) if (res == LOCKACQUIRE_NOT_AVAIL)
return false; return false;
...@@ -214,7 +214,7 @@ UnlockRelation(Relation relation, LOCKMODE lockmode) ...@@ -214,7 +214,7 @@ UnlockRelation(Relation relation, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.dbId, relation->rd_lockInfo.lockRelId.dbId,
relation->rd_lockInfo.lockRelId.relId); relation->rd_lockInfo.lockRelId.relId);
LockRelease(LockTableId, &tag, GetTopTransactionId(), lockmode); LockRelease(LockTableId, &tag, lockmode, false);
} }
/* /*
...@@ -230,14 +230,14 @@ UnlockRelation(Relation relation, LOCKMODE lockmode) ...@@ -230,14 +230,14 @@ UnlockRelation(Relation relation, LOCKMODE lockmode)
* relcache entry is up to date. * relcache entry is up to date.
*/ */
void void
LockRelationForSession(LockRelId *relid, LOCKMODE lockmode) LockRelationForSession(LockRelId *relid, bool istemprel, LOCKMODE lockmode)
{ {
LOCKTAG tag; LOCKTAG tag;
SET_LOCKTAG_RELATION(tag, relid->dbId, relid->relId); SET_LOCKTAG_RELATION(tag, relid->dbId, relid->relId);
(void) LockAcquire(LockTableId, &tag, InvalidTransactionId, (void) LockAcquire(LockTableId, &tag, istemprel,
lockmode, false); lockmode, true, false);
} }
/* /*
...@@ -250,7 +250,7 @@ UnlockRelationForSession(LockRelId *relid, LOCKMODE lockmode) ...@@ -250,7 +250,7 @@ UnlockRelationForSession(LockRelId *relid, LOCKMODE lockmode)
SET_LOCKTAG_RELATION(tag, relid->dbId, relid->relId); SET_LOCKTAG_RELATION(tag, relid->dbId, relid->relId);
LockRelease(LockTableId, &tag, InvalidTransactionId, lockmode); LockRelease(LockTableId, &tag, lockmode, true);
} }
/* /*
...@@ -272,8 +272,8 @@ LockRelationForExtension(Relation relation, LOCKMODE lockmode) ...@@ -272,8 +272,8 @@ LockRelationForExtension(Relation relation, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.dbId, relation->rd_lockInfo.lockRelId.dbId,
relation->rd_lockInfo.lockRelId.relId); relation->rd_lockInfo.lockRelId.relId);
(void) LockAcquire(LockTableId, &tag, GetTopTransactionId(), (void) LockAcquire(LockTableId, &tag, relation->rd_istemp,
lockmode, false); lockmode, false, false);
} }
/* /*
...@@ -288,7 +288,7 @@ UnlockRelationForExtension(Relation relation, LOCKMODE lockmode) ...@@ -288,7 +288,7 @@ UnlockRelationForExtension(Relation relation, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.dbId, relation->rd_lockInfo.lockRelId.dbId,
relation->rd_lockInfo.lockRelId.relId); relation->rd_lockInfo.lockRelId.relId);
LockRelease(LockTableId, &tag, GetTopTransactionId(), lockmode); LockRelease(LockTableId, &tag, lockmode, false);
} }
/* /*
...@@ -307,8 +307,8 @@ LockPage(Relation relation, BlockNumber blkno, LOCKMODE lockmode) ...@@ -307,8 +307,8 @@ LockPage(Relation relation, BlockNumber blkno, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.relId, relation->rd_lockInfo.lockRelId.relId,
blkno); blkno);
(void) LockAcquire(LockTableId, &tag, GetTopTransactionId(), (void) LockAcquire(LockTableId, &tag, relation->rd_istemp,
lockmode, false); lockmode, false, false);
} }
/* /*
...@@ -327,8 +327,8 @@ ConditionalLockPage(Relation relation, BlockNumber blkno, LOCKMODE lockmode) ...@@ -327,8 +327,8 @@ ConditionalLockPage(Relation relation, BlockNumber blkno, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.relId, relation->rd_lockInfo.lockRelId.relId,
blkno); blkno);
return (LockAcquire(LockTableId, &tag, GetTopTransactionId(), return (LockAcquire(LockTableId, &tag, relation->rd_istemp,
lockmode, true) != LOCKACQUIRE_NOT_AVAIL); lockmode, false, true) != LOCKACQUIRE_NOT_AVAIL);
} }
/* /*
...@@ -344,7 +344,7 @@ UnlockPage(Relation relation, BlockNumber blkno, LOCKMODE lockmode) ...@@ -344,7 +344,7 @@ UnlockPage(Relation relation, BlockNumber blkno, LOCKMODE lockmode)
relation->rd_lockInfo.lockRelId.relId, relation->rd_lockInfo.lockRelId.relId,
blkno); blkno);
LockRelease(LockTableId, &tag, GetTopTransactionId(), lockmode); LockRelease(LockTableId, &tag, lockmode, false);
} }
/* /*
...@@ -365,8 +365,8 @@ LockTuple(Relation relation, ItemPointer tid, LOCKMODE lockmode) ...@@ -365,8 +365,8 @@ LockTuple(Relation relation, ItemPointer tid, LOCKMODE lockmode)
ItemPointerGetBlockNumber(tid), ItemPointerGetBlockNumber(tid),
ItemPointerGetOffsetNumber(tid)); ItemPointerGetOffsetNumber(tid));
(void) LockAcquire(LockTableId, &tag, GetTopTransactionId(), (void) LockAcquire(LockTableId, &tag, relation->rd_istemp,
lockmode, false); lockmode, false, false);
} }
/* /*
...@@ -383,7 +383,7 @@ UnlockTuple(Relation relation, ItemPointer tid, LOCKMODE lockmode) ...@@ -383,7 +383,7 @@ UnlockTuple(Relation relation, ItemPointer tid, LOCKMODE lockmode)
ItemPointerGetBlockNumber(tid), ItemPointerGetBlockNumber(tid),
ItemPointerGetOffsetNumber(tid)); ItemPointerGetOffsetNumber(tid));
LockRelease(LockTableId, &tag, GetTopTransactionId(), lockmode); LockRelease(LockTableId, &tag, lockmode, false);
} }
/* /*
...@@ -400,8 +400,8 @@ XactLockTableInsert(TransactionId xid) ...@@ -400,8 +400,8 @@ XactLockTableInsert(TransactionId xid)
SET_LOCKTAG_TRANSACTION(tag, xid); SET_LOCKTAG_TRANSACTION(tag, xid);
(void) LockAcquire(LockTableId, &tag, GetTopTransactionId(), (void) LockAcquire(LockTableId, &tag, false,
ExclusiveLock, false); ExclusiveLock, false, false);
} }
/* /*
...@@ -419,7 +419,7 @@ XactLockTableDelete(TransactionId xid) ...@@ -419,7 +419,7 @@ XactLockTableDelete(TransactionId xid)
SET_LOCKTAG_TRANSACTION(tag, xid); SET_LOCKTAG_TRANSACTION(tag, xid);
LockRelease(LockTableId, &tag, GetTopTransactionId(), ExclusiveLock); LockRelease(LockTableId, &tag, ExclusiveLock, false);
} }
/* /*
...@@ -438,19 +438,18 @@ void ...@@ -438,19 +438,18 @@ void
XactLockTableWait(TransactionId xid) XactLockTableWait(TransactionId xid)
{ {
LOCKTAG tag; LOCKTAG tag;
TransactionId myxid = GetTopTransactionId();
for (;;) for (;;)
{ {
Assert(TransactionIdIsValid(xid)); Assert(TransactionIdIsValid(xid));
Assert(!TransactionIdEquals(xid, myxid)); Assert(!TransactionIdEquals(xid, GetTopTransactionId()));
SET_LOCKTAG_TRANSACTION(tag, xid); SET_LOCKTAG_TRANSACTION(tag, xid);
(void) LockAcquire(LockTableId, &tag, myxid, (void) LockAcquire(LockTableId, &tag, false,
ShareLock, false); ShareLock, false, false);
LockRelease(LockTableId, &tag, myxid, ShareLock); LockRelease(LockTableId, &tag, ShareLock, false);
if (!TransactionIdIsInProgress(xid)) if (!TransactionIdIsInProgress(xid))
break; break;
...@@ -470,9 +469,11 @@ XactLockTableWait(TransactionId xid) ...@@ -470,9 +469,11 @@ XactLockTableWait(TransactionId xid)
* LockDatabaseObject * LockDatabaseObject
* *
* Obtain a lock on a general object of the current database. Don't use * Obtain a lock on a general object of the current database. Don't use
* this for shared objects (such as tablespaces). It's usually unwise to * this for shared objects (such as tablespaces). It's unwise to apply it
* apply it to entire relations, also, since a lock taken this way will * to relations, also, since a lock taken this way will NOT conflict with
* NOT conflict with LockRelation. * LockRelation, and also may be wrongly marked if the relation is temp.
* (If we ever invent temp objects that aren't tables, we'll want to extend
* the API of this routine to include an isTempObject flag.)
*/ */
void void
LockDatabaseObject(Oid classid, Oid objid, uint16 objsubid, LockDatabaseObject(Oid classid, Oid objid, uint16 objsubid,
...@@ -486,8 +487,8 @@ LockDatabaseObject(Oid classid, Oid objid, uint16 objsubid, ...@@ -486,8 +487,8 @@ LockDatabaseObject(Oid classid, Oid objid, uint16 objsubid,
objid, objid,
objsubid); objsubid);
(void) LockAcquire(LockTableId, &tag, GetTopTransactionId(), (void) LockAcquire(LockTableId, &tag, false,
lockmode, false); lockmode, false, false);
} }
/* /*
...@@ -505,7 +506,7 @@ UnlockDatabaseObject(Oid classid, Oid objid, uint16 objsubid, ...@@ -505,7 +506,7 @@ UnlockDatabaseObject(Oid classid, Oid objid, uint16 objsubid,
objid, objid,
objsubid); objsubid);
LockRelease(LockTableId, &tag, GetTopTransactionId(), lockmode); LockRelease(LockTableId, &tag, lockmode, false);
} }
/* /*
...@@ -525,8 +526,8 @@ LockSharedObject(Oid classid, Oid objid, uint16 objsubid, ...@@ -525,8 +526,8 @@ LockSharedObject(Oid classid, Oid objid, uint16 objsubid,
objid, objid,
objsubid); objsubid);
(void) LockAcquire(LockTableId, &tag, GetTopTransactionId(), (void) LockAcquire(LockTableId, &tag, false,
lockmode, false); lockmode, false, false);
} }
/* /*
...@@ -544,5 +545,5 @@ UnlockSharedObject(Oid classid, Oid objid, uint16 objsubid, ...@@ -544,5 +545,5 @@ UnlockSharedObject(Oid classid, Oid objid, uint16 objsubid,
objid, objid,
objsubid); objsubid);
LockRelease(LockTableId, &tag, GetTopTransactionId(), lockmode); LockRelease(LockTableId, &tag, lockmode, false);
} }
This diff is collapsed.
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.158 2005/05/19 21:35:46 tgl Exp $ * $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.159 2005/06/14 22:15:32 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -644,8 +644,7 @@ ProcSleep(LockMethod lockMethodTable, ...@@ -644,8 +644,7 @@ ProcSleep(LockMethod lockMethodTable,
lockmode, lockmode,
lock, lock,
proclock, proclock,
MyProc, MyProc) == STATUS_OK)
NULL) == STATUS_OK)
{ {
/* Skip the wait and just grant myself the lock. */ /* Skip the wait and just grant myself the lock. */
GrantLock(lock, proclock, lockmode); GrantLock(lock, proclock, lockmode);
...@@ -846,8 +845,7 @@ ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock) ...@@ -846,8 +845,7 @@ ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock)
lockmode, lockmode,
lock, lock,
proc->waitProcLock, proc->waitProcLock,
proc, proc) == STATUS_OK)
NULL) == STATUS_OK)
{ {
/* OK to waken */ /* OK to waken */
GrantLock(lock, proc->waitProcLock, lockmode); GrantLock(lock, proc->waitProcLock, lockmode);
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/lmgr.h,v 1.48 2005/04/30 19:03:33 tgl Exp $ * $PostgreSQL: pgsql/src/include/storage/lmgr.h,v 1.49 2005/06/14 22:15:33 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -49,7 +49,8 @@ extern void LockRelation(Relation relation, LOCKMODE lockmode); ...@@ -49,7 +49,8 @@ extern void LockRelation(Relation relation, LOCKMODE lockmode);
extern bool ConditionalLockRelation(Relation relation, LOCKMODE lockmode); extern bool ConditionalLockRelation(Relation relation, LOCKMODE lockmode);
extern void UnlockRelation(Relation relation, LOCKMODE lockmode); extern void UnlockRelation(Relation relation, LOCKMODE lockmode);
extern void LockRelationForSession(LockRelId *relid, LOCKMODE lockmode); extern void LockRelationForSession(LockRelId *relid, bool istemprel,
LOCKMODE lockmode);
extern void UnlockRelationForSession(LockRelId *relid, LOCKMODE lockmode); extern void UnlockRelationForSession(LockRelId *relid, LOCKMODE lockmode);
/* Lock a relation for extension */ /* Lock a relation for extension */
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.87 2005/05/29 22:45:02 tgl Exp $ * $PostgreSQL: pgsql/src/include/storage/lock.h,v 1.88 2005/06/14 22:15:33 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -212,6 +212,10 @@ typedef struct LOCKTAG ...@@ -212,6 +212,10 @@ typedef struct LOCKTAG
* nRequested -- total requested locks of all types. * nRequested -- total requested locks of all types.
* granted -- count of each lock type currently granted on the lock. * granted -- count of each lock type currently granted on the lock.
* nGranted -- total granted locks of all types. * nGranted -- total granted locks of all types.
*
* Note: these counts count 1 for each backend. Internally to a backend,
* there may be multiple grabs on a particular lock, but this is not reflected
* into shared memory.
*/ */
typedef struct LOCK typedef struct LOCK
{ {
...@@ -235,7 +239,7 @@ typedef struct LOCK ...@@ -235,7 +239,7 @@ typedef struct LOCK
/* /*
* We may have several different transactions holding or awaiting locks * We may have several different backends holding or awaiting locks
* on the same lockable object. We need to store some per-holder/waiter * on the same lockable object. We need to store some per-holder/waiter
* information for each such holder (or would-be holder). This is kept in * information for each such holder (or would-be holder). This is kept in
* a PROCLOCK struct. * a PROCLOCK struct.
...@@ -244,14 +248,11 @@ typedef struct LOCK ...@@ -244,14 +248,11 @@ typedef struct LOCK
* proclock hashtable. A PROCLOCKTAG value uniquely identifies the combination * proclock hashtable. A PROCLOCKTAG value uniquely identifies the combination
* of a lockable object and a holder/waiter for that object. * of a lockable object and a holder/waiter for that object.
* *
* There are two possible kinds of proclock owners: a transaction (identified * Internally to a backend, it is possible for the same lock to be held
* both by the PGPROC of the backend running it, and the xact's own ID) and * for different purposes: the backend tracks transaction locks separately
* a session (identified by backend PGPROC, with XID = InvalidTransactionId). * from session locks. However, this is not reflected in the shared-memory
* * state: we only track which backend(s) hold the lock. This is OK since a
* Currently, session proclocks are used for user locks and for cross-xact * backend can never block itself.
* locks obtained for VACUUM. Note that a single backend can hold locks
* under several different XIDs at once (including session locks). We treat
* such locks as never conflicting (a backend can never block itself).
* *
* The holdMask field shows the already-granted locks represented by this * The holdMask field shows the already-granted locks represented by this
* proclock. Note that there will be a proclock object, possibly with * proclock. Note that there will be a proclock object, possibly with
...@@ -259,6 +260,10 @@ typedef struct LOCK ...@@ -259,6 +260,10 @@ typedef struct LOCK
* Otherwise, proclock objects whose holdMasks are zero are recycled * Otherwise, proclock objects whose holdMasks are zero are recycled
* as soon as convenient. * as soon as convenient.
* *
* releaseMask is workspace for LockReleaseAll(): it shows the locks due
* to be released during the current call. This must only be examined or
* set by the backend owning the PROCLOCK.
*
* Each PROCLOCK object is linked into lists for both the associated LOCK * Each PROCLOCK object is linked into lists for both the associated LOCK
* object and the owning PGPROC object. Note that the PROCLOCK is entered * object and the owning PGPROC object. Note that the PROCLOCK is entered
* into these lists as soon as it is created, even if no lock has yet been * into these lists as soon as it is created, even if no lock has yet been
...@@ -269,7 +274,6 @@ typedef struct PROCLOCKTAG ...@@ -269,7 +274,6 @@ typedef struct PROCLOCKTAG
{ {
SHMEM_OFFSET lock; /* link to per-lockable-object information */ SHMEM_OFFSET lock; /* link to per-lockable-object information */
SHMEM_OFFSET proc; /* link to PGPROC of owning backend */ SHMEM_OFFSET proc; /* link to PGPROC of owning backend */
TransactionId xid; /* xact ID, or InvalidTransactionId */
} PROCLOCKTAG; } PROCLOCKTAG;
typedef struct PROCLOCK typedef struct PROCLOCK
...@@ -279,9 +283,9 @@ typedef struct PROCLOCK ...@@ -279,9 +283,9 @@ typedef struct PROCLOCK
/* data */ /* data */
LOCKMASK holdMask; /* bitmask for lock types currently held */ LOCKMASK holdMask; /* bitmask for lock types currently held */
SHM_QUEUE lockLink; /* list link for lock's list of proclocks */ LOCKMASK releaseMask; /* bitmask for lock types to be released */
SHM_QUEUE procLink; /* list link for process's list of SHM_QUEUE lockLink; /* list link in LOCK's list of proclocks */
* proclocks */ SHM_QUEUE procLink; /* list link in PGPROC's list of proclocks */
} PROCLOCK; } PROCLOCK;
#define PROCLOCK_LOCKMETHOD(proclock) \ #define PROCLOCK_LOCKMETHOD(proclock) \
...@@ -299,15 +303,16 @@ typedef struct PROCLOCK ...@@ -299,15 +303,16 @@ typedef struct PROCLOCK
typedef struct LOCALLOCKTAG typedef struct LOCALLOCKTAG
{ {
LOCKTAG lock; /* identifies the lockable object */ LOCKTAG lock; /* identifies the lockable object */
TransactionId xid; /* xact ID, or InvalidTransactionId */
LOCKMODE mode; /* lock mode for this table entry */ LOCKMODE mode; /* lock mode for this table entry */
} LOCALLOCKTAG; } LOCALLOCKTAG;
typedef struct LOCALLOCKOWNER typedef struct LOCALLOCKOWNER
{ {
/* /*
* Note: owner can be NULL to indicate a non-transactional lock. Must * Note: if owner is NULL then the lock is held on behalf of the session;
* use a forward struct reference to avoid circularity. * otherwise it is held on behalf of my current transaction.
*
* Must use a forward struct reference to avoid circularity.
*/ */
struct ResourceOwnerData *owner; struct ResourceOwnerData *owner;
int nLocks; /* # of times held by this owner */ int nLocks; /* # of times held by this owner */
...@@ -321,6 +326,7 @@ typedef struct LOCALLOCK ...@@ -321,6 +326,7 @@ typedef struct LOCALLOCK
/* data */ /* data */
LOCK *lock; /* associated LOCK object in shared mem */ LOCK *lock; /* associated LOCK object in shared mem */
PROCLOCK *proclock; /* associated PROCLOCK object in shmem */ PROCLOCK *proclock; /* associated PROCLOCK object in shmem */
bool isTempObject; /* true if lock is on a temporary object */
int nLocks; /* total number of times lock is held */ int nLocks; /* total number of times lock is held */
int numLockOwners; /* # of relevant ResourceOwners */ int numLockOwners; /* # of relevant ResourceOwners */
int maxLockOwners; /* allocated size of array */ int maxLockOwners; /* allocated size of array */
...@@ -366,17 +372,20 @@ extern LOCKMETHODID LockMethodTableInit(const char *tabName, ...@@ -366,17 +372,20 @@ extern LOCKMETHODID LockMethodTableInit(const char *tabName,
const LOCKMASK *conflictsP, const LOCKMASK *conflictsP,
int numModes, int maxBackends); int numModes, int maxBackends);
extern LOCKMETHODID LockMethodTableRename(LOCKMETHODID lockmethodid); extern LOCKMETHODID LockMethodTableRename(LOCKMETHODID lockmethodid);
extern LockAcquireResult LockAcquire(LOCKMETHODID lockmethodid, LOCKTAG *locktag, extern LockAcquireResult LockAcquire(LOCKMETHODID lockmethodid,
TransactionId xid, LOCKMODE lockmode, bool dontWait); LOCKTAG *locktag,
bool isTempObject,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait);
extern bool LockRelease(LOCKMETHODID lockmethodid, LOCKTAG *locktag, extern bool LockRelease(LOCKMETHODID lockmethodid, LOCKTAG *locktag,
TransactionId xid, LOCKMODE lockmode); LOCKMODE lockmode, bool sessionLock);
extern void LockReleaseAll(LOCKMETHODID lockmethodid, bool allxids); extern void LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks);
extern void LockReleaseCurrentOwner(void); extern void LockReleaseCurrentOwner(void);
extern void LockReassignCurrentOwner(void); extern void LockReassignCurrentOwner(void);
extern int LockCheckConflicts(LockMethod lockMethodTable, extern int LockCheckConflicts(LockMethod lockMethodTable,
LOCKMODE lockmode, LOCKMODE lockmode,
LOCK *lock, PROCLOCK *proclock, PGPROC *proc, LOCK *lock, PROCLOCK *proclock, PGPROC *proc);
int *myHolding);
extern void GrantLock(LOCK *lock, PROCLOCK *proclock, LOCKMODE lockmode); extern void GrantLock(LOCK *lock, PROCLOCK *proclock, LOCKMODE lockmode);
extern void GrantAwaitedLock(void); extern void GrantAwaitedLock(void);
extern void RemoveFromWaitQueue(PGPROC *proc); extern void RemoveFromWaitQueue(PGPROC *proc);
......
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