]> granicus.if.org Git - postgresql/blobdiff - src/backend/storage/lmgr/proc.c
Banish caddr_t (mostly), use Datum where appropriate.
[postgresql] / src / backend / storage / lmgr / proc.c
index 74adf58a4de04f8e9522978effe5ddd096df6193..e68d96d772951f89f4aaf1093e92b152ca0eed26 100644 (file)
@@ -1,13 +1,14 @@
 /*-------------------------------------------------------------------------
  *
- * proc.c--
+ * proc.c
  *       routines to manage per-process shared memory data structure
  *
- * Copyright (c) 1994, Regents of the University of California
+ * Portions Copyright (c) 1996-2000, PostgreSQL, Inc
+ * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
- *       $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.44 1998/12/18 19:45:37 momjian Exp $
+ *       $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.80 2000/10/02 19:42:48 petere Exp $
  *
  *-------------------------------------------------------------------------
  */
  *             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.44 1998/12/18 19:45:37 momjian Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.80 2000/10/02 19:42:48 petere Exp $
  */
+#include "postgres.h"
+
 #include <sys/time.h>
 #include <unistd.h>
-#include <string.h>
 #include <signal.h>
 #include <sys/types.h>
 
-#if defined(solaris_sparc)
+#if defined(solaris_sparc) || defined(__CYGWIN__)
 #include <sys/ipc.h>
 #include <sys/sem.h>
 #endif
 
-#include "postgres.h"
 #include "miscadmin.h"
-#include "libpq/pqsignal.h"
 
-#include "access/xact.h"
-#include "utils/hsearch.h"
 
-#include "storage/ipc.h"
-/* In Ultrix, sem.h must be included after ipc.h */
+/* In Ultrix and QNX, sem.h must be included after ipc.h */
 #include <sys/sem.h>
-#include "storage/buf.h"
-#include "storage/lock.h"
-#include "storage/lmgr.h"
-#include "storage/shmem.h"
-#include "storage/spin.h"
+
 #include "storage/proc.h"
-#include "utils/trace.h"
 
-static void HandleDeadLock(void);
-static PROC *ProcWakeup(PROC *proc, int errType);
+void           HandleDeadLock(SIGNAL_ARGS);
+static void ProcFreeAllSemaphores(void);
+static bool GetOffWaitqueue(PROC *);
 
-#define DeadlockCheckTimer pg_options[OPT_DEADLOCKTIMEOUT]
+int DeadlockTimeout = 1000;
 
 /* --------------------
  * Spin lock for manipulating the shared process data structure:
@@ -91,12 +84,6 @@ static PROC *ProcWakeup(PROC *proc, int errType);
  */
 SPINLOCK       ProcStructLock;
 
-/*
- * For cleanup routines.  Don't cleanup if the initialization
- * has not happened.
- */
-static bool ProcInitialized = FALSE;
-
 static PROC_HDR *ProcGlobal = NULL;
 
 PROC      *MyProc = NULL;
@@ -105,25 +92,40 @@ static void ProcKill(int exitStatus, int pid);
 static void ProcGetNewSemKeyAndNum(IPCKey *key, int *semNum);
 static void ProcFreeSem(IpcSemaphoreKey semKey, int semNum);
 
+static char *DeadLockMessage = "Deadlock detected -- See the lock(l) manual page for a possible cause.";
+
 /*
  * InitProcGlobal -
  *       initializes the global process table. We put it here so that
- *       the postmaster can do this initialization. (ProcFreeAllSem needs
+ *       the postmaster can do this initialization. (ProcFreeAllSemaphores needs
  *       to read this table on exiting the postmaster. If we have the first
  *       backend do this, starting up and killing the postmaster without
  *       starting any backends will be a problem.)
+ *
+ *       We also allocate all the per-process semaphores we will need to support
+ *       the requested number of backends.  We used to allocate semaphores
+ *       only when backends were actually started up, but that is bad because
+ *       it lets Postgres fail under load --- a lot of Unix systems are
+ *       (mis)configured with small limits on the number of semaphores, and
+ *       running out when trying to start another backend is a common failure.
+ *       So, now we grab enough semaphores to support the desired max number
+ *       of backends immediately at initialization --- if the sysadmin has set
+ *       MaxBackends higher than his kernel will support, he'll find out sooner
+ *       rather than later.
  */
 void
-InitProcGlobal(IPCKey key)
+InitProcGlobal(IPCKey key, int maxBackends)
 {
        bool            found = false;
 
        /* attach to the free list */
        ProcGlobal = (PROC_HDR *)
-               ShmemInitStruct("Proc Header", (unsigned) sizeof(PROC_HDR), &found);
+               ShmemInitStruct("Proc Header", sizeof(PROC_HDR), &found);
 
        /* --------------------
         * We're the first - initialize.
+        * XXX if found should ever be true, it is a sign of impending doom ...
+        * ought to complain if so?
         * --------------------
         */
        if (!found)
@@ -134,6 +136,37 @@ InitProcGlobal(IPCKey key)
                ProcGlobal->currKey = IPCGetProcessSemaphoreInitKey(key);
                for (i = 0; i < MAX_PROC_SEMS / PROC_NSEMS_PER_SET; i++)
                        ProcGlobal->freeSemMap[i] = 0;
+
+               /*
+                * Arrange to delete semas on exit --- set this up now so that we
+                * will clean up if pre-allocation fails...
+                */
+               on_shmem_exit(ProcFreeAllSemaphores, 0);
+
+               /*
+                * Pre-create the semaphores for the first maxBackends processes,
+                * unless we are running as a standalone backend.
+                */
+               if (key != PrivateIPCKey)
+               {
+                       for (i = 0;
+                                i < (maxBackends + PROC_NSEMS_PER_SET - 1) / PROC_NSEMS_PER_SET;
+                                i++)
+                       {
+                               IPCKey          semKey = ProcGlobal->currKey + i;
+                               int                     semId;
+
+                               semId = IpcSemaphoreCreate(semKey,
+                                                                                  PROC_NSEMS_PER_SET,
+                                                                                  IPCProtection,
+                                                                                  IpcSemaphoreDefaultStartValue,
+                                                                                  0);
+                               if (semId < 0)
+                                       elog(FATAL, "InitProcGlobal: IpcSemaphoreCreate failed");
+                               /* mark this sema set allocated */
+                               ProcGlobal->freeSemMap[i] = (1 << PROC_NSEMS_PER_SET);
+                       }
+               }
        }
 }
 
@@ -146,23 +179,18 @@ void
 InitProcess(IPCKey key)
 {
        bool            found = false;
-       int                     semstat;
        unsigned long location,
                                myOffset;
 
-       /* ------------------
-        * Routine called if deadlock timer goes off. See ProcSleep()
-        * ------------------
-        */
        SpinAcquire(ProcStructLock);
 
        /* attach to the free list */
        ProcGlobal = (PROC_HDR *)
-               ShmemInitStruct("Proc Header", (unsigned) sizeof(PROC_HDR), &found);
+               ShmemInitStruct("Proc Header", sizeof(PROC_HDR), &found);
        if (!found)
        {
                /* this should not happen. InitProcGlobal() is called before this. */
-               elog(ERROR, "InitProcess: Proc Header uninitialized");
+               elog(STOP, "InitProcess: Proc Header uninitialized");
        }
 
        if (MyProc != NULL)
@@ -191,7 +219,7 @@ InitProcess(IPCKey key)
                 * cleanup dead processes).
                 */
 
-               MyProc = (PROC *) ShmemAlloc((unsigned) sizeof(PROC));
+               MyProc = (PROC *) ShmemAlloc(sizeof(PROC));
                if (!MyProc)
                {
                        SpinRelease(ProcStructLock);
@@ -220,12 +248,17 @@ InitProcess(IPCKey key)
 
                ProcGetNewSemKeyAndNum(&semKey, &semNum);
 
+               /*
+                * Note: because of the pre-allocation done in InitProcGlobal,
+                * this call should always attach to an existing semaphore. It
+                * will (try to) create a new group of semaphores only if the
+                * postmaster tries to start more backends than it said it would.
+                */
                semId = IpcSemaphoreCreate(semKey,
                                                                   PROC_NSEMS_PER_SET,
                                                                   IPCProtection,
                                                                   IpcSemaphoreDefaultStartValue,
-                                                                  0,
-                                                                  &semstat);
+                                                                  0);
 
                /*
                 * we might be reusing a semaphore that belongs to a dead backend.
@@ -249,10 +282,9 @@ InitProcess(IPCKey key)
        SpinRelease(ProcStructLock);
 
        MyProc->pid = MyProcPid;
+       MyProc->databaseId = MyDatabaseId;
        MyProc->xid = InvalidTransactionId;
-#ifdef LowLevelLocking
        MyProc->xmin = InvalidTransactionId;
-#endif
 
        /* ----------------
         * Start keeping spin lock stats from here on.  Any botch before
@@ -270,14 +302,47 @@ InitProcess(IPCKey key)
         */
        location = MAKE_OFFSET(MyProc);
        if ((!ShmemPIDLookup(MyProcPid, &location)) || (location != MAKE_OFFSET(MyProc)))
-               elog(FATAL, "InitProc: ShmemPID table broken");
+               elog(STOP, "InitProc: ShmemPID table broken");
 
        MyProc->errType = NO_ERROR;
        SHMQueueElemInit(&(MyProc->links));
 
-       on_shmem_exit(ProcKill, (caddr_t) MyProcPid);
+       on_shmem_exit(ProcKill, (Datum) MyProcPid);
+}
 
-       ProcInitialized = TRUE;
+/* -----------------------
+ * get off the wait queue
+ * -----------------------
+ */
+static bool
+GetOffWaitqueue(PROC *proc)
+{
+       bool            getoffed = false;
+
+       LockLockTable();
+       if (proc->links.next != INVALID_OFFSET)
+       {
+               int                     lockmode = proc->token;
+               LOCK    *waitLock = proc->waitLock;
+
+               Assert(waitLock);
+               Assert(waitLock->waitProcs.size > 0);
+               SHMQueueDelete(&(proc->links));
+               --waitLock->waitProcs.size;
+               Assert(waitLock->nHolding > 0);
+               Assert(waitLock->nHolding > proc->waitLock->nActive);
+               --waitLock->nHolding;
+               Assert(waitLock->holders[lockmode] > 0);
+               --waitLock->holders[lockmode];
+               if (waitLock->activeHolders[lockmode] == waitLock->holders[lockmode])
+                       waitLock->waitMask &= ~(1 << lockmode);
+               ProcLockWakeup(&(waitLock->waitProcs), LOCK_LOCKMETHOD(*waitLock), waitLock);
+               getoffed = true;
+       }
+       SHMQueueElemInit(&(proc->links));
+       UnlockLockTable();
+
+       return getoffed;
 }
 
 /*
@@ -290,6 +355,7 @@ ProcReleaseLocks()
        if (!MyProc)
                return;
        LockReleaseAll(1, &MyProc->lockQueue);
+       GetOffWaitqueue(MyProc);
 }
 
 /*
@@ -373,15 +439,7 @@ ProcKill(int exitStatus, int pid)
         * get off the wait queue
         * ----------------
         */
-       LockLockTable();
-       if (proc->links.next != INVALID_OFFSET)
-       {
-               Assert(proc->waitLock->waitProcs.size > 0);
-               SHMQueueDelete(&(proc->links));
-               --proc->waitLock->waitProcs.size;
-       }
-       SHMQueueElemInit(&(proc->links));
-       UnlockLockTable();
+       GetOffWaitqueue(proc);
 
        return;
 }
@@ -403,7 +461,7 @@ ProcQueueAlloc(char *name)
 {
        bool            found;
        PROC_QUEUE *queue = (PROC_QUEUE *)
-       ShmemInitStruct(name, (unsigned) sizeof(PROC_QUEUE), &found);
+               ShmemInitStruct(name, sizeof(PROC_QUEUE), &found);
 
        if (!queue)
                return NULL;
@@ -425,6 +483,50 @@ 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 lock");
+                       }
+               }
+       }
+}
+void
+LockWaitCancel(void)
+{
+       struct itimerval timeval,
+                               dummy;
+
+       if (!lockWaiting)
+               return;
+       lockWaiting = false;
+       /* Deadlock timer off */
+       MemSet(&timeval, 0, sizeof(struct itimerval));
+       setitimer(ITIMER_REAL, &timeval, &dummy);
+       if (GetOffWaitqueue(MyProc))
+               elog(ERROR, "Query cancel requested while waiting lock");
+}
 
 /*
  * ProcSleep -- put a process to sleep
@@ -440,68 +542,79 @@ ProcQueueInit(PROC_QUEUE *queue)
  */
 int
 ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
-                 SPINLOCK spinlock,
+                 LOCKMETHODCTL *lockctl,
                  int token,                    /* lockmode */
-                 int prio,
-                 LOCK *lock,
-                 TransactionId xid)    /* needed by user locks, see below */
+                 LOCK *lock)
 {
        int                     i;
-       bool            deadlock_checked = false;
+       SPINLOCK        spinlock = lockctl->masterLock;
        PROC       *proc;
-       struct timeval timeval;
+       int                     myMask = (1 << token);
+       int                     waitMask = lock->waitMask;
+       int                     aheadHolders[MAX_LOCKMODES];
+       bool            selfConflict = (lockctl->conflictTab[token] & myMask),
+                               prevSame = false;
+       bool            deadlock_checked = false;
+       struct itimerval timeval,
+                               dummy;
+
+       MyProc->token = token;
+       MyProc->waitLock = lock;
 
-       /*
-        * If the first entries in the waitQueue have a greater priority than
-        * we have, we must be a reader, and they must be a writers, and we
-        * must be here because the current holder is a writer or a reader but
-        * we don't share shared locks if a writer is waiting. We put
-        * ourselves after the writers.  This way, we have a FIFO, but keep
-        * the readers together to give them decent priority, and no one
-        * starves.  Because we group all readers together, a non-empty queue
-        * only has a few possible configurations:
-        *
-        * [readers] [writers] [readers][writers] [writers][readers]
-        * [writers][readers][writers]
-        *
-        * In a full queue, we would have a reader holding a lock, then a writer
-        * gets the lock, then a bunch of readers, made up of readers who
-        * could not share the first readlock because a writer was waiting,
-        * and new readers arriving while the writer had the lock.
-        *
-        */
        proc = (PROC *) MAKE_PTR(waitQueue->links.prev);
 
-       /* If we are a reader, and they are writers, skip past them */
-       for (i = 0; i < waitQueue->size && proc->prio > prio; i++)
-               proc = (PROC *) MAKE_PTR(proc->links.prev);
+       /* if we don't conflict with any waiter - be first in queue */
+       if (!(lockctl->conflictTab[token] & waitMask))
+               goto ins;
 
-       /* The rest of the queue is FIFO, with readers first, writers last */
-       for (; i < waitQueue->size && proc->prio <= prio; i++)
-               proc = (PROC *) MAKE_PTR(proc->links.prev);
+       for (i = 1; i < MAX_LOCKMODES; i++)
+               aheadHolders[i] = lock->activeHolders[i];
+       (aheadHolders[token])++;
 
-       MyProc->prio = prio;
-       MyProc->token = token;
-       MyProc->waitLock = lock;
+       for (i = 0; i < waitQueue->size; i++)
+       {
+               /* am I waiting for him ? */
+               if (lockctl->conflictTab[token] & proc->holdLock)
+               {
+                       /* is he waiting for me ? */
+                       if (lockctl->conflictTab[proc->token] & MyProc->holdLock)
+                       {
+                               MyProc->errType = STATUS_ERROR;
+                               elog(NOTICE, DeadLockMessage);
+                               goto rt;
+                       }
+                       /* being waiting for him - go past */
+               }
+               /* if he waits for me */
+               else if (lockctl->conflictTab[proc->token] & MyProc->holdLock)
+                       break;
+               /* if conflicting locks requested */
+               else if (lockctl->conflictTab[proc->token] & myMask)
+               {
 
-#ifdef USER_LOCKS
-       /* -------------------
-        * Currently, we only need this for the ProcWakeup routines.
-        * This must be 0 for user lock, so we can't just use the value
-        * from GetCurrentTransactionId().
-        * -------------------
-        */
-       TransactionIdStore(xid, &MyProc->xid);
-#else
-#ifndef LowLevelLocking
-       /* -------------------
-        * currently, we only need this for the ProcWakeup routines
-        * -------------------
-        */
-       TransactionIdStore((TransactionId) GetCurrentTransactionId(), &MyProc->xid);
-#endif
-#endif
+                       /*
+                        * If I request non self-conflicting lock and there are others
+                        * requesting the same lock just before me - stay here.
+                        */
+                       if (!selfConflict && prevSame)
+                               break;
+               }
 
+               /*
+                * Last attempt to don't move any more: if we don't conflict with
+                * rest waiters in queue.
+                */
+               else if (!(lockctl->conflictTab[token] & waitMask))
+                       break;
+
+               prevSame = (proc->token == token);
+               (aheadHolders[proc->token])++;
+               if (aheadHolders[proc->token] == lock->holders[proc->token])
+                       waitMask &= ~(1 << proc->token);
+               proc = (PROC *) MAKE_PTR(proc->links.prev);
+       }
+
+ins:;
        /* -------------------
         * assume that these two operations are atomic (because
         * of the spinlock).
@@ -510,6 +623,7 @@ ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
        SHMQueueInsertTL(&(proc->links), &(MyProc->links));
        waitQueue->size++;
 
+       lock->waitMask |= myMask;
        SpinRelease(spinlock);
 
        /* --------------
@@ -521,26 +635,20 @@ ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
         * to 0.
         * --------------
         */
-       MemSet(&timeval, 0, sizeof(struct timeval));
-       timeval.tv_sec = \
-               (DeadlockCheckTimer ? DeadlockCheckTimer : DEADLOCK_CHECK_TIMER);
+       MemSet(&timeval, 0, sizeof(struct itimerval));
+       timeval.it_value.tv_sec = DeadlockTimeout / 1000;
+       timeval.it_value.tv_usec = (DeadlockTimeout % 1000) * 1000;
 
+       SetWaitingForLock(true);
        do
        {
-               int expire;
-               
                MyProc->errType = NO_ERROR;             /* reset flag after deadlock check */
 
-               if ((expire = select(0, NULL, NULL, NULL,
-                       (deadlock_checked == false) ? &timeval : NULL)) == -1)
-                       elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
+               if (!deadlock_checked)
+                       if (setitimer(ITIMER_REAL, &timeval, &dummy))
+                               elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
+               deadlock_checked = true;
 
-               if (expire == 0 /* timeout reached */ && deadlock_checked == false)
-               {
-                       HandleDeadLock();
-                       deadlock_checked = true;
-               }
-               
                /* --------------
                 * if someone wakes us between SpinRelease and IpcSemaphoreLock,
                 * IpcSemaphoreLock will not block.  The wakeup is "saved" by
@@ -551,6 +659,16 @@ ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
                                                 IpcExclusiveLock);
        } while (MyProc->errType == STATUS_NOT_FOUND);          /* sleep after deadlock
                                                                                                                 * check */
+       lockWaiting = false;
+
+       /* ---------------
+        * We were awoken before a timeout - now disable the timer
+        * ---------------
+        */
+       timeval.it_value.tv_sec = 0;
+       timeval.it_value.tv_usec = 0;
+       if (setitimer(ITIMER_REAL, &timeval, &dummy))
+               elog(FATAL, "ProcSleep: Unable to diable timer for process wakeup");
 
        /* ----------------
         * We were assumed to be in a critical section when we went
@@ -559,7 +677,9 @@ ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
         */
        SpinAcquire(spinlock);
 
-#ifdef LOCK_MGR_DEBUG
+rt:;
+
+#ifdef LOCK_DEBUG
        /* Just to get meaningful debug messages from DumpLocks() */
        MyProc->waitLock = (LOCK *) NULL;
 #endif
@@ -574,7 +694,7 @@ ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
  *      remove the process from the wait queue and set its links invalid.
  *      RETURN: the next process in the wait queue.
  */
-static PROC *
+PROC *
 ProcWakeup(PROC *proc, int errType)
 {
        PROC       *retProc;
@@ -606,9 +726,8 @@ int
 ProcLockWakeup(PROC_QUEUE *queue, LOCKMETHOD lockmethod, LOCK *lock)
 {
        PROC       *proc;
-       int                     count;
-       int                     trace_flag;
-       int                     last_locktype = -1;
+       int                     count = 0;
+       int                     last_locktype = 0;
        int                     queue_size = queue->size;
 
        Assert(queue->size >= 0);
@@ -617,7 +736,6 @@ ProcLockWakeup(PROC_QUEUE *queue, LOCKMETHOD lockmethod, LOCK *lock)
                return STATUS_NOT_FOUND;
 
        proc = (PROC *) MAKE_PTR(queue->links.prev);
-       count = 0;
        while ((queue_size--) && (proc))
        {
 
@@ -629,7 +747,7 @@ ProcLockWakeup(PROC_QUEUE *queue, LOCKMETHOD lockmethod, LOCK *lock)
                        continue;
 
                /*
-                * This proc conflicts with locks held by others, ignored.
+                * Does this proc conflict with locks held by others ?
                 */
                if (LockResolveConflicts(lockmethod,
                                                                 lock,
@@ -637,6 +755,8 @@ ProcLockWakeup(PROC_QUEUE *queue, LOCKMETHOD lockmethod, LOCK *lock)
                                                                 proc->xid,
                                                                 (XIDLookupEnt *) NULL) != STATUS_OK)
                {
+                       if (count != 0)
+                               break;
                        last_locktype = proc->token;
                        continue;
                }
@@ -666,14 +786,13 @@ ProcLockWakeup(PROC_QUEUE *queue, LOCKMETHOD lockmethod, LOCK *lock)
        else
        {
                /* Something is still blocking us.      May have deadlocked. */
-               trace_flag = (lock->tag.lockmethod == USER_LOCKMETHOD) ? \
-                       TRACE_USERLOCKS : TRACE_LOCKS;
-               TPRINTF(trace_flag,
-                               "ProcLockWakeup: lock(%x) can't wake up any process",
-                               MAKE_OFFSET(lock));
-#ifdef DEADLOCK_DEBUG
-               if (pg_options[trace_flag] >= 2)
+#ifdef LOCK_DEBUG
+               if (lock->tag.lockmethod == USER_LOCKMETHOD ? Trace_userlocks : Trace_locks)
+               {
+                       elog(DEBUG, "ProcLockWakeup: lock(%lx) can't wake up any process", MAKE_OFFSET(lock));
+                       if (Debug_deadlocks)
                        DumpAllLocks();
+               }
 #endif
                return STATUS_NOT_FOUND;
        }
@@ -686,15 +805,15 @@ ProcAddLock(SHM_QUEUE *elem)
 }
 
 /* --------------------
- * We only get to this routine if we got SIGALRM after DEADLOCK_CHECK_TIMER
+ * 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.
  * --------------------
  */
-static void
-HandleDeadLock()
+void
+HandleDeadLock(SIGNAL_ARGS)
 {
        LOCK       *mywaitlock;
 
@@ -735,14 +854,15 @@ HandleDeadLock()
                return;
        }
 
-#ifdef DEADLOCK_DEBUG
-       DumpAllLocks();
+#ifdef LOCK_DEBUG
+    if (Debug_deadlocks)
+        DumpAllLocks();
 #endif
 
-       if (!DeadLockCheck(&(MyProc->lockQueue), MyProc->waitLock, true))
+       MyProc->errType = STATUS_NOT_FOUND;
+       if (!DeadLockCheck(MyProc, MyProc->waitLock))
        {
                UnlockLockTable();
-               MyProc->errType = STATUS_NOT_FOUND;
                return;
        }
 
@@ -753,6 +873,7 @@ HandleDeadLock()
         * ------------------------
         */
        Assert(mywaitlock->waitProcs.size > 0);
+       lockWaiting = false;
        --mywaitlock->waitProcs.size;
        SHMQueueDelete(&(MyProc->links));
        SHMQueueElemInit(&(MyProc->links));
@@ -779,7 +900,7 @@ HandleDeadLock()
         */
        UnlockLockTable();
 
-       elog(NOTICE, "Deadlock detected -- See the lock(l) manual page for a possible cause.");
+       elog(NOTICE, DeadLockMessage);
        return;
 }
 
@@ -801,6 +922,7 @@ ProcReleaseSpins(PROC *proc)
                        SpinRelease(i);
                }
        }
+       AbortBufferIO();
 }
 
 /*****************************************************************************
@@ -819,20 +941,20 @@ ProcGetNewSemKeyAndNum(IPCKey *key, int *semNum)
 {
        int                     i;
        int32      *freeSemMap = ProcGlobal->freeSemMap;
-       unsigned int fullmask;
+       int32           fullmask = (1 << (PROC_NSEMS_PER_SET + 1)) - 1;
 
        /*
         * we hold ProcStructLock when entering this routine. We scan through
         * the bitmap to look for a free semaphore.
         */
-       fullmask = ~0 >> (32 - PROC_NSEMS_PER_SET);
+
        for (i = 0; i < MAX_PROC_SEMS / PROC_NSEMS_PER_SET; i++)
        {
                int                     mask = 1;
                int                     j;
 
                if (freeSemMap[i] == fullmask)
-                       continue;                       /* none free for this set */
+                       continue;                       /* this set is fully allocated */
 
                for (j = 0; j < PROC_NSEMS_PER_SET; j++)
                {
@@ -840,9 +962,10 @@ ProcGetNewSemKeyAndNum(IPCKey *key, int *semNum)
                        {
 
                                /*
-                                * a free semaphore found. Mark it as allocated.
+                                * a free semaphore found. Mark it as allocated. Also set
+                                * the bit indicating whole set is allocated.
                                 */
-                               freeSemMap[i] |= mask;
+                               freeSemMap[i] |= mask + (1 << PROC_NSEMS_PER_SET);
 
                                *key = ProcGlobal->currKey + i;
                                *semNum = j;
@@ -858,8 +981,7 @@ ProcGetNewSemKeyAndNum(IPCKey *key, int *semNum)
 
 /*
  * ProcFreeSem -
- *       free up our semaphore in the semaphore set. If we're the last one
- *       in the set, also remove the semaphore set.
+ *       free up our semaphore in the semaphore set.
  */
 static void
 ProcFreeSem(IpcSemaphoreKey semKey, int semNum)
@@ -872,16 +994,22 @@ ProcFreeSem(IpcSemaphoreKey semKey, int semNum)
        mask = ~(1 << semNum);
        freeSemMap[i] &= mask;
 
-       if (freeSemMap[i] == 0)
-               IpcSemaphoreKill(semKey);
+       /*
+        * Formerly we'd release a semaphore set if it was now completely
+        * unused, but now we keep the semaphores to ensure we won't run out
+        * when starting new backends --- cf. InitProcGlobal.  Note that the
+        * PROC_NSEMS_PER_SET+1'st bit of the freeSemMap entry remains set to
+        * indicate it is still allocated; ProcFreeAllSemaphores() needs that.
+        */
 }
 
 /*
  * ProcFreeAllSemaphores -
- *       on exiting the postmaster, we free up all the semaphores allocated
- *       to the lmgrs of the backends.
+ *       called at shmem_exit time, ie when exiting the postmaster or
+ *       destroying shared state for a failed set of backends.
+ *       Free up all the semaphores allocated to the lmgrs of the backends.
  */
-void
+static void
 ProcFreeAllSemaphores()
 {
        int                     i;