Create VXID locks "lazily" in the main lock table.
authorRobert Haas <rhaas@postgresql.org>
Tue, 4 Oct 2011 16:37:26 +0000 (12:37 -0400)
committerRobert Haas <rhaas@postgresql.org>
Fri, 1 Jun 2012 12:29:52 +0000 (08:29 -0400)
Instead of entering them on transaction startup, we materialize them
only when someone wants to wait, which will occur only during CREATE
INDEX CONCURRENTLY.  In Hot Standby mode, the startup process must also
be able to probe for conflicting VXID locks, but the lock need never be
fully materialized, because the startup process does not use the normal
lock wait mechanism.  Since most VXID locks never need to touch the
lock manager partition locks, this can significantly reduce blocking
contention on read-heavy workloads.

Patch by me.  Review by Jeff Davis.

Conflicts:

src/backend/storage/lmgr/lock.c

src/backend/commands/indexcmds.c
src/backend/postmaster/autovacuum.c
src/backend/storage/ipc/sinvaladt.c
src/backend/storage/ipc/standby.c
src/backend/storage/lmgr/README
src/backend/storage/lmgr/lmgr.c
src/backend/storage/lmgr/lock.c
src/include/storage/lmgr.h
src/include/storage/lock.h
src/include/storage/proc.h
src/include/storage/sinvaladt.h

index b7c021d943a99e96ff5aab93caff7aecc83df66f..a5833995654b71377fbc76939b3a52424a6b532a 100644 (file)
@@ -482,7 +482,7 @@ DefineIndex(RangeVar *heapRelation,
 
        while (VirtualTransactionIdIsValid(*old_lockholders))
        {
-               VirtualXactLockTableWait(*old_lockholders);
+               VirtualXactLock(*old_lockholders, true);
                old_lockholders++;
        }
 
@@ -568,7 +568,7 @@ DefineIndex(RangeVar *heapRelation,
 
        while (VirtualTransactionIdIsValid(*old_lockholders))
        {
-               VirtualXactLockTableWait(*old_lockholders);
+               VirtualXactLock(*old_lockholders, true);
                old_lockholders++;
        }
 
@@ -665,7 +665,7 @@ DefineIndex(RangeVar *heapRelation,
                }
 
                if (VirtualTransactionIdIsValid(old_snapshots[i]))
-                       VirtualXactLockTableWait(old_snapshots[i]);
+                       VirtualXactLock(old_snapshots[i], true);
        }
 
        /*
index 3d57c453772351f3c9c17b0ea7a1680291f002bd..ac524f20d31a2dea0333f45683206c87d894b671 100644 (file)
@@ -2021,7 +2021,7 @@ do_autovacuum(void)
                        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
index 4f446aab7a42e02e615f956f646806e90fbf1f2b..807b7bee547a8b817ed8848914defebc6505804f 100644 (file)
@@ -139,6 +139,7 @@ typedef struct ProcState
 {
        /* 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 */
@@ -245,6 +246,7 @@ CreateSharedInvalidationState(void)
        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;
@@ -313,6 +315,7 @@ SharedInvalBackendInit(bool sendOnly)
 
        /* 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;
@@ -352,6 +355,7 @@ CleanupInvalidationState(int status, Datum arg)
 
        /* Mark myself inactive */
        stateP->procPid = 0;
+       stateP->proc = NULL;
        stateP->nextMsgNum = 0;
        stateP->resetState = false;
        stateP->signaled = false;
@@ -368,13 +372,16 @@ CleanupInvalidationState(int status, Datum arg)
 }
 
 /*
- * 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 */
@@ -384,10 +391,8 @@ BackendIdIsActive(int backendID)
        {
                ProcState  *stateP = &segP->procState[backendID - 1];
 
-               result = (stateP->procPid != 0);
+               result = stateP->proc;
        }
-       else
-               result = false;
 
        LWLockRelease(SInvalWriteLock);
 
index 1e51914d05160b2cd42fe672dd6be7c3b4c5efb6..85e030f064839d125e8ea96253d880239c091436 100644 (file)
@@ -201,7 +201,7 @@ ResolveRecoveryConflictWithVirtualXIDs(VirtualTransactionId *waitlist,
                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
index ca54e5167836816b3a8084e0200895d05d03329b..3165bb17c75ffb11d7c995bf7187589bcba61393 100644 (file)
@@ -263,16 +263,23 @@ Fast Path Locking
 -----------------
 
 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
@@ -284,8 +291,8 @@ even on 2-core servers, and becomes very pronounced as core count increases.
 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
@@ -317,6 +324,17 @@ the strong locker has yet to acquire the per-backend LWLock we now hold (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
 --------------------------------
index 859b3852dbd112dcddb776bf1e44f64613027e4a..9d0994ec064657fa83634b1b4533529970aa3538 100644 (file)
@@ -514,70 +514,6 @@ ConditionalXactLockTableWait(TransactionId xid)
        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
  *
index 04f7fa49e63c2f15d3402ae84768edf99b59a381..05ea6e2d2536fa05fdca103a665db4b1f31096b7 100644 (file)
@@ -39,6 +39,7 @@
 #include "pg_trace.h"
 #include "pgstat.h"
 #include "storage/spin.h"
+#include "storage/sinvaladt.h"
 #include "storage/standby.h"
 #include "utils/memutils.h"
 #include "utils/ps_status.h"
@@ -161,6 +162,7 @@ static bool FastPathUnGrantRelationLock(Oid relid, LOCKMODE lockmode);
 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
@@ -1748,6 +1750,15 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
                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;
 
        /*
@@ -3081,6 +3092,33 @@ GetLockStatusData(void)
                        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);
        }
 
@@ -3586,3 +3624,166 @@ lock_twophase_postabort(TransactionId xid, uint16 info,
 {
        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;
+}
index bd44d92be3c07db32923f0d4b5901c3477fee9de..340f6a3d30560ed710990f96ec39c2d3e0dac12d 100644 (file)
@@ -56,11 +56,6 @@ extern void XactLockTableDelete(TransactionId xid);
 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);
index f7ad3e80f419ea9e9e13d3284465beff25b6f9c4..17b9ed698e6c872ea7faca87dcca33d8ad017498 100644 (file)
@@ -545,4 +545,8 @@ extern void DumpLocks(PGPROC *proc);
 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 */
index 243da4fc049a73de0c31133fd44ff678479389ac..eba7acec9e482f0a674c3cac60985e48d917e8c4 100644 (file)
@@ -152,6 +152,8 @@ struct PGPROC
        /* 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. */
index c70355847af832bd02722ebf827291b15daa6289..a61d6965084d5467575d9b2983703a2114f7593f 100644 (file)
@@ -22,6 +22,7 @@
 #ifndef SINVALADT_H
 #define SINVALADT_H
 
+#include "storage/proc.h"
 #include "storage/sinval.h"
 
 /*
@@ -30,7 +31,7 @@
 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);