while (VirtualTransactionIdIsValid(*old_lockholders))
{
- VirtualXactLockTableWait(*old_lockholders);
+ VirtualXactLock(*old_lockholders, true);
old_lockholders++;
}
while (VirtualTransactionIdIsValid(*old_lockholders))
{
- VirtualXactLockTableWait(*old_lockholders);
+ VirtualXactLock(*old_lockholders, true);
old_lockholders++;
}
}
if (VirtualTransactionIdIsValid(old_snapshots[i]))
- VirtualXactLockTableWait(old_snapshots[i]);
+ VirtualXactLock(old_snapshots[i], true);
}
/*
backendID = GetTempNamespaceBackendId(classForm->relnamespace);
/* We just ignore it if the owning backend is still active */
- if (backendID == MyBackendId || !BackendIdIsActive(backendID))
+ if (backendID == MyBackendId || BackendIdGetProc(backendID) == NULL)
{
/*
* We found an orphan temp table (which was probably left
{
/* procPid is zero in an inactive ProcState array entry. */
pid_t procPid; /* PID of backend, for signaling */
+ PGPROC *proc; /* PGPROC of backend */
/* nextMsgNum is meaningless if procPid == 0 or resetState is true. */
int nextMsgNum; /* next message number to read */
bool resetState; /* backend needs to reset its state */
for (i = 0; i < shmInvalBuffer->maxBackends; i++)
{
shmInvalBuffer->procState[i].procPid = 0; /* inactive */
+ shmInvalBuffer->procState[i].proc = NULL;
shmInvalBuffer->procState[i].nextMsgNum = 0; /* meaningless */
shmInvalBuffer->procState[i].resetState = false;
shmInvalBuffer->procState[i].signaled = false;
/* mark myself active, with all extant messages already read */
stateP->procPid = MyProcPid;
+ stateP->proc = MyProc;
stateP->nextMsgNum = segP->maxMsgNum;
stateP->resetState = false;
stateP->signaled = false;
/* Mark myself inactive */
stateP->procPid = 0;
+ stateP->proc = NULL;
stateP->nextMsgNum = 0;
stateP->resetState = false;
stateP->signaled = false;
}
/*
- * BackendIdIsActive
- * Test if the given backend ID is currently assigned to a process.
+ * BackendIdGetProc
+ * Get the PGPROC structure for a backend, given the backend ID.
+ * The result may be out of date arbitrarily quickly, so the caller
+ * must be careful about how this information is used. NULL is
+ * returned if the backend is not active.
*/
-bool
-BackendIdIsActive(int backendID)
+PGPROC *
+BackendIdGetProc(int backendID)
{
- bool result;
+ PGPROC *result = NULL;
SISeg *segP = shmInvalBuffer;
/* Need to lock out additions/removals of backends */
{
ProcState *stateP = &segP->procState[backendID - 1];
- result = (stateP->procPid != 0);
+ result = stateP->proc;
}
- else
- result = false;
LWLockRelease(SInvalWriteLock);
standbyWait_us = STANDBY_INITIAL_WAIT_US;
/* wait until the virtual xid is gone */
- while (!ConditionalVirtualXactLockTableWait(*waitlist))
+ while (!VirtualXactLock(*waitlist, false))
{
/*
* Report via ps if we have been waiting for more than 500 msec
-----------------
Fast path locking is a special purpose mechanism designed to reduce the
-overhead of taking and releasing weak relation locks. SELECT, INSERT,
-UPDATE, and DELETE must acquire a lock on every relation they operate on,
-as well as various system catalogs that can be used internally. These locks
-are notable not only for the very high frequency with which they are taken
-and released, but also for the fact that they virtually never conflict.
-Many DML operations can proceed in parallel against the same table at the
-same time; only DDL operations such as CLUSTER, ALTER TABLE, or DROP -- or
-explicit user action such as LOCK TABLE -- will create lock conflicts with
-the "weak" locks (AccessShareLock, RowShareLock, RowExclusiveLock) acquired
-by DML operations.
+overhead of taking and releasing certain types of locks which are taken
+and released very frequently but rarely conflict. Currently, this includes
+two categories of locks:
+
+(1) Weak relation locks. SELECT, INSERT, UPDATE, and DELETE must acquire a
+lock on every relation they operate on, as well as various system catalogs
+that can be used internally. Many DML operations can proceed in parallel
+against the same table at the same time; only DDL operations such as
+CLUSTER, ALTER TABLE, or DROP -- or explicit user action such as LOCK TABLE
+-- will create lock conflicts with the "weak" locks (AccessShareLock,
+RowShareLock, RowExclusiveLock) acquired by DML operations.
+
+(2) VXID locks. Every transaction takes a lock on its own virtual
+transaction ID. Currently, the only operations that wait for these locks
+are CREATE INDEX CONCURRENTLY and Hot Standby (in the case of a conflict),
+so most VXID locks are taken and released by the owner without anyone else
+needing to care.
The primary locking mechanism does not cope well with this workload. Even
though the lock manager locks are partitioned, the locktag for any given
To alleviate this bottleneck, beginning in PostgreSQL 9.2, each backend is
permitted to record a limited number of locks on unshared relations in an
array within its PGPROC structure, rather than using the primary lock table.
-This is called the "fast path" mechanism, and can only be used when the
-locker can verify that no conflicting locks can possibly exist.
+This mechanism can only be used when the locker can verify that no conflicting
+locks can possibly exist.
A key point of this algorithm is that it must be possible to verify the
absence of possibly conflicting locks without fighting over a shared LWLock or
indeed, even the first per-backend LWLock) and will notice any weak lock we
take when it does.
+Fast-path VXID locks do not use the FastPathStrongLocks table. The first
+lock taken on a VXID is always the ExclusiveLock taken by its owner. Any
+subsequent lockers are share lockers waiting for the VXID to terminate.
+Indeed, the only reason VXID locks use the lock manager at all (rather than
+waiting for the VXID to terminate via some other method) is for deadlock
+detection. Thus, the initial VXID lock can *always* be taken via the fast
+path without checking for conflicts. Any subsequent locker must check
+whether the lock has been transferred to the main lock table, and if not,
+do so. The backend owning the VXID must be careful to clean up any entry
+made in the main lock table at end of transaction.
+
The Deadlock Detection Algorithm
--------------------------------
return true;
}
-
-/*
- * VirtualXactLockTableInsert
- *
- * Insert a lock showing that the given virtual transaction ID is running ---
- * this is done at main transaction start when its VXID is assigned.
- * The lock can then be used to wait for the transaction to finish.
- */
-void
-VirtualXactLockTableInsert(VirtualTransactionId vxid)
-{
- LOCKTAG tag;
-
- Assert(VirtualTransactionIdIsValid(vxid));
-
- SET_LOCKTAG_VIRTUALTRANSACTION(tag, vxid);
-
- (void) LockAcquire(&tag, ExclusiveLock, false, false);
-}
-
-/*
- * VirtualXactLockTableWait
- *
- * Waits until the lock on the given VXID is released, which shows that
- * the top-level transaction owning the VXID has ended.
- */
-void
-VirtualXactLockTableWait(VirtualTransactionId vxid)
-{
- LOCKTAG tag;
-
- Assert(VirtualTransactionIdIsValid(vxid));
-
- SET_LOCKTAG_VIRTUALTRANSACTION(tag, vxid);
-
- (void) LockAcquire(&tag, ShareLock, false, false);
-
- LockRelease(&tag, ShareLock, false);
-}
-
-/*
- * ConditionalVirtualXactLockTableWait
- *
- * As above, but only lock if we can get the lock without blocking.
- * Returns TRUE if the lock was acquired.
- */
-bool
-ConditionalVirtualXactLockTableWait(VirtualTransactionId vxid)
-{
- LOCKTAG tag;
-
- Assert(VirtualTransactionIdIsValid(vxid));
-
- SET_LOCKTAG_VIRTUALTRANSACTION(tag, vxid);
-
- if (LockAcquire(&tag, ShareLock, false, true) == LOCKACQUIRE_NOT_AVAIL)
- return false;
-
- LockRelease(&tag, ShareLock, false);
-
- return true;
-}
-
-
/*
* LockDatabaseObject
*
#include "miscadmin.h"
#include "pg_trace.h"
#include "pgstat.h"
+#include "storage/sinvaladt.h"
#include "storage/standby.h"
#include "utils/memutils.h"
#include "utils/ps_status.h"
static bool FastPathTransferRelationLocks(LockMethod lockMethodTable,
const LOCKTAG *locktag, uint32 hashcode);
static PROCLOCK *FastPathGetRelationLockEntry(LOCALLOCK *locallock);
+static void VirtualXactLockTableCleanup(void);
/*
* To make the fast-path lock mechanism work, we must have some way of
elog(LOG, "LockReleaseAll: lockmethod=%d", lockmethodid);
#endif
+ /*
+ * Get rid of our fast-path VXID lock, if appropriate. Note that this
+ * is the only way that the lock we hold on our own VXID can ever get
+ * released: it is always and only released when a toplevel transaction
+ * ends.
+ */
+ if (lockmethodid == DEFAULT_LOCKMETHOD)
+ VirtualXactLockTableCleanup();
+
numLockModes = lockMethodTable->numLockModes;
/*
el++;
}
+ if (proc->fpVXIDLock)
+ {
+ VirtualTransactionId vxid;
+ LockInstanceData *instance;
+
+ if (el >= els)
+ {
+ els += MaxBackends;
+ data->locks = (LockInstanceData *)
+ repalloc(data->locks, sizeof(LockInstanceData) * els);
+ }
+
+ vxid.backendId = proc->backendId;
+ vxid.localTransactionId = proc->fpLocalTransactionId;
+
+ instance = &data->locks[el];
+ SET_LOCKTAG_VIRTUALTRANSACTION(instance->locktag, vxid);
+ instance->holdMask = LOCKBIT_ON(ExclusiveLock);
+ instance->waitLockMode = NoLock;
+ instance->backend = proc->backendId;
+ instance->lxid = proc->lxid;
+ instance->pid = proc->pid;
+ instance->fastpath = true;
+
+ el++;
+ }
+
LWLockRelease(proc->backendLock);
}
{
lock_twophase_postcommit(xid, info, recdata, len);
}
+
+/*
+ * VirtualXactLockTableInsert
+ *
+ * Take vxid lock via the fast-path. There can't be any pre-existing
+ * lockers, as we haven't advertised this vxid via the ProcArray yet.
+ *
+ * Since MyProc->fpLocalTransactionId will normally contain the same data
+ * as MyProc->lxid, you might wonder if we really need both. The
+ * difference is that MyProc->lxid is set and cleared unlocked, and
+ * examined by procarray.c, while fpLocalTransactionId is protected by
+ * backendLock and is used only by the locking subsystem. Doing it this
+ * way makes it easier to verify that there are no funny race conditions.
+ *
+ * We don't bother recording this lock in the local lock table, since it's
+ * only ever released at the end of a transaction. Instead,
+ * LockReleaseAll() calls VirtualXactLockTableCleanup().
+ */
+void
+VirtualXactLockTableInsert(VirtualTransactionId vxid)
+{
+ Assert(VirtualTransactionIdIsValid(vxid));
+
+ LWLockAcquire(MyProc->backendLock, LW_EXCLUSIVE);
+
+ Assert(MyProc->backendId == vxid.backendId);
+ Assert(MyProc->fpLocalTransactionId == InvalidLocalTransactionId);
+ Assert(MyProc->fpVXIDLock == false);
+
+ MyProc->fpVXIDLock = true;
+ MyProc->fpLocalTransactionId = vxid.localTransactionId;
+
+ LWLockRelease(MyProc->backendLock);
+}
+
+/*
+ * VirtualXactLockTableCleanup
+ *
+ * Check whether a VXID lock has been materialized; if so, release it,
+ * unblocking waiters.
+ */
+static void
+VirtualXactLockTableCleanup()
+{
+ bool fastpath;
+ LocalTransactionId lxid;
+
+ Assert(MyProc->backendId != InvalidBackendId);
+
+ /*
+ * Clean up shared memory state.
+ */
+ LWLockAcquire(MyProc->backendLock, LW_EXCLUSIVE);
+
+ fastpath = MyProc->fpVXIDLock;
+ lxid = MyProc->fpLocalTransactionId;
+ MyProc->fpVXIDLock = false;
+ MyProc->fpLocalTransactionId = InvalidLocalTransactionId;
+
+ LWLockRelease(MyProc->backendLock);
+
+ /*
+ * If fpVXIDLock has been cleared without touching fpLocalTransactionId,
+ * that means someone transferred the lock to the main lock table.
+ */
+ if (!fastpath && LocalTransactionIdIsValid(lxid))
+ {
+ VirtualTransactionId vxid;
+ LOCKTAG locktag;
+
+ vxid.backendId = MyBackendId;
+ vxid.localTransactionId = lxid;
+ SET_LOCKTAG_VIRTUALTRANSACTION(locktag, vxid);
+
+ LockRefindAndRelease(LockMethods[DEFAULT_LOCKMETHOD], MyProc,
+ &locktag, ExclusiveLock, false);
+ }
+}
+
+/*
+ * VirtualXactLock
+ *
+ * If wait = true, wait until the given VXID has been released, and then
+ * return true.
+ *
+ * If wait = false, just check whether the VXID is still running, and return
+ * true or false.
+ */
+bool
+VirtualXactLock(VirtualTransactionId vxid, bool wait)
+{
+ LOCKTAG tag;
+ PGPROC *proc;
+
+ Assert(VirtualTransactionIdIsValid(vxid));
+
+ SET_LOCKTAG_VIRTUALTRANSACTION(tag, vxid);
+
+ /*
+ * If a lock table entry must be made, this is the PGPROC on whose behalf
+ * it must be done. Note that the transaction might end or the PGPROC
+ * might be reassigned to a new backend before we get around to examining
+ * it, but it doesn't matter. If we find upon examination that the
+ * relevant lxid is no longer running here, that's enough to prove that
+ * it's no longer running anywhere.
+ */
+ proc = BackendIdGetProc(vxid.backendId);
+
+ /*
+ * We must acquire this lock before checking the backendId and lxid
+ * against the ones we're waiting for. The target backend will only
+ * set or clear lxid while holding this lock.
+ */
+ LWLockAcquire(proc->backendLock, LW_EXCLUSIVE);
+
+ /* If the transaction has ended, our work here is done. */
+ if (proc->backendId != vxid.backendId
+ || proc->fpLocalTransactionId != vxid.localTransactionId)
+ {
+ LWLockRelease(proc->backendLock);
+ return true;
+ }
+
+ /*
+ * If we aren't asked to wait, there's no need to set up a lock table
+ * entry. The transaction is still in progress, so just return false.
+ */
+ if (!wait)
+ {
+ LWLockRelease(proc->backendLock);
+ return false;
+ }
+
+ /*
+ * OK, we're going to need to sleep on the VXID. But first, we must set
+ * up the primary lock table entry, if needed.
+ */
+ if (proc->fpVXIDLock)
+ {
+ PROCLOCK *proclock;
+ uint32 hashcode;
+
+ hashcode = LockTagHashCode(&tag);
+ proclock = SetupLockInTable(LockMethods[DEFAULT_LOCKMETHOD], proc,
+ &tag, hashcode, ExclusiveLock);
+ if (!proclock)
+ 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, ExclusiveLock);
+ proc->fpVXIDLock = false;
+ }
+
+ /* Done with proc->fpLockBits */
+ LWLockRelease(proc->backendLock);
+
+ /* Time to wait. */
+ (void) LockAcquire(&tag, ShareLock, false, false);
+
+ LockRelease(&tag, ShareLock, false);
+ return true;
+}
extern void XactLockTableWait(TransactionId xid);
extern bool ConditionalXactLockTableWait(TransactionId xid);
-/* Lock a VXID (used to wait for a transaction to finish) */
-extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
-extern void VirtualXactLockTableWait(VirtualTransactionId vxid);
-extern bool ConditionalVirtualXactLockTableWait(VirtualTransactionId vxid);
-
/* Lock a general object (other than a relation) of the current database */
extern void LockDatabaseObject(Oid classid, Oid objid, uint16 objsubid,
LOCKMODE lockmode);
extern void DumpAllLocks(void);
#endif
+/* Lock a VXID (used to wait for a transaction to finish) */
+extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
+extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+
#endif /* LOCK_H */
/* Lock manager data, recording fast-path locks taken by this backend. */
uint64 fpLockBits; /* lock modes held for each fast-path slot */
Oid fpRelId[FP_LOCK_SLOTS_PER_BACKEND]; /* slots for rel oids */
+ bool fpVXIDLock; /* are we holding a fast-path VXID lock? */
+ LocalTransactionId fpLocalTransactionId; /* lxid for fast-path VXID lock */
};
/* NOTE: "typedef struct PGPROC PGPROC" appears in storage/lock.h. */
#ifndef SINVALADT_H
#define SINVALADT_H
+#include "storage/proc.h"
#include "storage/sinval.h"
/*
extern Size SInvalShmemSize(void);
extern void CreateSharedInvalidationState(void);
extern void SharedInvalBackendInit(bool sendOnly);
-extern bool BackendIdIsActive(int backendID);
+extern PGPROC *BackendIdGetProc(int backendID);
extern void SIInsertDataEntries(const SharedInvalidationMessage *data, int n);
extern int SIGetDataEntries(SharedInvalidationMessage *data, int datasize);