summaryrefslogtreecommitdiff
path: root/src/backend/storage
diff options
context:
space:
mode:
authorTom Lane2001-01-14 05:08:17 +0000
committerTom Lane2001-01-14 05:08:17 +0000
commit36839c192706f5abd75bdcb02b6a7cace14ce108 (patch)
tree3022631b1208e1227684db86c12cbba7da15f611 /src/backend/storage
parent027f144e390afa6f189270e8c2a2a56c0a88f646 (diff)
Restructure backend SIGINT/SIGTERM handling so that 'die' interrupts
are treated more like 'cancel' interrupts: the signal handler sets a flag that is examined at well-defined spots, rather than trying to cope with an interrupt that might happen anywhere. See pghackers discussion of 1/12/01.
Diffstat (limited to 'src/backend/storage')
-rw-r--r--src/backend/storage/buffer/bufmgr.c73
-rw-r--r--src/backend/storage/buffer/s_lock.c9
-rw-r--r--src/backend/storage/ipc/ipc.c44
-rw-r--r--src/backend/storage/ipc/spin.c37
-rw-r--r--src/backend/storage/lmgr/lock.c35
-rw-r--r--src/backend/storage/lmgr/proc.c386
6 files changed, 350 insertions, 234 deletions
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 2be519193bb..6b897588621 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/buffer/bufmgr.c,v 1.103 2001/01/12 21:53:57 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/buffer/bufmgr.c,v 1.104 2001/01/14 05:08:15 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -92,6 +92,7 @@ static Buffer ReadBufferWithBufferLock(Relation relation, BlockNumber blockNum,
bool bufferLockHeld);
static BufferDesc *BufferAlloc(Relation reln, BlockNumber blockNum,
bool *foundPtr, bool bufferLockHeld);
+static int ReleaseBufferWithBufferLock(Buffer buffer);
static int BufferReplace(BufferDesc *bufHdr);
void PrintBufferDescs(void);
@@ -687,10 +688,14 @@ ReleaseAndReadBuffer(Buffer buffer,
{
bufHdr = &BufferDescriptors[buffer - 1];
Assert(PrivateRefCount[buffer - 1] > 0);
- PrivateRefCount[buffer - 1]--;
- if (PrivateRefCount[buffer - 1] == 0)
+ if (PrivateRefCount[buffer - 1] > 1)
+ {
+ PrivateRefCount[buffer - 1]--;
+ }
+ else
{
SpinAcquire(BufMgrLock);
+ PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->refcount--;
if (bufHdr->refcount == 0)
@@ -1185,10 +1190,7 @@ recheck:
/* Assert checks that buffer will actually get freed! */
Assert(PrivateRefCount[i - 1] == 1 &&
bufHdr->refcount == 1);
- /* ReleaseBuffer expects we do not hold the lock at entry */
- SpinRelease(BufMgrLock);
- ReleaseBuffer(i);
- SpinAcquire(BufMgrLock);
+ ReleaseBufferWithBufferLock(i);
}
/*
* And mark the buffer as no longer occupied by this rel.
@@ -1270,10 +1272,7 @@ recheck:
/* Assert checks that buffer will actually get freed! */
Assert(PrivateRefCount[i - 1] == 1 &&
bufHdr->refcount == 1);
- /* ReleaseBuffer expects we do not hold the lock at entry */
- SpinRelease(BufMgrLock);
- ReleaseBuffer(i);
- SpinAcquire(BufMgrLock);
+ ReleaseBufferWithBufferLock(i);
}
/*
* And mark the buffer as no longer occupied by this rel.
@@ -1624,10 +1623,14 @@ ReleaseBuffer(Buffer buffer)
bufHdr = &BufferDescriptors[buffer - 1];
Assert(PrivateRefCount[buffer - 1] > 0);
- PrivateRefCount[buffer - 1]--;
- if (PrivateRefCount[buffer - 1] == 0)
+ if (PrivateRefCount[buffer - 1] > 1)
+ {
+ PrivateRefCount[buffer - 1]--;
+ }
+ else
{
SpinAcquire(BufMgrLock);
+ PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->refcount--;
if (bufHdr->refcount == 0)
@@ -1641,6 +1644,48 @@ ReleaseBuffer(Buffer buffer)
return STATUS_OK;
}
+/*
+ * ReleaseBufferWithBufferLock
+ * Same as ReleaseBuffer except we hold the lock
+ */
+static int
+ReleaseBufferWithBufferLock(Buffer buffer)
+{
+ BufferDesc *bufHdr;
+
+ if (BufferIsLocal(buffer))
+ {
+ Assert(LocalRefCount[-buffer - 1] > 0);
+ LocalRefCount[-buffer - 1]--;
+ return STATUS_OK;
+ }
+
+ if (BAD_BUFFER_ID(buffer))
+ return STATUS_ERROR;
+
+ bufHdr = &BufferDescriptors[buffer - 1];
+
+ Assert(PrivateRefCount[buffer - 1] > 0);
+ if (PrivateRefCount[buffer - 1] > 1)
+ {
+ PrivateRefCount[buffer - 1]--;
+ }
+ else
+ {
+ PrivateRefCount[buffer - 1] = 0;
+ Assert(bufHdr->refcount > 0);
+ bufHdr->refcount--;
+ if (bufHdr->refcount == 0)
+ {
+ AddBufferToFreelist(bufHdr);
+ bufHdr->flags |= BM_FREE;
+ }
+ }
+
+ return STATUS_OK;
+}
+
+
#ifdef NOT_USED
void
IncrBufferRefCount_Debug(char *file, int line, Buffer buffer)
@@ -2217,9 +2262,9 @@ MarkBufferForCleanup(Buffer buffer, void (*CleanupFunc)(Buffer))
SpinRelease(BufMgrLock);
LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
- PrivateRefCount[buffer - 1]--;
SpinAcquire(BufMgrLock);
+ PrivateRefCount[buffer - 1] = 0;
Assert(bufHdr->refcount > 0);
bufHdr->flags |= (BM_DIRTY | BM_JUST_DIRTIED);
bufHdr->CleanupFunc = CleanupFunc;
diff --git a/src/backend/storage/buffer/s_lock.c b/src/backend/storage/buffer/s_lock.c
index 932e5b0049b..00a934c3832 100644
--- a/src/backend/storage/buffer/s_lock.c
+++ b/src/backend/storage/buffer/s_lock.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/buffer/Attic/s_lock.c,v 1.28 2000/12/29 21:31:20 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/buffer/Attic/s_lock.c,v 1.29 2001/01/14 05:08:15 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -17,6 +17,7 @@
#include <sys/time.h>
#include <unistd.h>
+#include "miscadmin.h"
#include "storage/s_lock.h"
@@ -101,10 +102,16 @@ s_lock(volatile slock_t *lock, const char *file, const int line)
/*
* If you are thinking of changing this code, be careful. This same
* loop logic is used in other places that call TAS() directly.
+ *
+ * While waiting for a lock, we check for cancel/die interrupts (which
+ * is a no-op if we are inside a critical section). The interrupt check
+ * can be omitted in places that know they are inside a critical section.
+ * Note that an interrupt must NOT be accepted after acquiring the lock.
*/
while (TAS(lock))
{
s_lock_sleep(spins++, 0, lock, file, line);
+ CHECK_FOR_INTERRUPTS();
}
}
diff --git a/src/backend/storage/ipc/ipc.c b/src/backend/storage/ipc/ipc.c
index d592a179867..9d796299dc6 100644
--- a/src/backend/storage/ipc/ipc.c
+++ b/src/backend/storage/ipc/ipc.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/ipc/ipc.c,v 1.59 2001/01/07 04:30:41 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/ipc/ipc.c,v 1.60 2001/01/14 05:08:15 tgl Exp $
*
* NOTES
*
@@ -131,8 +131,12 @@ proc_exit(int code)
* to close up shop already. Note that the signal handlers will not
* set these flags again, now that proc_exit_inprogress is set.
*/
- QueryCancel = false;
+ InterruptPending = false;
ProcDiePending = false;
+ QueryCancelPending = false;
+ /* And let's just make *sure* we're not interrupted ... */
+ ImmediateInterruptOK = false;
+ CritSectionCount = 1;
if (DebugLvl > 1)
elog(DEBUG, "proc_exit(%d)", code);
@@ -367,7 +371,7 @@ CallbackSemaphoreKill(int status, Datum semId)
/* IpcSemaphoreLock(semId, sem) - locks a semaphore */
/****************************************************************************/
void
-IpcSemaphoreLock(IpcSemaphoreId semId, int sem)
+IpcSemaphoreLock(IpcSemaphoreId semId, int sem, bool interruptOK)
{
int errStatus;
struct sembuf sops;
@@ -380,11 +384,43 @@ IpcSemaphoreLock(IpcSemaphoreId semId, int sem)
* Note: if errStatus is -1 and errno == EINTR then it means we
* returned from the operation prematurely because we were
* sent a signal. So we try and lock the semaphore again.
- * ----------------
+ *
+ * Each time around the loop, we check for a cancel/die interrupt.
+ * We assume that if such an interrupt comes in while we are waiting,
+ * it will cause the semop() call to exit with errno == EINTR, so that
+ * we will be able to service the interrupt (if not in a critical
+ * section already).
+ *
+ * Once we acquire the lock, we do NOT check for an interrupt before
+ * returning. The caller needs to be able to record ownership of
+ * the lock before any interrupt can be accepted.
+ *
+ * There is a window of a few instructions between CHECK_FOR_INTERRUPTS
+ * and entering the semop() call. If a cancel/die interrupt occurs in
+ * that window, we would fail to notice it until after we acquire the
+ * lock (or get another interrupt to escape the semop()). We can avoid
+ * this problem by temporarily setting ImmediateInterruptOK = true
+ * before we do CHECK_FOR_INTERRUPTS; then, a die() interrupt in this
+ * interval will execute directly. However, there is a huge pitfall:
+ * there is another window of a few instructions after the semop()
+ * before we are able to reset ImmediateInterruptOK. If an interrupt
+ * occurs then, we'll lose control, which means that the lock has been
+ * acquired but our caller did not get a chance to record the fact.
+ * Therefore, we only set ImmediateInterruptOK if the caller tells us
+ * it's OK to do so, ie, the caller does not need to record acquiring
+ * the lock. (This is currently true for lockmanager locks, since the
+ * process that granted us the lock did all the necessary state updates.
+ * It's not true for SysV semaphores used to emulate spinlocks --- but
+ * our performance on such platforms is so horrible anyway that I'm
+ * not going to worry too much about it.)
+ * ----------------
*/
do
{
+ ImmediateInterruptOK = interruptOK;
+ CHECK_FOR_INTERRUPTS();
errStatus = semop(semId, &sops, 1);
+ ImmediateInterruptOK = false;
} while (errStatus == -1 && errno == EINTR);
if (errStatus == -1)
diff --git a/src/backend/storage/ipc/spin.c b/src/backend/storage/ipc/spin.c
index ed71d79ad9f..b27c1810020 100644
--- a/src/backend/storage/ipc/spin.c
+++ b/src/backend/storage/ipc/spin.c
@@ -14,7 +14,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/ipc/Attic/spin.c,v 1.28 2001/01/12 21:53:59 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/ipc/Attic/spin.c,v 1.29 2001/01/14 05:08:15 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -25,9 +25,11 @@
#include <sys/sem.h>
#endif
+#include "miscadmin.h"
#include "storage/proc.h"
#include "storage/s_lock.h"
+
/* Probably should move these to an appropriate header file */
extern SPINLOCK ShmemLock;
extern SPINLOCK ShmemIndexLock;
@@ -145,19 +147,20 @@ SpinAcquire(SPINLOCK lockid)
PRINT_SLDEBUG("SpinAcquire", lockid, slckP);
/*
- * Lock out die() until we exit the critical section protected by the
- * spinlock. This ensures that die() will not interrupt manipulations
- * of data structures in shared memory. We don't want die() to
- * interrupt this routine between S_LOCK and PROC_INCR_SLOCK, either,
- * so must do it before acquiring the lock, not after.
- */
- START_CRIT_SECTION();
- /*
* Acquire the lock, then record that we have done so (for recovery
- * in case of elog(ERROR) during the critical section).
+ * in case of elog(ERROR) during the critical section). Note we assume
+ * here that S_LOCK will not accept cancel/die interrupts once it has
+ * acquired the lock. However, interrupts should be accepted while
+ * waiting, if CritSectionCount is zero.
*/
S_LOCK(&(slckP->shlock));
PROC_INCR_SLOCK(lockid);
+ /*
+ * Lock out cancel/die interrupts until we exit the critical section
+ * protected by the spinlock. This ensures that interrupts will not
+ * interfere with manipulations of data structures in shared memory.
+ */
+ START_CRIT_SECTION();
PRINT_SLDEBUG("SpinAcquire/done", lockid, slckP);
}
@@ -317,10 +320,16 @@ SpinFreeAllSemaphores(void)
void
SpinAcquire(SPINLOCK lock)
{
- /* See the TAS() version of this routine for commentary */
- START_CRIT_SECTION();
- IpcSemaphoreLock(SpinLockIds[0], lock);
+ /*
+ * See the TAS() version of this routine for primary commentary.
+ *
+ * NOTE we must pass interruptOK = false to IpcSemaphoreLock, to ensure
+ * that a cancel/die interrupt cannot prevent us from recording ownership
+ * of a lock we have just acquired.
+ */
+ IpcSemaphoreLock(SpinLockIds[0], lock, false);
PROC_INCR_SLOCK(lock);
+ START_CRIT_SECTION();
}
/*
@@ -338,8 +347,8 @@ SpinRelease(SPINLOCK lock)
semval = IpcSemaphoreGetValue(SpinLockIds[0], lock);
Assert(semval < 1);
- Assert(!MyProc || MyProc->sLocks[lockid] > 0);
#endif
+ Assert(!MyProc || MyProc->sLocks[lockid] > 0);
PROC_DECR_SLOCK(lock);
IpcSemaphoreUnlock(SpinLockIds[0], lock);
END_CRIT_SECTION();
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index f15ee9f8bd6..e7d1b678bef 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/lock.c,v 1.76 2001/01/10 01:24:19 inoue Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/lock.c,v 1.77 2001/01/14 05:08:15 tgl Exp $
*
* NOTES
* Outside modules can create a lock table and acquire/release
@@ -727,6 +727,12 @@ LockAcquire(LOCKMETHOD lockmethod, LOCKTAG *locktag,
status = WaitOnLock(lockmethod, lockmode, lock, holder);
/*
+ * 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 holder entry status, in case something in the ipc
* communication doesn't work correctly.
*/
@@ -921,6 +927,8 @@ GrantLock(LOCK *lock, HOLDER *holder, LOCKMODE lockmode)
lock->nActive++;
lock->activeHolders[lockmode]++;
lock->mask |= BITS_ON[lockmode];
+ if (lock->activeHolders[lockmode] == lock->holders[lockmode])
+ lock->waitMask &= BITS_OFF[lockmode];
LOCK_PRINT("GrantLock", lock, lockmode);
Assert((lock->nActive > 0) && (lock->activeHolders[lockmode] > 0));
Assert(lock->nActive <= lock->nHolding);
@@ -960,6 +968,17 @@ WaitOnLock(LOCKMETHOD lockmethod, LOCKMODE lockmode,
strcat(new_status, " waiting");
set_ps_display(new_status);
+ /*
+ * NOTE: Think not to put any lock state cleanup after the call to
+ * ProcSleep, in either the normal or failure path. The lock state
+ * must be fully set by the lock grantor, or by HandleDeadlock if we
+ * give up waiting for the lock. This is necessary because of the
+ * possibility that a cancel/die interrupt will interrupt ProcSleep
+ * after someone else grants us the lock, but before we've noticed it.
+ * Hence, after granting, the locktable state must fully reflect the
+ * fact that we own the lock; we can't do additional work on return.
+ */
+
if (ProcSleep(lockMethodTable->ctl,
lockmode,
lock,
@@ -967,26 +986,16 @@ WaitOnLock(LOCKMETHOD lockmethod, LOCKMODE lockmode,
{
/* -------------------
* We failed as a result of a deadlock, see HandleDeadLock().
- * Decrement the lock nHolding and holders fields as
- * we are no longer waiting on this lock. Removal of the holder and
- * lock objects, if no longer needed, will happen in xact cleanup.
+ * Quit now. Removal of the holder and lock objects, if no longer
+ * needed, will happen in xact cleanup (see above for motivation).
* -------------------
*/
- lock->nHolding--;
- lock->holders[lockmode]--;
LOCK_PRINT("WaitOnLock: aborting on lock", lock, lockmode);
- Assert((lock->nHolding >= 0) && (lock->holders[lockmode] >= 0));
- Assert(lock->nActive <= lock->nHolding);
- if (lock->activeHolders[lockmode] == lock->holders[lockmode])
- lock->waitMask &= BITS_OFF[lockmode];
SpinRelease(lockMethodTable->ctl->masterLock);
elog(ERROR, DeadLockMessage);
/* not reached */
}
- if (lock->activeHolders[lockmode] == lock->holders[lockmode])
- lock->waitMask &= BITS_OFF[lockmode];
-
set_ps_display(old_status);
pfree(old_status);
pfree(new_status);
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index baa31413e2f..b5a22bb2321 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.91 2001/01/12 21:53:59 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.92 2001/01/14 05:08:16 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -48,7 +48,7 @@
* This is so that we can support more backends. (system-wide semaphore
* sets run out pretty fast.) -ay 4/95
*
- * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.91 2001/01/12 21:53:59 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.92 2001/01/14 05:08:16 tgl Exp $
*/
#include "postgres.h"
@@ -78,11 +78,6 @@
#include "storage/proc.h"
-
-void HandleDeadLock(SIGNAL_ARGS);
-static void ProcFreeAllSemaphores(void);
-static bool GetOffWaitQueue(PROC *);
-
int DeadlockTimeout = 1000;
/* --------------------
@@ -98,9 +93,14 @@ static PROC_HDR *ProcGlobal = NULL;
PROC *MyProc = NULL;
-static void ProcKill(int exitStatus, Datum pid);
+static bool waitingForLock = false;
+
+static void ProcKill(void);
static void ProcGetNewSemIdAndNum(IpcSemaphoreId *semId, int *semNum);
static void ProcFreeSem(IpcSemaphoreId semId, int semNum);
+static void ZeroProcSemaphore(PROC *proc);
+static void ProcFreeAllSemaphores(void);
+
/*
* InitProcGlobal -
@@ -241,27 +241,23 @@ InitProcess(void)
MemSet(MyProc->sLocks, 0, sizeof(MyProc->sLocks));
MyProc->sLocks[ProcStructLock] = 1;
+ /*
+ * Set up a wait-semaphore for the proc.
+ */
if (IsUnderPostmaster)
{
- IpcSemaphoreId semId;
- int semNum;
- union semun semun;
-
- ProcGetNewSemIdAndNum(&semId, &semNum);
-
+ ProcGetNewSemIdAndNum(&MyProc->sem.semId, &MyProc->sem.semNum);
/*
* we might be reusing a semaphore that belongs to a dead backend.
* So be careful and reinitialize its value here.
*/
- semun.val = 1;
- semctl(semId, semNum, SETVAL, semun);
-
- IpcSemaphoreLock(semId, semNum);
- MyProc->sem.semId = semId;
- MyProc->sem.semNum = semNum;
+ ZeroProcSemaphore(MyProc);
}
else
+ {
MyProc->sem.semId = -1;
+ MyProc->sem.semNum = -1;
+ }
MyProc->pid = MyProcPid;
MyProc->databaseId = MyDatabaseId;
@@ -282,67 +278,126 @@ InitProcess(void)
* -------------------------
*/
location = MAKE_OFFSET(MyProc);
- if ((!ShmemPIDLookup(MyProcPid, &location)) || (location != MAKE_OFFSET(MyProc)))
+ if ((!ShmemPIDLookup(MyProcPid, &location)) ||
+ (location != MAKE_OFFSET(MyProc)))
elog(STOP, "InitProcess: ShmemPID table broken");
MyProc->errType = NO_ERROR;
SHMQueueElemInit(&(MyProc->links));
- on_shmem_exit(ProcKill, (Datum) MyProcPid);
+ on_shmem_exit(ProcKill, 0);
}
-/* -----------------------
- * get process off any wait queue it might be on
+/*
+ * Initialize the proc's wait-semaphore to count zero.
+ */
+static void
+ZeroProcSemaphore(PROC *proc)
+{
+ union semun semun;
+
+ semun.val = 0;
+ if (semctl(proc->sem.semId, proc->sem.semNum, SETVAL, semun) < 0)
+ {
+ fprintf(stderr, "ZeroProcSemaphore: semctl(id=%d,SETVAL) failed: %s\n",
+ proc->sem.semId, strerror(errno));
+ proc_exit(255);
+ }
+}
+
+/*
+ * Remove a proc from the wait-queue it is on
+ * (caller must know it is on one).
+ * Locktable lock must be held by caller.
*
* NB: this does not remove the process' holder object, nor the lock object,
* even though their holder counts might now have gone to zero. That will
* happen during a subsequent LockReleaseAll call, which we expect will happen
* during transaction cleanup. (Removal of a proc from its wait queue by
* this routine can only happen if we are aborting the transaction.)
- * -----------------------
*/
-static bool
-GetOffWaitQueue(PROC *proc)
+static void
+RemoveFromWaitQueue(PROC *proc)
{
- bool gotoff = false;
+ LOCK *waitLock = proc->waitLock;
+ LOCKMODE lockmode = proc->waitLockMode;
- LockLockTable();
- if (proc->links.next != INVALID_OFFSET)
+ /* Make sure proc is waiting */
+ Assert(proc->links.next != INVALID_OFFSET);
+ Assert(waitLock);
+ Assert(waitLock->waitProcs.size > 0);
+
+ /* Remove proc from lock's wait queue */
+ SHMQueueDelete(&(proc->links));
+ waitLock->waitProcs.size--;
+
+ /* Undo increments of holder counts by waiting process */
+ Assert(waitLock->nHolding > 0);
+ Assert(waitLock->nHolding > proc->waitLock->nActive);
+ waitLock->nHolding--;
+ Assert(waitLock->holders[lockmode] > 0);
+ waitLock->holders[lockmode]--;
+ /* don't forget to clear waitMask bit if appropriate */
+ if (waitLock->activeHolders[lockmode] == waitLock->holders[lockmode])
+ waitLock->waitMask &= ~(1 << lockmode);
+
+ /* Clean up the proc's own state */
+ SHMQueueElemInit(&(proc->links));
+ proc->waitLock = NULL;
+ proc->waitHolder = NULL;
+
+ /* See if any other waiters for the lock can be woken up now */
+ ProcLockWakeup(LOCK_LOCKMETHOD(*waitLock), waitLock);
+}
+
+/*
+ * Cancel any pending wait for lock, when aborting a transaction.
+ *
+ * (Normally, this would only happen if we accept a cancel/die
+ * interrupt while waiting; but an elog(ERROR) while waiting is
+ * within the realm of possibility, too.)
+ */
+void
+LockWaitCancel(void)
+{
+ /* Nothing to do if we weren't waiting for a lock */
+ if (!waitingForLock)
+ return;
+ waitingForLock = false;
+
+ /* Turn off the deadlock timer, if it's still running (see ProcSleep) */
+#ifndef __BEOS__
{
- LOCK *waitLock = proc->waitLock;
- LOCKMODE lockmode = proc->waitLockMode;
-
- /* Remove proc from lock's wait queue */
- Assert(waitLock);
- Assert(waitLock->waitProcs.size > 0);
- SHMQueueDelete(&(proc->links));
- --waitLock->waitProcs.size;
-
- /* Undo increments of holder counts by waiting process */
- Assert(waitLock->nHolding > 0);
- Assert(waitLock->nHolding > proc->waitLock->nActive);
- --waitLock->nHolding;
- Assert(waitLock->holders[lockmode] > 0);
- --waitLock->holders[lockmode];
- /* don't forget to clear waitMask bit if appropriate */
- if (waitLock->activeHolders[lockmode] == waitLock->holders[lockmode])
- waitLock->waitMask &= ~(1 << lockmode);
-
- /* Clean up the proc's own state */
- SHMQueueElemInit(&(proc->links));
- proc->waitLock = NULL;
- proc->waitHolder = NULL;
-
- /* See if any other waiters can be woken up now */
- ProcLockWakeup(LOCK_LOCKMETHOD(*waitLock), waitLock);
-
- gotoff = true;
+ struct itimerval timeval,
+ dummy;
+
+ MemSet(&timeval, 0, sizeof(struct itimerval));
+ setitimer(ITIMER_REAL, &timeval, &dummy);
}
+#else
+ /* BeOS doesn't have setitimer, but has set_alarm */
+ set_alarm(B_INFINITE_TIMEOUT, B_PERIODIC_ALARM);
+#endif /* __BEOS__ */
+
+ /* Unlink myself from the wait queue, if on it (might not be anymore!) */
+ LockLockTable();
+ if (MyProc->links.next != INVALID_OFFSET)
+ RemoveFromWaitQueue(MyProc);
UnlockLockTable();
- return gotoff;
+ /*
+ * Reset the proc wait semaphore to zero. This is necessary in the
+ * scenario where someone else granted us the lock we wanted before we
+ * were able to remove ourselves from the wait-list. The semaphore will
+ * have been bumped to 1 by the would-be grantor, and since we are no
+ * longer going to wait on the sema, we have to force it back to zero.
+ * Otherwise, our next attempt to wait for a lock will fall through
+ * prematurely.
+ */
+ ZeroProcSemaphore(MyProc);
}
+
/*
* ProcReleaseLocks() -- release locks associated with current transaction
* at transaction commit or abort
@@ -360,15 +415,17 @@ ProcReleaseLocks(bool isCommit)
{
if (!MyProc)
return;
- GetOffWaitQueue(MyProc);
+ /* If waiting, get off wait queue (should only be needed after error) */
+ LockWaitCancel();
+ /* Release locks */
LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc,
!isCommit, GetCurrentTransactionId());
}
/*
* ProcRemove -
- * used by the postmaster to clean up the global tables. This also frees
- * up the semaphore used for the lmgr of the process.
+ * called by the postmaster to clean up the global tables after a
+ * backend exits. This also frees up the proc's wait semaphore.
*/
bool
ProcRemove(int pid)
@@ -376,8 +433,6 @@ ProcRemove(int pid)
SHMEM_OFFSET location;
PROC *proc;
- location = INVALID_OFFSET;
-
location = ShmemPIDDestroy(pid);
if (location == INVALID_OFFSET)
return FALSE;
@@ -398,43 +453,30 @@ ProcRemove(int pid)
/*
* ProcKill() -- Destroy the per-proc data structure for
* this process. Release any of its held spin locks.
+ *
+ * This is done inside the backend process before it exits.
+ * ProcRemove, above, will be done by the postmaster afterwards.
*/
static void
-ProcKill(int exitStatus, Datum pid)
+ProcKill(void)
{
- PROC *proc;
-
- if ((int) pid == MyProcPid)
- {
- proc = MyProc;
- MyProc = NULL;
- }
- else
- {
- /* This path is dead code at the moment ... */
- SHMEM_OFFSET location = INVALID_OFFSET;
-
- ShmemPIDLookup((int) pid, &location);
- if (location == INVALID_OFFSET)
- return;
- proc = (PROC *) MAKE_PTR(location);
- }
-
- Assert(proc);
+ Assert(MyProc);
- /* Release any spinlocks the proc is holding */
- ProcReleaseSpins(proc);
+ /* Release any spinlocks I am holding */
+ ProcReleaseSpins(MyProc);
- /* Get the proc off any wait queue it might be on */
- GetOffWaitQueue(proc);
+ /* Get off any wait queue I might be on */
+ LockWaitCancel();
/* Remove from the standard lock table */
- LockReleaseAll(DEFAULT_LOCKMETHOD, proc, true, InvalidTransactionId);
+ LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, true, InvalidTransactionId);
#ifdef USER_LOCKS
/* Remove from the user lock table */
- LockReleaseAll(USER_LOCKMETHOD, proc, true, InvalidTransactionId);
+ LockReleaseAll(USER_LOCKMETHOD, MyProc, true, InvalidTransactionId);
#endif
+
+ MyProc = NULL;
}
/*
@@ -477,68 +519,13 @@ ProcQueueInit(PROC_QUEUE *queue)
/*
- * Handling cancel request while waiting for lock
- *
- */
-static bool lockWaiting = false;
-
-void
-SetWaitingForLock(bool waiting)
-{
- if (waiting == lockWaiting)
- return;
- lockWaiting = waiting;
- if (lockWaiting)
- {
- /* The lock was already released ? */
- if (MyProc->links.next == INVALID_OFFSET)
- {
- lockWaiting = false;
- return;
- }
- if (QueryCancel) /* cancel request pending */
- {
- if (GetOffWaitQueue(MyProc))
- {
- lockWaiting = false;
- elog(ERROR, "Query cancel requested while waiting for lock");
- }
- }
- }
-}
-
-void
-LockWaitCancel(void)
-{
-#ifndef __BEOS__
- struct itimerval timeval,
- dummy;
-
- if (!lockWaiting)
- return;
- lockWaiting = false;
- /* Deadlock timer off */
- MemSet(&timeval, 0, sizeof(struct itimerval));
- setitimer(ITIMER_REAL, &timeval, &dummy);
-#else
- /* BeOS doesn't have setitimer, but has set_alarm */
- if (!lockWaiting)
- return;
- lockWaiting = false;
- /* Deadlock timer off */
- set_alarm(B_INFINITE_TIMEOUT, B_PERIODIC_ALARM);
-#endif /* __BEOS__ */
-
- if (GetOffWaitQueue(MyProc))
- elog(ERROR, "Query cancel requested while waiting for lock");
-}
-
-/*
* ProcSleep -- put a process to sleep
*
* P() on the semaphore should put us to sleep. The process
- * semaphore is cleared by default, so the first time we try
- * to acquire it, we sleep.
+ * semaphore is normally zero, so when we try to acquire it, we sleep.
+ *
+ * Locktable's spinlock must be held at entry, and will be held
+ * at exit.
*
* Result is NO_ERROR if we acquired the lock, STATUS_ERROR if not (deadlock).
*
@@ -629,7 +616,7 @@ ProcSleep(LOCKMETHODCTL *lockctl,
ins:;
/* -------------------
- * assume that these two operations are atomic (because
+ * Insert self into queue. These operations are atomic (because
* of the spinlock).
* -------------------
*/
@@ -640,6 +627,18 @@ ins:;
MyProc->errType = NO_ERROR; /* initialize result for success */
+ /* mark that we are waiting for a lock */
+ waitingForLock = true;
+
+ /* -------------------
+ * Release the locktable's spin lock.
+ *
+ * NOTE: this may also cause us to exit critical-section state,
+ * possibly allowing a cancel/die interrupt to be accepted.
+ * This is OK because we have recorded the fact that we are waiting for
+ * a lock, and so LockWaitCancel will clean up if cancel/die happens.
+ * -------------------
+ */
SpinRelease(spinlock);
/* --------------
@@ -667,8 +666,6 @@ ins:;
elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
#endif
- SetWaitingForLock(true);
-
/* --------------
* If someone wakes us between SpinRelease and IpcSemaphoreLock,
* IpcSemaphoreLock will not block. The wakeup is "saved" by
@@ -676,19 +673,22 @@ ins:;
* is invoked but does not detect a deadlock, IpcSemaphoreLock()
* will continue to wait. There used to be a loop here, but it
* was useless code...
+ *
+ * We pass interruptOK = true, which eliminates a window in which
+ * cancel/die interrupts would be held off undesirably. This is a
+ * promise that we don't mind losing control to a cancel/die interrupt
+ * here. We don't, because we have no state-change work to do after
+ * being granted the lock (the grantor did it all).
* --------------
*/
- IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum);
-
- lockWaiting = false;
+ IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum, true);
/* ---------------
* Disable the timer, if it's still running
* ---------------
*/
#ifndef __BEOS__
- timeval.it_value.tv_sec = 0;
- timeval.it_value.tv_usec = 0;
+ MemSet(&timeval, 0, sizeof(struct itimerval));
if (setitimer(ITIMER_REAL, &timeval, &dummy))
elog(FATAL, "ProcSleep: Unable to disable timer for process wakeup");
#else
@@ -696,9 +696,16 @@ ins:;
elog(FATAL, "ProcSleep: Unable to disable timer for process wakeup");
#endif
+ /*
+ * Now there is nothing for LockWaitCancel to do.
+ */
+ waitingForLock = false;
+
/* ----------------
- * We were assumed to be in a critical section when we went
- * to sleep.
+ * Re-acquire the locktable's spin lock.
+ *
+ * We could accept a cancel/die interrupt here. That's OK because
+ * the lock is now registered as being held by this process.
* ----------------
*/
SpinAcquire(spinlock);
@@ -836,20 +843,24 @@ ProcAddLock(SHM_QUEUE *elem)
/* --------------------
* We only get to this routine if we got SIGALRM after DeadlockTimeout
- * while waiting for a lock to be released by some other process. If we have
- * a real deadlock, we must also indicate that I'm no longer waiting
- * on a lock so that other processes don't try to wake me up and screw
- * up my semaphore.
+ * while waiting for a lock to be released by some other process. Look
+ * to see if there's a deadlock; if not, just return and continue waiting.
+ * If we have a real deadlock, remove ourselves from the lock's wait queue
+ * and signal an error to ProcSleep.
* --------------------
*/
void
HandleDeadLock(SIGNAL_ARGS)
{
int save_errno = errno;
- LOCK *mywaitlock;
- bool isWaitingForLock = lockWaiting; /* save waiting status */
- SetWaitingForLock(false); /* disable query cancel during this fuction */
+ /*
+ * Acquire locktable lock. Note that the SIGALRM interrupt had better
+ * not be enabled anywhere that this process itself holds the locktable
+ * lock, else this will wait forever. Also note that this calls
+ * SpinAcquire which creates a critical section, so that this routine
+ * cannot be interrupted by cancel/die interrupts.
+ */
LockLockTable();
/* ---------------------
@@ -869,7 +880,6 @@ HandleDeadLock(SIGNAL_ARGS)
{
UnlockLockTable();
errno = save_errno;
- SetWaitingForLock(isWaitingForLock); /* restore waiting status */
return;
}
@@ -883,22 +893,23 @@ HandleDeadLock(SIGNAL_ARGS)
/* No deadlock, so keep waiting */
UnlockLockTable();
errno = save_errno;
- SetWaitingForLock(isWaitingForLock); /* restore waiting status */
return;
}
/* ------------------------
- * Get this process off the lock's wait queue
+ * Oops. We have a deadlock.
+ *
+ * Get this process out of wait state.
* ------------------------
*/
- mywaitlock = MyProc->waitLock;
- Assert(mywaitlock->waitProcs.size > 0);
- --mywaitlock->waitProcs.size;
- SHMQueueDelete(&(MyProc->links));
- SHMQueueElemInit(&(MyProc->links));
- MyProc->waitLock = NULL;
- MyProc->waitHolder = NULL;
- isWaitingForLock = false; /* wait for lock no longer */
+ RemoveFromWaitQueue(MyProc);
+
+ /* -------------
+ * Set MyProc->errType to STATUS_ERROR so that ProcSleep will
+ * report an error after we return from this signal handler.
+ * -------------
+ */
+ MyProc->errType = STATUS_ERROR;
/* ------------------
* Unlock my semaphore so that the interrupted ProcSleep() call can finish.
@@ -906,17 +917,16 @@ HandleDeadLock(SIGNAL_ARGS)
*/
IpcSemaphoreUnlock(MyProc->sem.semId, MyProc->sem.semNum);
- /* -------------
- * Set MyProc->errType to STATUS_ERROR so that we abort after
- * returning from this handler.
- * -------------
- */
- MyProc->errType = STATUS_ERROR;
-
- /*
- * if this doesn't follow the IpcSemaphoreUnlock then we get lock
- * table corruption ("LockReplace: xid table corrupted") due to race
- * conditions. i don't claim to understand this...
+ /* ------------------
+ * We're done here. Transaction abort caused by the error that ProcSleep
+ * will raise will cause any other locks we hold to be released, thus
+ * allowing other processes to wake up; we don't need to do that here.
+ * NOTE: an exception is that releasing locks we hold doesn't consider
+ * the possibility of waiters that were blocked behind us on the lock
+ * we just failed to get, and might now be wakable because we're not
+ * in front of them anymore. However, RemoveFromWaitQueue took care of
+ * waking up any such processes.
+ * ------------------
*/
UnlockLockTable();
errno = save_errno;