"AccessExclusiveLock"
};
+/*
+ * Count of the number of fast path lock slots we believe to be used. This
+ * might be higher than the real number if another backend has transferred
+ * our locks to the primary lock table, but it can never be lower than the
+ * real value, since only we can acquire locks on our own behalf.
+ */
+static int FastPathLocalUseCount = 0;
+
+/* Macros for manipulating proc->fpLockBits */
+#define FAST_PATH_BITS_PER_SLOT 3
+#define FAST_PATH_LOCKNUMBER_OFFSET 1
+#define FAST_PATH_MASK ((1 << FAST_PATH_BITS_PER_SLOT) - 1)
+#define FAST_PATH_GET_BITS(proc, n) \
+ (((proc)->fpLockBits >> (FAST_PATH_BITS_PER_SLOT * n)) & FAST_PATH_MASK)
+#define FAST_PATH_BIT_POSITION(n, l) \
+ (AssertMacro((l) >= FAST_PATH_LOCKNUMBER_OFFSET), \
+ AssertMacro((l) < FAST_PATH_BITS_PER_SLOT+FAST_PATH_LOCKNUMBER_OFFSET), \
+ AssertMacro((n) < FP_LOCK_SLOTS_PER_BACKEND), \
+ ((l) - FAST_PATH_LOCKNUMBER_OFFSET + FAST_PATH_BITS_PER_SLOT * (n)))
+#define FAST_PATH_SET_LOCKMODE(proc, n, l) \
+ (proc)->fpLockBits |= UINT64CONST(1) << FAST_PATH_BIT_POSITION(n, l)
+#define FAST_PATH_CLEAR_LOCKMODE(proc, n, l) \
+ (proc)->fpLockBits &= ~(UINT64CONST(1) << FAST_PATH_BIT_POSITION(n, l))
+#define FAST_PATH_CHECK_LOCKMODE(proc, n, l) \
+ ((proc)->fpLockBits & (UINT64CONST(1) << FAST_PATH_BIT_POSITION(n, l)))
+
+/*
+ * The fast-path lock mechanism is concerned only with relation locks on
+ * unshared relations by backends bound to a database. The fast-path
+ * mechanism exists mostly to accelerate acquisition and release of locks
+ * that rarely conflict. Because ShareUpdateExclusiveLock is
+ * self-conflicting, it can't use the fast-path mechanism; but it also does
+ * not conflict with any of the locks that do, so we can ignore it completely.
+ */
+#define FastPathTag(locktag) \
+ ((locktag)->locktag_lockmethodid == DEFAULT_LOCKMETHOD && \
+ (locktag)->locktag_type == LOCKTAG_RELATION && \
+ (locktag)->locktag_field1 == MyDatabaseId && \
+ MyDatabaseId != InvalidOid)
+#define FastPathWeakMode(mode) ((mode) < ShareUpdateExclusiveLock)
+#define FastPathStrongMode(mode) ((mode) > ShareUpdateExclusiveLock)
+#define FastPathRelevantMode(mode) ((mode) != ShareUpdateExclusiveLock)
+
+static bool FastPathGrantLock(Oid relid, LOCKMODE lockmode);
+static bool FastPathUnGrantLock(Oid relid, LOCKMODE lockmode);
+static bool FastPathTransferLocks(LockMethod lockMethodTable,
+ const LOCKTAG *locktag, uint32 hashcode);
+static PROCLOCK *FastPathGetLockEntry(LOCALLOCK *locallock);
+
+/*
+ * To make the fast-path lock mechanism work, we must have some way of
+ * preventing the use of the fast-path when a conflicting lock might be
+ * present. We partition* the locktag space into FAST_PATH_HASH_BUCKETS
+ * partitions, and maintain an integer count of the number of "strong" lockers
+ * in each partition. When any "strong" lockers are present (which is
+ * hopefully not very often), the fast-path mechanism can't be used, and we
+ * must fall back to the slower method of pushing matching locks directly
+ * into the main lock tables.
+ *
+ * The deadlock detector does not know anything about the fast path mechanism,
+ * so any locks that might be involved in a deadlock must be transferred from
+ * the fast-path queues to the main lock table.
+ */
+
+#define FAST_PATH_STRONG_LOCK_HASH_BITS 10
+#define FAST_PATH_STRONG_LOCK_HASH_PARTITIONS \
+ (1 << FAST_PATH_STRONG_LOCK_HASH_BITS)
+#define FastPathStrongLockHashPartition(hashcode) \
+ ((hashcode) % FAST_PATH_STRONG_LOCK_HASH_PARTITIONS)
+
+typedef struct
+{
+ slock_t mutex;
+ uint32 count[FAST_PATH_STRONG_LOCK_HASH_PARTITIONS];
+} FastPathStrongLockData;
+
+FastPathStrongLockData *FastPathStrongLocks;
+
#ifndef LOCK_DEBUG
static bool Dummy_trace = false;
#endif
static uint32 proclock_hash(const void *key, Size keysize);
static void RemoveLocalLock(LOCALLOCK *locallock);
+static PROCLOCK *SetupLockInTable(LockMethod lockMethodTable, PGPROC *proc,
+ const LOCKTAG *locktag, uint32 hashcode, LOCKMODE lockmode);
static void GrantLockLocal(LOCALLOCK *locallock, ResourceOwner owner);
static void WaitOnLock(LOCALLOCK *locallock, ResourceOwner owner);
static void ReleaseLockForOwner(LOCALLOCK *locallock, ResourceOwner owner);
static void CleanUpLock(LOCK *lock, PROCLOCK *proclock,
LockMethod lockMethodTable, uint32 hashcode,
bool wakeupNeeded);
+static void LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
+ LOCKTAG *locktag, LOCKMODE lockmode,
+ bool decrement_strong_lock_count);
/*
int hash_flags;
long init_table_size,
max_table_size;
+ bool found;
/*
* Compute init/max size to request for lock hashtables. Note these
&info,
hash_flags);
+ /*
+ * Allocate fast-path structures.
+ */
+ FastPathStrongLocks = ShmemInitStruct("Fast Path Strong Lock Data",
+ sizeof(FastPathStrongLockData), &found);
+ if (!found)
+ SpinLockInit(&FastPathStrongLocks->mutex);
+
/*
* Allocate non-shared hash table for LOCALLOCK structs. This stores lock
* counts and resource owner information.
LOCALLOCK *locallock;
LOCK *lock;
PROCLOCK *proclock;
- PROCLOCKTAG proclocktag;
bool found;
ResourceOwner owner;
uint32 hashcode;
- uint32 proclock_hashcode;
- int partition;
LWLockId partitionLock;
int status;
bool log_lock = false;
locallock->nLocks = 0;
locallock->numLockOwners = 0;
locallock->maxLockOwners = 8;
+ locallock->holdsStrongLockCount = FALSE;
locallock->lockOwners = NULL;
locallock->lockOwners = (LOCALLOCKOWNER *)
MemoryContextAlloc(TopMemoryContext,
locallock->maxLockOwners = newsize;
}
}
+ hashcode = locallock->hashcode;
/*
* If we already hold the lock, we can just increase the count locally.
log_lock = true;
}
+ /* Locks that participate in the fast path require special handling. */
+ if (FastPathTag(locktag) && FastPathRelevantMode(lockmode))
+ {
+ uint32 fasthashcode;
+
+ fasthashcode = FastPathStrongLockHashPartition(hashcode);
+
+ /*
+ * If we remember having filled up the fast path array, we don't
+ * attempt to make any further use of it until we release some locks.
+ * It's possible that some other backend has transferred some of those
+ * locks to the shared hash table, leaving space free, but it's not
+ * worth acquiring the LWLock just to check. It's also possible that
+ * we're acquiring a second or third lock type on a relation we have
+ * already locked using the fast-path, but for now we don't worry about
+ * that case either.
+ */
+ if (FastPathWeakMode(lockmode)
+ && FastPathLocalUseCount < FP_LOCK_SLOTS_PER_BACKEND)
+ {
+ bool acquired;
+
+ /*
+ * LWLockAcquire acts as a memory sequencing point, so it's safe
+ * to assume that any strong locker whose increment to
+ * FastPathStrongLocks->counts becomes visible after we test it has
+ * yet to begin to transfer fast-path locks.
+ */
+ LWLockAcquire(MyProc->backendLock, LW_EXCLUSIVE);
+ if (FastPathStrongLocks->count[fasthashcode] != 0)
+ acquired = false;
+ else
+ acquired = FastPathGrantLock(locktag->locktag_field2, lockmode);
+ LWLockRelease(MyProc->backendLock);
+ if (acquired)
+ {
+ GrantLockLocal(locallock, owner);
+ return LOCKACQUIRE_OK;
+ }
+ }
+ else if (FastPathStrongMode(lockmode))
+ {
+ /*
+ * Adding to a memory location is not atomic, so we take a
+ * spinlock to ensure we don't collide with someone else trying
+ * to bump the count at the same time.
+ *
+ * XXX: It might be worth considering using an atomic fetch-and-add
+ * instruction here, on architectures where that is supported.
+ */
+ Assert(locallock->holdsStrongLockCount == FALSE);
+ SpinLockAcquire(&FastPathStrongLocks->mutex);
+ FastPathStrongLocks->count[fasthashcode]++;
+ locallock->holdsStrongLockCount = TRUE;
+ SpinLockRelease(&FastPathStrongLocks->mutex);
+ if (!FastPathTransferLocks(lockMethodTable, locktag, hashcode))
+ {
+ if (reportMemoryError)
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of shared memory"),
+ errhint("You might need to increase max_locks_per_transaction.")));
+ else
+ return LOCKACQUIRE_NOT_AVAIL;
+ }
+ }
+ }
+
/*
* Otherwise we've got to mess with the shared lock table.
*/
- hashcode = locallock->hashcode;
- partition = LockHashPartition(hashcode);
partitionLock = LockHashPartitionLock(hashcode);
LWLockAcquire(partitionLock, LW_EXCLUSIVE);
+ /*
+ * Find or create a proclock entry with this tag
+ */
+ proclock = SetupLockInTable(lockMethodTable, MyProc, locktag,
+ hashcode, lockmode);
+ if (!proclock)
+ {
+ LWLockRelease(partitionLock);
+ if (reportMemoryError)
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of shared memory"),
+ errhint("You might need to increase max_locks_per_transaction.")));
+ else
+ return LOCKACQUIRE_NOT_AVAIL;
+ }
+ locallock->proclock = proclock;
+ lock = proclock->tag.myLock;
+ locallock->lock = lock;
+
+ /*
+ * If lock requested conflicts with locks requested by waiters, must join
+ * wait queue. Otherwise, check for conflict with already-held locks.
+ * (That's last because most complex check.)
+ */
+ if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
+ status = STATUS_FOUND;
+ else
+ status = LockCheckConflicts(lockMethodTable, lockmode,
+ lock, proclock, MyProc);
+
+ if (status == STATUS_OK)
+ {
+ /* No conflict with held or previously requested locks */
+ GrantLock(lock, proclock, lockmode);
+ GrantLockLocal(locallock, owner);
+ }
+ else
+ {
+ Assert(status == STATUS_FOUND);
+
+ /*
+ * We can't acquire the lock immediately. If caller specified no
+ * blocking, remove useless table entries and return NOT_AVAIL without
+ * waiting.
+ */
+ if (dontWait)
+ {
+ if (proclock->holdMask == 0)
+ {
+ uint32 proclock_hashcode;
+
+ proclock_hashcode = ProcLockHashCode(&proclock->tag, hashcode);
+ SHMQueueDelete(&proclock->lockLink);
+ SHMQueueDelete(&proclock->procLink);
+ if (!hash_search_with_hash_value(LockMethodProcLockHash,
+ (void *) &(proclock->tag),
+ proclock_hashcode,
+ HASH_REMOVE,
+ NULL))
+ elog(PANIC, "proclock table corrupted");
+ }
+ else
+ PROCLOCK_PRINT("LockAcquire: NOWAIT", proclock);
+ lock->nRequested--;
+ lock->requested[lockmode]--;
+ LOCK_PRINT("LockAcquire: conditional lock failed", lock, lockmode);
+ Assert((lock->nRequested > 0) && (lock->requested[lockmode] >= 0));
+ Assert(lock->nGranted <= lock->nRequested);
+ LWLockRelease(partitionLock);
+ if (locallock->nLocks == 0)
+ RemoveLocalLock(locallock);
+ return LOCKACQUIRE_NOT_AVAIL;
+ }
+
+ /*
+ * In Hot Standby perform early deadlock detection in normal backends.
+ * If deadlock found we release partition lock but do not return.
+ */
+ if (RecoveryInProgress() && !InRecovery)
+ CheckRecoveryConflictDeadlock(partitionLock);
+
+ /*
+ * Set bitmask of locks this process already holds on this object.
+ */
+ MyProc->heldLocks = proclock->holdMask;
+
+ /*
+ * Sleep till someone wakes me up.
+ */
+
+ TRACE_POSTGRESQL_LOCK_WAIT_START(locktag->locktag_field1,
+ locktag->locktag_field2,
+ locktag->locktag_field3,
+ locktag->locktag_field4,
+ locktag->locktag_type,
+ lockmode);
+
+ WaitOnLock(locallock, owner);
+
+ TRACE_POSTGRESQL_LOCK_WAIT_DONE(locktag->locktag_field1,
+ locktag->locktag_field2,
+ locktag->locktag_field3,
+ locktag->locktag_field4,
+ locktag->locktag_type,
+ lockmode);
+
+ /*
+ * NOTE: do not do any material change of state between here and
+ * return. All required changes in locktable state must have been
+ * done when the lock was granted to us --- see notes in WaitOnLock.
+ */
+
+ /*
+ * Check the proclock entry status, in case something in the ipc
+ * communication doesn't work correctly.
+ */
+ if (!(proclock->holdMask & LOCKBIT_ON(lockmode)))
+ {
+ PROCLOCK_PRINT("LockAcquire: INCONSISTENT", proclock);
+ LOCK_PRINT("LockAcquire: INCONSISTENT", lock, lockmode);
+ /* Should we retry ? */
+ LWLockRelease(partitionLock);
+ elog(ERROR, "LockAcquire failed");
+ }
+ PROCLOCK_PRINT("LockAcquire: granted", proclock);
+ LOCK_PRINT("LockAcquire: granted", lock, lockmode);
+ }
+
+ LWLockRelease(partitionLock);
+
+ /*
+ * Emit a WAL record if acquisition of this lock need to be replayed in a
+ * standby server.
+ */
+ if (log_lock)
+ {
+ /*
+ * Decode the locktag back to the original values, to avoid sending
+ * lots of empty bytes with every message. See lock.h to check how a
+ * locktag is defined for LOCKTAG_RELATION
+ */
+ LogAccessExclusiveLock(locktag->locktag_field1,
+ locktag->locktag_field2);
+ }
+
+ return LOCKACQUIRE_OK;
+}
+
+/*
+ * Find or create LOCK and PROCLOCK objects as needed for a new lock
+ * request.
+ */
+static PROCLOCK *
+SetupLockInTable(LockMethod lockMethodTable, PGPROC *proc,
+ const LOCKTAG *locktag, uint32 hashcode, LOCKMODE lockmode)
+{
+ LOCK *lock;
+ PROCLOCK *proclock;
+ PROCLOCKTAG proclocktag;
+ uint32 proclock_hashcode;
+ bool found;
+
/*
* Find or create a lock with this tag.
*
HASH_ENTER_NULL,
&found);
if (!lock)
- {
- LWLockRelease(partitionLock);
- if (reportMemoryError)
- ereport(ERROR,
- (errcode(ERRCODE_OUT_OF_MEMORY),
- errmsg("out of shared memory"),
- errhint("You might need to increase max_locks_per_transaction.")));
- else
- return LOCKACQUIRE_NOT_AVAIL;
- }
- locallock->lock = lock;
+ return NULL;
/*
* if it's a new lock object, initialize it
* Create the hash key for the proclock table.
*/
proclocktag.myLock = lock;
- proclocktag.myProc = MyProc;
+ proclocktag.myProc = proc;
proclock_hashcode = ProcLockHashCode(&proclocktag, hashcode);
NULL))
elog(PANIC, "lock table corrupted");
}
- LWLockRelease(partitionLock);
- if (reportMemoryError)
- ereport(ERROR,
- (errcode(ERRCODE_OUT_OF_MEMORY),
- errmsg("out of shared memory"),
- errhint("You might need to increase max_locks_per_transaction.")));
- else
- return LOCKACQUIRE_NOT_AVAIL;
+ return NULL;
}
- locallock->proclock = proclock;
/*
* If new, initialize the new entry
*/
if (!found)
{
+ uint32 partition = LockHashPartition(hashcode);
+
proclock->holdMask = 0;
proclock->releaseMask = 0;
/* Add proclock to appropriate lists */
SHMQueueInsertBefore(&lock->procLocks, &proclock->lockLink);
- SHMQueueInsertBefore(&(MyProc->myProcLocks[partition]),
+ SHMQueueInsertBefore(&(proc->myProcLocks[partition]),
&proclock->procLink);
PROCLOCK_PRINT("LockAcquire: new", proclock);
}
lock->tag.locktag_field1, lock->tag.locktag_field2,
lock->tag.locktag_field3);
- /*
- * If lock requested conflicts with locks requested by waiters, must join
- * wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
- */
- if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
- status = STATUS_FOUND;
- else
- status = LockCheckConflicts(lockMethodTable, lockmode,
- lock, proclock, MyProc);
-
- if (status == STATUS_OK)
- {
- /* No conflict with held or previously requested locks */
- GrantLock(lock, proclock, lockmode);
- GrantLockLocal(locallock, owner);
- }
- else
- {
- Assert(status == STATUS_FOUND);
-
- /*
- * We can't acquire the lock immediately. If caller specified no
- * blocking, remove useless table entries and return NOT_AVAIL without
- * waiting.
- */
- if (dontWait)
- {
- if (proclock->holdMask == 0)
- {
- SHMQueueDelete(&proclock->lockLink);
- SHMQueueDelete(&proclock->procLink);
- if (!hash_search_with_hash_value(LockMethodProcLockHash,
- (void *) &(proclock->tag),
- proclock_hashcode,
- HASH_REMOVE,
- NULL))
- elog(PANIC, "proclock table corrupted");
- }
- else
- PROCLOCK_PRINT("LockAcquire: NOWAIT", proclock);
- lock->nRequested--;
- lock->requested[lockmode]--;
- LOCK_PRINT("LockAcquire: conditional lock failed", lock, lockmode);
- Assert((lock->nRequested > 0) && (lock->requested[lockmode] >= 0));
- Assert(lock->nGranted <= lock->nRequested);
- LWLockRelease(partitionLock);
- if (locallock->nLocks == 0)
- RemoveLocalLock(locallock);
- return LOCKACQUIRE_NOT_AVAIL;
- }
-
- /*
- * In Hot Standby perform early deadlock detection in normal backends.
- * If deadlock found we release partition lock but do not return.
- */
- if (RecoveryInProgress() && !InRecovery)
- CheckRecoveryConflictDeadlock(partitionLock);
-
- /*
- * Set bitmask of locks this process already holds on this object.
- */
- MyProc->heldLocks = proclock->holdMask;
-
- /*
- * Sleep till someone wakes me up.
- */
-
- TRACE_POSTGRESQL_LOCK_WAIT_START(locktag->locktag_field1,
- locktag->locktag_field2,
- locktag->locktag_field3,
- locktag->locktag_field4,
- locktag->locktag_type,
- lockmode);
-
- WaitOnLock(locallock, owner);
-
- TRACE_POSTGRESQL_LOCK_WAIT_DONE(locktag->locktag_field1,
- locktag->locktag_field2,
- locktag->locktag_field3,
- locktag->locktag_field4,
- locktag->locktag_type,
- lockmode);
-
- /*
- * NOTE: do not do any material change of state between here and
- * return. All required changes in locktable state must have been
- * done when the lock was granted to us --- see notes in WaitOnLock.
- */
-
- /*
- * Check the proclock entry status, in case something in the ipc
- * communication doesn't work correctly.
- */
- if (!(proclock->holdMask & LOCKBIT_ON(lockmode)))
- {
- PROCLOCK_PRINT("LockAcquire: INCONSISTENT", proclock);
- LOCK_PRINT("LockAcquire: INCONSISTENT", lock, lockmode);
- /* Should we retry ? */
- LWLockRelease(partitionLock);
- elog(ERROR, "LockAcquire failed");
- }
- PROCLOCK_PRINT("LockAcquire: granted", proclock);
- LOCK_PRINT("LockAcquire: granted", lock, lockmode);
- }
-
- LWLockRelease(partitionLock);
-
- /*
- * Emit a WAL record if acquisition of this lock need to be replayed in a
- * standby server.
- */
- if (log_lock)
- {
- /*
- * Decode the locktag back to the original values, to avoid sending
- * lots of empty bytes with every message. See lock.h to check how a
- * locktag is defined for LOCKTAG_RELATION
- */
- LogAccessExclusiveLock(locktag->locktag_field1,
- locktag->locktag_field2);
- }
-
- return LOCKACQUIRE_OK;
+ return proclock;
}
/*
{
pfree(locallock->lockOwners);
locallock->lockOwners = NULL;
+ if (locallock->holdsStrongLockCount)
+ {
+ uint32 fasthashcode;
+ fasthashcode = FastPathStrongLockHashPartition(locallock->hashcode);
+
+ SpinLockAcquire(&FastPathStrongLocks->mutex);
+ Assert(FastPathStrongLocks->count[fasthashcode] > 0);
+ FastPathStrongLocks->count[fasthashcode]--;
+ locallock->holdsStrongLockCount = FALSE;
+ SpinLockRelease(&FastPathStrongLocks->mutex);
+ }
if (!hash_search(LockMethodLocalHash,
(void *) &(locallock->tag),
HASH_REMOVE, NULL))
if (locallock->nLocks > 0)
return TRUE;
+ /* Locks that participate in the fast path require special handling. */
+ if (FastPathTag(locktag) && FastPathWeakMode(lockmode)
+ && FastPathLocalUseCount > 0)
+ {
+ bool released;
+
+ /*
+ * We might not find the lock here, even if we originally entered
+ * it here. Another backend may have moved it to the main table.
+ */
+ LWLockAcquire(MyProc->backendLock, LW_EXCLUSIVE);
+ released = FastPathUnGrantLock(locktag->locktag_field2, lockmode);
+ LWLockRelease(MyProc->backendLock);
+ if (released)
+ {
+ RemoveLocalLock(locallock);
+ return TRUE;
+ }
+ }
+
/*
* Otherwise we've got to mess with the shared lock table.
*/
LWLockAcquire(partitionLock, LW_EXCLUSIVE);
/*
- * We don't need to re-find the lock or proclock, since we kept their
- * addresses in the locallock table, and they couldn't have been removed
- * while we were holding a lock on them.
+ * Normally, we don't need to re-find the lock or proclock, since we kept
+ * their addresses in the locallock table, and they couldn't have been
+ * removed while we were holding a lock on them. But it's possible that
+ * the locks have been moved to the main hash table by another backend, in
+ * which case we might need to go look them up after all.
*/
lock = locallock->lock;
+ if (!lock)
+ {
+ PROCLOCKTAG proclocktag;
+ bool found;
+
+ Assert(FastPathTag(locktag) && FastPathWeakMode(lockmode));
+ lock = (LOCK *) hash_search_with_hash_value(LockMethodLockHash,
+ (void *) locktag,
+ locallock->hashcode,
+ HASH_FIND,
+ &found);
+ Assert(found && lock != NULL);
+ locallock->lock = lock;
+
+ proclocktag.myLock = lock;
+ proclocktag.myProc = MyProc;
+ locallock->proclock = (PROCLOCK *) hash_search(LockMethodProcLockHash,
+ (void *) &proclocktag,
+ HASH_FIND, &found);
+ Assert(found);
+ }
LOCK_PRINT("LockRelease: found", lock, lockmode);
proclock = locallock->proclock;
PROCLOCK_PRINT("LockRelease: found", proclock);
LOCK *lock;
PROCLOCK *proclock;
int partition;
+ bool have_fast_path_lwlock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
{
if (locallock->proclock == NULL || locallock->lock == NULL)
{
+ LOCKMODE lockmode = locallock->tag.mode;
+ Oid relid;
+
/*
- * We must've run out of shared memory while trying to set up this
- * lock. Just forget the local entry.
+ * If the LOCALLOCK entry is unused, we must've run out of shared
+ * memory while trying to set up this lock. Just forget the local
+ * entry.
*/
- Assert(locallock->nLocks == 0);
+ if (locallock->nLocks == 0)
+ {
+ RemoveLocalLock(locallock);
+ continue;
+ }
+
+ /*
+ * Otherwise, we should be dealing with a lock acquired via the
+ * fast-path. If not, we've got trouble.
+ */
+ if (!FastPathTag(&locallock->tag.lock)
+ || !FastPathWeakMode(lockmode))
+ elog(PANIC, "locallock table corrupted");
+
+ /*
+ * If we don't currently hold the LWLock that protects our
+ * fast-path data structures, we must acquire it before
+ * attempting to release the lock via the fast-path.
+ */
+ if (!have_fast_path_lwlock)
+ {
+ LWLockAcquire(MyProc->backendLock, LW_EXCLUSIVE);
+ have_fast_path_lwlock = true;
+ }
+
+ /* Attempt fast-path release. */
+ relid = locallock->tag.lock.locktag_field2;
+ if (FastPathUnGrantLock(relid, lockmode))
+ {
+ RemoveLocalLock(locallock);
+ continue;
+ }
+
+ /*
+ * Our lock, originally taken via the fast path, has been
+ * transferred to the main lock table. That's going to require
+ * some extra work, so release our fast-path lock before starting.
+ */
+ LWLockRelease(MyProc->backendLock);
+ have_fast_path_lwlock = false;
+
+ /*
+ * Now dump the lock. We haven't got a pointer to the LOCK or
+ * PROCLOCK in this case, so we have to handle this a bit
+ * differently than a normal lock release. Unfortunately, this
+ * requires an extra LWLock acquire-and-release cycle on the
+ * partitionLock, but hopefully it shouldn't happen often.
+ */
+ LockRefindAndRelease(lockMethodTable, MyProc,
+ &locallock->tag.lock, lockmode, false);
RemoveLocalLock(locallock);
continue;
}
RemoveLocalLock(locallock);
}
+ if (have_fast_path_lwlock)
+ LWLockRelease(MyProc->backendLock);
+
/*
* Now, scan each lock partition separately.
*/
if (ic < 0)
continue; /* no current locks */
- if (ip < 0)
- {
- /* Parent has no slot, so just give it child's slot */
- lockOwners[ic].owner = parent;
- }
- else
+ if (ip < 0)
+ {
+ /* Parent has no slot, so just give it child's slot */
+ lockOwners[ic].owner = parent;
+ }
+ else
+ {
+ /* Merge child's count with parent's */
+ lockOwners[ip].nLocks += lockOwners[ic].nLocks;
+ /* compact out unused slot */
+ locallock->numLockOwners--;
+ if (ic < locallock->numLockOwners)
+ lockOwners[ic] = lockOwners[locallock->numLockOwners];
+ }
+ }
+}
+
+/*
+ * FastPathGrantLock
+ * Grant lock using per-backend fast-path array, if there is space.
+ */
+static bool
+FastPathGrantLock(Oid relid, LOCKMODE lockmode)
+{
+ uint32 f;
+ uint32 unused_slot = FP_LOCK_SLOTS_PER_BACKEND;
+
+ /* Scan for existing entry for this relid, remembering empty slot. */
+ for (f = 0; f < FP_LOCK_SLOTS_PER_BACKEND; f++)
+ {
+ if (FAST_PATH_GET_BITS(MyProc, f) == 0)
+ unused_slot = f;
+ else if (MyProc->fpRelId[f] == relid)
+ {
+ Assert(!FAST_PATH_CHECK_LOCKMODE(MyProc, f, lockmode));
+ FAST_PATH_SET_LOCKMODE(MyProc, f, lockmode);
+ return true;
+ }
+ }
+
+ /* If no existing entry, use any empty slot. */
+ if (unused_slot < FP_LOCK_SLOTS_PER_BACKEND)
+ {
+ MyProc->fpRelId[unused_slot] = relid;
+ FAST_PATH_SET_LOCKMODE(MyProc, unused_slot, lockmode);
+ ++FastPathLocalUseCount;
+ return true;
+ }
+
+ /* No existing entry, and no empty slot. */
+ return false;
+}
+
+/*
+ * FastPathUnGrantLock
+ * Release fast-path lock, if present. Update backend-private local
+ * use count, while we're at it.
+ */
+static bool
+FastPathUnGrantLock(Oid relid, LOCKMODE lockmode)
+{
+ uint32 f;
+ bool result = false;
+
+ FastPathLocalUseCount = 0;
+ for (f = 0; f < FP_LOCK_SLOTS_PER_BACKEND; f++)
+ {
+ if (MyProc->fpRelId[f] == relid
+ && FAST_PATH_CHECK_LOCKMODE(MyProc, f, lockmode))
+ {
+ Assert(!result);
+ FAST_PATH_CLEAR_LOCKMODE(MyProc, f, lockmode);
+ result = true;
+ }
+ if (FAST_PATH_GET_BITS(MyProc, f) != 0)
+ ++FastPathLocalUseCount;
+ }
+ return result;
+}
+
+/*
+ * FastPathTransferLocks
+ * Transfer locks matching the given lock tag from per-backend fast-path
+ * arrays to the shared hash table.
+ */
+static bool
+FastPathTransferLocks(LockMethod lockMethodTable, const LOCKTAG *locktag,
+ uint32 hashcode)
+{
+ LWLockId partitionLock = LockHashPartitionLock(hashcode);
+ Oid relid = locktag->locktag_field2;
+ uint32 i;
+
+ /*
+ * Every PGPROC that can potentially hold a fast-path lock is present
+ * in ProcGlobal->allProcs. Prepared transactions are not, but
+ * any outstanding fast-path locks held by prepared transactions are
+ * transferred to the main lock table.
+ */
+ for (i = 0; i < ProcGlobal->allProcCount; i++)
+ {
+ PGPROC *proc = &ProcGlobal->allProcs[i];
+ uint32 f;
+
+ LWLockAcquire(proc->backendLock, LW_EXCLUSIVE);
+
+ /*
+ * If the target backend isn't referencing the same database as we are,
+ * then we needn't examine the individual relation IDs at all; none of
+ * them can be relevant.
+ *
+ * proc->databaseId is set at backend startup time and never changes
+ * thereafter, so it might be safe to perform this test before
+ * acquiring proc->backendLock. In particular, it's certainly safe to
+ * assume that if the target backend holds any fast-path locks, it must
+ * have performed a memory-fencing operation (in particular, an LWLock
+ * acquisition) since setting proc->databaseId. However, it's less
+ * clear that our backend is certain to have performed a memory fencing
+ * operation since the other backend set proc->databaseId. So for now,
+ * we test it after acquiring the LWLock just to be safe.
+ */
+ if (proc->databaseId != MyDatabaseId)
+ {
+ LWLockRelease(proc->backendLock);
+ continue;
+ }
+
+ for (f = 0; f < FP_LOCK_SLOTS_PER_BACKEND; f++)
+ {
+ uint32 lockmode;
+
+ /* Look for an allocated slot matching the given relid. */
+ if (relid != proc->fpRelId[f] || FAST_PATH_GET_BITS(proc, f) == 0)
+ continue;
+
+ /* Find or create lock object. */
+ LWLockAcquire(partitionLock, LW_EXCLUSIVE);
+ for (lockmode = FAST_PATH_LOCKNUMBER_OFFSET;
+ lockmode < FAST_PATH_LOCKNUMBER_OFFSET+FAST_PATH_BITS_PER_SLOT;
+ ++lockmode)
+ {
+ PROCLOCK *proclock;
+
+ if (!FAST_PATH_CHECK_LOCKMODE(proc, f, lockmode))
+ continue;
+ proclock = SetupLockInTable(lockMethodTable, proc, locktag,
+ hashcode, lockmode);
+ if (!proclock)
+ {
+ LWLockRelease(partitionLock);
+ return false;
+ }
+ GrantLock(proclock->tag.myLock, proclock, lockmode);
+ FAST_PATH_CLEAR_LOCKMODE(proc, f, lockmode);
+ }
+ LWLockRelease(partitionLock);
+ }
+ LWLockRelease(proc->backendLock);
+ }
+ return true;
+}
+
+/*
+ * FastPathGetLockEntry
+ * Return the PROCLOCK for a lock originally taken via the fast-path,
+ * transferring it to the primary lock table if necessary.
+ */
+static PROCLOCK *
+FastPathGetLockEntry(LOCALLOCK *locallock)
+{
+ LockMethod lockMethodTable = LockMethods[DEFAULT_LOCKMETHOD];
+ LOCKTAG *locktag = &locallock->tag.lock;
+ PROCLOCK *proclock = NULL;
+ LWLockId partitionLock = LockHashPartitionLock(locallock->hashcode);
+ Oid relid = locktag->locktag_field2;
+ uint32 f;
+
+ LWLockAcquire(MyProc->backendLock, LW_EXCLUSIVE);
+
+ for (f = 0; f < FP_LOCK_SLOTS_PER_BACKEND; f++)
+ {
+ uint32 lockmode;
+
+ /* Look for an allocated slot matching the given relid. */
+ if (relid != MyProc->fpRelId[f] || FAST_PATH_GET_BITS(MyProc, f) == 0)
+ continue;
+
+ /* If we don't have a lock of the given mode, forget it! */
+ lockmode = locallock->tag.mode;
+ if (!FAST_PATH_CHECK_LOCKMODE(MyProc, f, lockmode))
+ break;
+
+ /* Find or create lock object. */
+ LWLockAcquire(partitionLock, LW_EXCLUSIVE);
+
+ proclock = SetupLockInTable(lockMethodTable, MyProc, locktag,
+ locallock->hashcode, lockmode);
+ if (!proclock)
{
- /* Merge child's count with parent's */
- lockOwners[ip].nLocks += lockOwners[ic].nLocks;
- /* compact out unused slot */
- locallock->numLockOwners--;
- if (ic < locallock->numLockOwners)
- lockOwners[ic] = lockOwners[locallock->numLockOwners];
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of shared memory"),
+ errhint("You might need to increase max_locks_per_transaction.")));
}
+ GrantLock(proclock->tag.myLock, proclock, lockmode);
+ FAST_PATH_CLEAR_LOCKMODE(MyProc, f, lockmode);
+
+ LWLockRelease(partitionLock);
+ }
+
+ LWLockRelease(MyProc->backendLock);
+
+ /* Lock may have already been transferred by some other backend. */
+ if (proclock == NULL)
+ {
+ LOCK *lock;
+ PROCLOCKTAG proclocktag;
+ uint32 proclock_hashcode;
+
+ LWLockAcquire(partitionLock, LW_SHARED);
+
+ lock = (LOCK *) hash_search_with_hash_value(LockMethodLockHash,
+ (void *) locktag,
+ locallock->hashcode,
+ HASH_FIND,
+ NULL);
+ if (!lock)
+ elog(ERROR, "failed to re-find shared lock object");
+
+ proclocktag.myLock = lock;
+ proclocktag.myProc = MyProc;
+
+ proclock_hashcode = ProcLockHashCode(&proclocktag, locallock->hashcode);
+ proclock = (PROCLOCK *)
+ hash_search_with_hash_value(LockMethodProcLockHash,
+ (void *) &proclocktag,
+ proclock_hashcode,
+ HASH_FIND,
+ NULL);
+ if (!proclock)
+ elog(ERROR, "failed to re-find shared proclock object");
+ LWLockRelease(partitionLock);
}
-}
+ return proclock;
+}
/*
* GetLockConflicts
uint32 hashcode;
LWLockId partitionLock;
int count = 0;
+ int fast_count = 0;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
vxids = (VirtualTransactionId *)
palloc0(sizeof(VirtualTransactionId) * (MaxBackends + 1));
- /*
- * Look up the lock object matching the tag.
- */
+ /* Compute hash code and partiton lock, and look up conflicting modes. */
hashcode = LockTagHashCode(locktag);
partitionLock = LockHashPartitionLock(hashcode);
+ conflictMask = lockMethodTable->conflictTab[lockmode];
+
+ /*
+ * Fast path locks might not have been entered in the primary lock table.
+ * But only strong locks can conflict with anything that might have been
+ * taken via the fast-path mechanism.
+ */
+ if (FastPathTag(locktag) && FastPathStrongMode(lockmode))
+ {
+ int i;
+ Oid relid = locktag->locktag_field2;
+ VirtualTransactionId vxid;
+
+ /*
+ * Iterate over relevant PGPROCs. Anything held by a prepared
+ * transaction will have been transferred to the primary lock table,
+ * so we need not worry about those. This is all a bit fuzzy,
+ * because new locks could be taken after we've visited a particular
+ * partition, but the callers had better be prepared to deal with
+ * that anyway, since the locks could equally well be taken between the
+ * time we return the value and the time the caller does something
+ * with it.
+ */
+ for (i = 0; i < ProcGlobal->allProcCount; i++)
+ {
+ PGPROC *proc = &ProcGlobal->allProcs[i];
+ uint32 f;
+
+ /* A backend never blocks itself */
+ if (proc == MyProc)
+ continue;
+
+ LWLockAcquire(proc->backendLock, LW_SHARED);
+
+ /*
+ * If the target backend isn't referencing the same database as we
+ * are, then we needn't examine the individual relation IDs at all;
+ * none of them can be relevant.
+ *
+ * See FastPathTransferLocks() for discussion of why we do this
+ * test after acquiring the lock.
+ */
+ if (proc->databaseId != MyDatabaseId)
+ {
+ LWLockRelease(proc->backendLock);
+ continue;
+ }
+
+ for (f = 0; f < FP_LOCK_SLOTS_PER_BACKEND; f++)
+ {
+ uint32 lockmask;
+
+ /* Look for an allocated slot matching the given relid. */
+ if (relid != proc->fpRelId[f])
+ continue;
+ lockmask = FAST_PATH_GET_BITS(proc, f);
+ if (!lockmask)
+ continue;
+ lockmask <<= FAST_PATH_LOCKNUMBER_OFFSET;
+
+ /*
+ * There can only be one entry per relation, so if we found
+ * it and it doesn't conflict, we can skip the rest of the
+ * slots.
+ */
+ if ((lockmask & conflictMask) == 0)
+ break;
+
+ /* Conflict! */
+ GET_VXID_FROM_PGPROC(vxid, *proc);
+
+ /*
+ * If we see an invalid VXID, then either the xact has already
+ * committed (or aborted), or it's a prepared xact. In either
+ * case we may ignore it.
+ */
+ if (VirtualTransactionIdIsValid(vxid))
+ vxids[count++] = vxid;
+ break;
+ }
+ LWLockRelease(proc->backendLock);
+ }
+ }
+
+ /* Remember how many fast-path conflicts we found. */
+ fast_count = count;
+
+ /*
+ * Look up the lock object matching the tag.
+ */
LWLockAcquire(partitionLock, LW_SHARED);
lock = (LOCK *) hash_search_with_hash_value(LockMethodLockHash,
/*
* Examine each existing holder (or awaiter) of the lock.
*/
- conflictMask = lockMethodTable->conflictTab[lockmode];
procLocks = &(lock->procLocks);
* case we may ignore it.
*/
if (VirtualTransactionIdIsValid(vxid))
- vxids[count++] = vxid;
+ {
+ int i;
+
+ /* Avoid duplicate entries. */
+ for (i = 0; i < fast_count; ++i)
+ if (VirtualTransactionIdEquals(vxids[i], vxid))
+ break;
+ if (i >= fast_count)
+ vxids[count++] = vxid;
+ }
}
}
return vxids;
}
+/*
+ * Find a lock in the shared lock table and release it. It is the caller's
+ * responsibility to verify that this is a sane thing to do. (For example, it
+ * would be bad to release a lock here if there might still be a LOCALLOCK
+ * object with pointers to it.)
+ *
+ * We currently use this in two situations: first, to release locks held by
+ * prepared transactions on commit (see lock_twophase_postcommit); and second,
+ * to release locks taken via the fast-path, transferred to the main hash
+ * table, and then released (see LockReleaseAll).
+ */
+static void
+LockRefindAndRelease(LockMethod lockMethodTable, PGPROC *proc,
+ LOCKTAG *locktag, LOCKMODE lockmode,
+ bool decrement_strong_lock_count)
+{
+ LOCK *lock;
+ PROCLOCK *proclock;
+ PROCLOCKTAG proclocktag;
+ uint32 hashcode;
+ uint32 proclock_hashcode;
+ LWLockId partitionLock;
+ bool wakeupNeeded;
+
+ hashcode = LockTagHashCode(locktag);
+ partitionLock = LockHashPartitionLock(hashcode);
+
+ LWLockAcquire(partitionLock, LW_EXCLUSIVE);
+
+ /*
+ * Re-find the lock object (it had better be there).
+ */
+ lock = (LOCK *) hash_search_with_hash_value(LockMethodLockHash,
+ (void *) locktag,
+ hashcode,
+ HASH_FIND,
+ NULL);
+ if (!lock)
+ elog(PANIC, "failed to re-find shared lock object");
+
+ /*
+ * Re-find the proclock object (ditto).
+ */
+ proclocktag.myLock = lock;
+ proclocktag.myProc = proc;
+
+ proclock_hashcode = ProcLockHashCode(&proclocktag, hashcode);
+
+ proclock = (PROCLOCK *) hash_search_with_hash_value(LockMethodProcLockHash,
+ (void *) &proclocktag,
+ proclock_hashcode,
+ HASH_FIND,
+ NULL);
+ if (!proclock)
+ elog(PANIC, "failed to re-find shared proclock object");
+
+ /*
+ * Double-check that we are actually holding a lock of the type we want to
+ * release.
+ */
+ if (!(proclock->holdMask & LOCKBIT_ON(lockmode)))
+ {
+ PROCLOCK_PRINT("lock_twophase_postcommit: WRONGTYPE", proclock);
+ LWLockRelease(partitionLock);
+ elog(WARNING, "you don't own a lock of type %s",
+ lockMethodTable->lockModeNames[lockmode]);
+ return;
+ }
+
+ /*
+ * Do the releasing. CleanUpLock will waken any now-wakable waiters.
+ */
+ wakeupNeeded = UnGrantLock(lock, lockmode, proclock, lockMethodTable);
+
+ CleanUpLock(lock, proclock,
+ lockMethodTable, hashcode,
+ wakeupNeeded);
+
+ LWLockRelease(partitionLock);
+
+ /*
+ * Decrement strong lock count. This logic is needed only for 2PC.
+ */
+ if (decrement_strong_lock_count
+ && FastPathTag(&lock->tag) && FastPathStrongMode(lockmode))
+ {
+ uint32 fasthashcode = FastPathStrongLockHashPartition(hashcode);
+ SpinLockAcquire(&FastPathStrongLocks->mutex);
+ FastPathStrongLocks->count[fasthashcode]--;
+ SpinLockRelease(&FastPathStrongLocks->mutex);
+ }
+}
/*
* AtPrepare_Locks
LOCALLOCK *locallock;
/*
- * We don't need to touch shared memory for this --- all the necessary
- * state information is in the locallock table.
+ * For the most part, we don't need to touch shared memory for this ---
+ * all the necessary state information is in the locallock table.
+ * Fast-path locks are an exception, however: we move any such locks
+ * to the main table before allowing PREPARE TRANSACTION to succeed.
*/
hash_seq_init(&status, LockMethodLocalHash);
elog(ERROR, "cannot PREPARE when session locks exist");
}
+ /*
+ * If the local lock was taken via the fast-path, we need to move it
+ * to the primary lock table, or just get a pointer to the existing
+ * primary lock table if by chance it's already been transferred.
+ */
+ if (locallock->proclock == NULL)
+ {
+ locallock->proclock = FastPathGetLockEntry(locallock);
+ locallock->lock = locallock->proclock->tag.myLock;
+ }
+
+ /*
+ * Arrange not to release any strong lock count held by this lock
+ * entry. We must retain the count until the prepared transaction
+ * is committed or rolled back.
+ */
+ locallock->holdsStrongLockCount = FALSE;
+
/*
* Create a 2PC record.
*/
data = (LockData *) palloc(sizeof(LockData));
+ /* Guess how much space we'll need. */
+ els = MaxBackends;
+ el = 0;
+ data->locks = (LockInstanceData *) palloc(sizeof(LockInstanceData) * els);
+
+ /*
+ * First, we iterate through the per-backend fast-path arrays, locking
+ * them one at a time. This might produce an inconsistent picture of the
+ * system state, but taking all of those LWLocks at the same time seems
+ * impractical (in particular, note MAX_SIMUL_LWLOCKS). It shouldn't
+ * matter too much, because none of these locks can be involved in lock
+ * conflicts anyway - anything that might must be present in the main
+ * lock table.
+ */
+ for (i = 0; i < ProcGlobal->allProcCount; ++i)
+ {
+ PGPROC *proc = &ProcGlobal->allProcs[i];
+ uint32 f;
+
+ LWLockAcquire(proc->backendLock, LW_SHARED);
+
+ for (f = 0; f < FP_LOCK_SLOTS_PER_BACKEND; ++f)
+ {
+ LockInstanceData *instance = &data->locks[el];
+ uint32 lockbits = FAST_PATH_GET_BITS(proc, f);
+
+ /* Skip unallocated slots. */
+ if (!lockbits)
+ continue;
+
+ if (el >= els)
+ {
+ els += MaxBackends;
+ data->locks = (LockInstanceData *)
+ repalloc(data->locks, sizeof(LockInstanceData) * els);
+ }
+
+ SET_LOCKTAG_RELATION(instance->locktag, proc->databaseId,
+ proc->fpRelId[f]);
+ instance->holdMask = lockbits << FAST_PATH_LOCKNUMBER_OFFSET;
+ instance->waitLockMode = NoLock;
+ instance->backend = proc->backendId;
+ instance->lxid = proc->lxid;
+ instance->pid = proc->pid;
+ instance->fastpath = true;
+
+ el++;
+ }
+
+ LWLockRelease(proc->backendLock);
+ }
+
/*
- * Acquire lock on the entire shared lock data structure. We can't
- * operate one partition at a time if we want to deliver a self-consistent
- * view of the state.
+ * Next, acquire lock on the entire shared lock data structure. We do
+ * this so that, at least for locks in the primary lock table, the state
+ * will be self-consistent.
*
* Since this is a read-only operation, we take shared instead of
* exclusive lock. There's not a whole lot of point to this, because all
LWLockAcquire(FirstLockMgrLock + i, LW_SHARED);
/* Now we can safely count the number of proclocks */
- els = hash_get_num_entries(LockMethodProcLockHash);
-
- data->nelements = els;
- data->proclocks = (PROCLOCK *) palloc(sizeof(PROCLOCK) * els);
- data->procs = (PGPROC *) palloc(sizeof(PGPROC) * els);
- data->locks = (LOCK *) palloc(sizeof(LOCK) * els);
+ data->nelements = el + hash_get_num_entries(LockMethodProcLockHash);
+ if (data->nelements > els)
+ {
+ els = data->nelements;
+ data->locks = (LockInstanceData *)
+ repalloc(data->locks, sizeof(LockInstanceData) * els);
+ }
/* Now scan the tables to copy the data */
hash_seq_init(&seqstat, LockMethodProcLockHash);
- el = 0;
while ((proclock = (PROCLOCK *) hash_seq_search(&seqstat)))
{
PGPROC *proc = proclock->tag.myProc;
LOCK *lock = proclock->tag.myLock;
+ LockInstanceData *instance = &data->locks[el];
- memcpy(&(data->proclocks[el]), proclock, sizeof(PROCLOCK));
- memcpy(&(data->procs[el]), proc, sizeof(PGPROC));
- memcpy(&(data->locks[el]), lock, sizeof(LOCK));
+ memcpy(&instance->locktag, &lock->tag, sizeof(LOCKTAG));
+ instance->holdMask = proclock->holdMask;
+ if (proc->waitLock == proclock->tag.myLock)
+ instance->waitLockMode = proc->waitLockMode;
+ else
+ instance->waitLockMode = NoLock;
+ instance->backend = proc->backendId;
+ instance->lxid = proc->lxid;
+ instance->pid = proc->pid;
+ instance->fastpath = false;
el++;
}
*/
GrantLock(lock, proclock, lockmode);
+ /*
+ * Bump strong lock count, to make sure any fast-path lock requests won't
+ * be granted without consulting the primary lock table.
+ */
+ if (FastPathTag(&lock->tag) && FastPathStrongMode(lockmode))
+ {
+ uint32 fasthashcode = FastPathStrongLockHashPartition(hashcode);
+ SpinLockAcquire(&FastPathStrongLocks->mutex);
+ FastPathStrongLocks->count[fasthashcode]++;
+ SpinLockRelease(&FastPathStrongLocks->mutex);
+ }
+
LWLockRelease(partitionLock);
}
TwoPhaseLockRecord *rec = (TwoPhaseLockRecord *) recdata;
PGPROC *proc = TwoPhaseGetDummyProc(xid);
LOCKTAG *locktag;
- LOCKMODE lockmode;
LOCKMETHODID lockmethodid;
- LOCK *lock;
- PROCLOCK *proclock;
- PROCLOCKTAG proclocktag;
- uint32 hashcode;
- uint32 proclock_hashcode;
- LWLockId partitionLock;
LockMethod lockMethodTable;
- bool wakeupNeeded;
Assert(len == sizeof(TwoPhaseLockRecord));
locktag = &rec->locktag;
- lockmode = rec->lockmode;
lockmethodid = locktag->locktag_lockmethodid;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
- hashcode = LockTagHashCode(locktag);
- partitionLock = LockHashPartitionLock(hashcode);
-
- LWLockAcquire(partitionLock, LW_EXCLUSIVE);
-
- /*
- * Re-find the lock object (it had better be there).
- */
- lock = (LOCK *) hash_search_with_hash_value(LockMethodLockHash,
- (void *) locktag,
- hashcode,
- HASH_FIND,
- NULL);
- if (!lock)
- elog(PANIC, "failed to re-find shared lock object");
-
- /*
- * Re-find the proclock object (ditto).
- */
- proclocktag.myLock = lock;
- proclocktag.myProc = proc;
-
- proclock_hashcode = ProcLockHashCode(&proclocktag, hashcode);
-
- proclock = (PROCLOCK *) hash_search_with_hash_value(LockMethodProcLockHash,
- (void *) &proclocktag,
- proclock_hashcode,
- HASH_FIND,
- NULL);
- if (!proclock)
- elog(PANIC, "failed to re-find shared proclock object");
-
- /*
- * Double-check that we are actually holding a lock of the type we want to
- * release.
- */
- if (!(proclock->holdMask & LOCKBIT_ON(lockmode)))
- {
- PROCLOCK_PRINT("lock_twophase_postcommit: WRONGTYPE", proclock);
- LWLockRelease(partitionLock);
- elog(WARNING, "you don't own a lock of type %s",
- lockMethodTable->lockModeNames[lockmode]);
- return;
- }
-
- /*
- * Do the releasing. CleanUpLock will waken any now-wakable waiters.
- */
- wakeupNeeded = UnGrantLock(lock, lockmode, proclock, lockMethodTable);
-
- CleanUpLock(lock, proclock,
- lockMethodTable, hashcode,
- wakeupNeeded);
-
- LWLockRelease(partitionLock);
+ LockRefindAndRelease(lockMethodTable, proc, locktag, rec->lockmode, true);
}
/*