]> granicus.if.org Git - postgresql/commitdiff
Create VXID locks "lazily" in the main lock table.
authorRobert Haas <rhaas@postgresql.org>
Thu, 4 Aug 2011 16:38:33 +0000 (12:38 -0400)
committerRobert Haas <rhaas@postgresql.org>
Thu, 4 Aug 2011 16:38:33 +0000 (12:38 -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.

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 a1c299d00ee4f6485c79c9831fe864a577672f20..77648b47743ee1983ba94545ee206b8343c301f7 100644 (file)
@@ -683,7 +683,7 @@ DefineIndex(RangeVar *heapRelation,
 
        while (VirtualTransactionIdIsValid(*old_lockholders))
        {
-               VirtualXactLockTableWait(*old_lockholders);
+               VirtualXactLock(*old_lockholders, true);
                old_lockholders++;
        }
 
@@ -769,7 +769,7 @@ DefineIndex(RangeVar *heapRelation,
 
        while (VirtualTransactionIdIsValid(*old_lockholders))
        {
-               VirtualXactLockTableWait(*old_lockholders);
+               VirtualXactLock(*old_lockholders, true);
                old_lockholders++;
        }
 
@@ -866,7 +866,7 @@ DefineIndex(RangeVar *heapRelation,
                }
 
                if (VirtualTransactionIdIsValid(old_snapshots[i]))
-                       VirtualXactLockTableWait(old_snapshots[i]);
+                       VirtualXactLock(old_snapshots[i], true);
        }
 
        /*
index 2f3fcbf04098483a167389eec1831692f305a420..f96c3505465eed5d5d9e6702b57bc72114a1ca32 100644 (file)
@@ -1992,7 +1992,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 47f1a6156091fbd0e6af9f65f5e45eced901fd6e..a1d8e26dd9fcb7c74fd5c282e00cf3255f687a76 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 */
@@ -246,6 +247,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;
@@ -353,6 +356,7 @@ CleanupInvalidationState(int status, Datum arg)
 
        /* Mark myself inactive */
        stateP->procPid = 0;
+       stateP->proc = NULL;
        stateP->nextMsgNum = 0;
        stateP->resetState = false;
        stateP->signaled = false;
@@ -369,13 +373,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 */
@@ -385,10 +392,8 @@ BackendIdIsActive(int backendID)
        {
                ProcState  *stateP = &segP->procState[backendID - 1];
 
-               result = (stateP->procPid != 0);
+               result = stateP->proc;
        }
-       else
-               result = false;
 
        LWLockRelease(SInvalWriteLock);
 
index bf92d259950c38868d3cd4898c0943865e6fadb6..5673c27cbe000b8fe29be9fbc7bb2dd2ca253344 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 436ba472e8a8a7ed5abb642edb450004b7418f64..e3bb116344270a72aabf15dffadec5c816a118c8 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 3ac098b2a9c3cb77b518e268c16998f375f175f3..25ead29a4802febf565562c9cc64037c22ac3640 100644 (file)
@@ -515,70 +515,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 416f5aae994145a6ee0512cf3e82ab022ec28db7..05efafc0b06c534ed1dc8d0c8761ca1e7dfe438b 100644 (file)
@@ -38,6 +38,7 @@
 #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"
@@ -160,6 +161,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
@@ -1772,6 +1774,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;
 
        /*
@@ -3034,6 +3045,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);
        }
 
@@ -3529,3 +3567,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 21b77f305992f19eef8b03a8138bf3e555487e01..e106ad54019bb915edf84cfca014cae0eebdcaa6 100644 (file)
@@ -543,4 +543,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 92b3b19f2d6af627b081f77d2b2fae6ac775ff5b..a149a55855c7d5a900e5d58d22f05f37bab7d1c5 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);