Make the different Unix-y semaphore implementations ABI-compatible.
authorTom Lane <tgl@sss.pgh.pa.us>
Mon, 12 Dec 2016 18:32:10 +0000 (13:32 -0500)
committerTom Lane <tgl@sss.pgh.pa.us>
Mon, 12 Dec 2016 18:32:10 +0000 (13:32 -0500)
Previously, the "sem" field of PGPROC varied in size depending on which
kernel semaphore API we were using.  That was okay as long as there was
only one likely choice per platform, but in the wake of commit ecb0d20a9,
that assumption seems rather shaky.  It doesn't seem out of the question
anymore that an extension compiled against one API choice might be loaded
into a postmaster built with another choice.  Moreover, this prevents any
possibility of selecting the semaphore API at postmaster startup, which
might be something we want to do in future.

Hence, change PGPROC.sem to be PGSemaphore (i.e. a pointer) for all Unix
semaphore APIs, and turn the pointed-to data into an opaque struct whose
contents are only known within the responsible modules.

For the SysV and unnamed-POSIX APIs, the pointed-to data has to be
allocated elsewhere in shared memory, which takes a little bit of
rejiggering of the InitShmemAllocation code sequence.  (I invented a
ShmemAllocUnlocked() function to make that a little cleaner than it used
to be.  That function is not meant for any uses other than the ones it
has now, but it beats having InitShmemAllocation() know explicitly about
allocation of space for semaphores and spinlocks.)  This change means an
extra indirection to access the semaphore data, but since we only touch
that when blocking or awakening a process, there shouldn't be any
meaningful performance penalty.  Moreover, at least for the unnamed-POSIX
case on Linux, the sem_t type is quite a bit wider than a pointer, so this
reduces sizeof(PGPROC) which seems like a good thing.

For the named-POSIX API, there's effectively no change: the PGPROC.sem
field was and still is a pointer to something returned by sem_open() in
the postmaster's memory space.  Document and check the pre-existing
limitation that this case can't work in EXEC_BACKEND mode.

It did not seem worth unifying the Windows semaphore ABI with the Unix
cases, since there's no likelihood of needing ABI compatibility much less
runtime switching across those cases.  However, we can simplify the Windows
code a bit if we define PGSemaphore as being directly a HANDLE, rather than
pointer to HANDLE, so let's do that while we're here.  (This also ends up
being no change in what's physically stored in PGPROC.sem.  We're just
moving the HANDLE fetch from callees to callers.)

It would take a bunch of additional code shuffling to get to the point of
actually choosing a semaphore API at postmaster start, but the effects
of that would now be localized in the port/XXX_sema.c files, so it seems
like fit material for a separate patch.  The need for it is unproven as
yet, anyhow, whereas the ABI risk to extensions seems real enough.

Discussion: https://postgr.es/m/4029.1481413370@sss.pgh.pa.us

14 files changed:
src/backend/port/posix_sema.c
src/backend/port/sysv_sema.c
src/backend/port/win32_sema.c
src/backend/postmaster/postmaster.c
src/backend/storage/ipc/ipci.c
src/backend/storage/ipc/procarray.c
src/backend/storage/ipc/shmem.c
src/backend/storage/lmgr/lwlock.c
src/backend/storage/lmgr/proc.c
src/backend/storage/lmgr/spin.c
src/include/storage/pg_sema.h
src/include/storage/proc.h
src/include/storage/shmem.h
src/include/storage/spin.h

index 2b4b11ce4e5234cec9a943b38ef5aa23af09b117..603dc5a41c2e3755e436a2fa4fc1ff0203e00f4a 100644 (file)
@@ -6,6 +6,14 @@
  * We prefer the unnamed style of POSIX semaphore (the kind made with
  * sem_init).  We can cope with the kind made with sem_open, however.
  *
+ * In either implementation, typedef PGSemaphore is equivalent to "sem_t *".
+ * With unnamed semaphores, the sem_t structs live in an array in shared
+ * memory.  With named semaphores, that's not true because we cannot persuade
+ * sem_open to do its allocation there.  Therefore, the named-semaphore code
+ * *does not cope with EXEC_BACKEND*.  The sem_t structs will just be in the
+ * postmaster's private memory, where they are successfully inherited by
+ * forked backends, but they could not be accessed by exec'd backends.
+ *
  *
  * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
 #include "postgres.h"
 
 #include <fcntl.h>
+#include <semaphore.h>
 #include <signal.h>
 #include <unistd.h>
 
 #include "miscadmin.h"
 #include "storage/ipc.h"
 #include "storage/pg_sema.h"
+#include "storage/shmem.h"
 
 
-#ifdef USE_NAMED_POSIX_SEMAPHORES
-/* PGSemaphore is pointer to pointer to sem_t */
-#define PG_SEM_REF(x)  (*(x))
-#else
-/* PGSemaphore is pointer to sem_t */
-#define PG_SEM_REF(x)  (x)
+/* see file header comment */
+#if defined(USE_NAMED_POSIX_SEMAPHORES) && defined(EXEC_BACKEND)
+#error cannot use named POSIX semaphores with EXEC_BACKEND
 #endif
 
+/* typedef PGSemaphore is equivalent to pointer to sem_t */
+typedef struct PGSemaphoreData
+{
+   sem_t       pgsem;
+} PGSemaphoreData;
+
+#define PG_SEM_REF(x)  (&(x)->pgsem)
 
 #define IPCProtection  (0600)  /* access/modify by user only */
 
+#ifdef USE_NAMED_POSIX_SEMAPHORES
 static sem_t **mySemPointers;  /* keep track of created semaphores */
+#else
+static PGSemaphore sharedSemas; /* array of PGSemaphoreData in shared memory */
+#endif
 static int numSems;            /* number of semas acquired so far */
-static int maxSems;            /* allocated size of mySemaPointers array */
+static int maxSems;            /* allocated size of above arrays */
 static int nextSemKey;         /* next name to try */
 
 
@@ -133,6 +151,21 @@ PosixSemaphoreKill(sem_t * sem)
 }
 
 
+/*
+ * Report amount of shared memory needed for semaphores
+ */
+Size
+PGSemaphoreShmemSize(int maxSemas)
+{
+#ifdef USE_NAMED_POSIX_SEMAPHORES
+   /* No shared memory needed in this case */
+   return 0;
+#else
+   /* Need a PGSemaphoreData per semaphore */
+   return mul_size(maxSemas, sizeof(PGSemaphoreData));
+#endif
+}
+
 /*
  * PGReserveSemaphores --- initialize semaphore support
  *
@@ -147,15 +180,33 @@ PosixSemaphoreKill(sem_t * sem)
  * zero will be passed.
  *
  * In the Posix implementation, we acquire semaphores on-demand; the
- * maxSemas parameter is just used to size the array that keeps track of
- * acquired semas for subsequent releasing.
+ * maxSemas parameter is just used to size the arrays.  For unnamed
+ * semaphores, there is an array of PGSemaphoreData structs in shared memory.
+ * For named semaphores, we keep a postmaster-local array of sem_t pointers,
+ * which we use for releasing the semphores when done.
+ * (This design minimizes the dependency of postmaster shutdown on the
+ * contents of shared memory, which a failed backend might have clobbered.
+ * We can't do much about the possibility of sem_destroy() crashing, but
+ * we don't have to expose the counters to other processes.)
  */
 void
 PGReserveSemaphores(int maxSemas, int port)
 {
+#ifdef USE_NAMED_POSIX_SEMAPHORES
    mySemPointers = (sem_t **) malloc(maxSemas * sizeof(sem_t *));
    if (mySemPointers == NULL)
        elog(PANIC, "out of memory");
+#else
+
+   /*
+    * We must use ShmemAllocUnlocked(), since the spinlock protecting
+    * ShmemAlloc() won't be ready yet.  (This ordering is necessary when we
+    * are emulating spinlocks with semaphores.)
+    */
+   sharedSemas = (PGSemaphore)
+       ShmemAllocUnlocked(PGSemaphoreShmemSize(maxSemas));
+#endif
+
    numSems = 0;
    maxSems = maxSemas;
    nextSemKey = port * 1000;
@@ -173,19 +224,27 @@ ReleaseSemaphores(int status, Datum arg)
 {
    int         i;
 
+#ifdef USE_NAMED_POSIX_SEMAPHORES
    for (i = 0; i < numSems; i++)
        PosixSemaphoreKill(mySemPointers[i]);
    free(mySemPointers);
+#endif
+
+#ifdef USE_UNNAMED_POSIX_SEMAPHORES
+   for (i = 0; i < numSems; i++)
+       PosixSemaphoreKill(PG_SEM_REF(sharedSemas + i));
+#endif
 }
 
 /*
  * PGSemaphoreCreate
  *
- * Initialize a PGSemaphore structure to represent a sema with count 1
+ * Allocate a PGSemaphore structure with initial count 1
  */
-void
-PGSemaphoreCreate(PGSemaphore sema)
+PGSemaphore
+PGSemaphoreCreate(void)
 {
+   PGSemaphore sema;
    sem_t      *newsem;
 
    /* Can't do this in a backend, because static state is postmaster's */
@@ -195,14 +254,19 @@ PGSemaphoreCreate(PGSemaphore sema)
        elog(PANIC, "too many semaphores created");
 
 #ifdef USE_NAMED_POSIX_SEMAPHORES
-   *sema = newsem = PosixSemaphoreCreate();
+   newsem = PosixSemaphoreCreate();
+   /* Remember new sema for ReleaseSemaphores */
+   mySemPointers[numSems] = newsem;
+   sema = (PGSemaphore) newsem;
 #else
-   PosixSemaphoreCreate(sema);
-   newsem = sema;
+   sema = &sharedSemas[numSems];
+   newsem = PG_SEM_REF(sema);
+   PosixSemaphoreCreate(newsem);
 #endif
 
-   /* Remember new sema for ReleaseSemaphores */
-   mySemPointers[numSems++] = newsem;
+   numSems++;
+
+   return sema;
 }
 
 /*
index f6f15169200a03e9da46ae348994f04297d22017..531d4265504f03416cbdd0c06ed58ea799a3c914 100644 (file)
 #include "miscadmin.h"
 #include "storage/ipc.h"
 #include "storage/pg_sema.h"
+#include "storage/shmem.h"
 
 
+typedef struct PGSemaphoreData
+{
+   int         semId;          /* semaphore set identifier */
+   int         semNum;         /* semaphore number within set */
+} PGSemaphoreData;
+
 #ifndef HAVE_UNION_SEMUN
 union semun
 {
@@ -54,6 +61,9 @@ typedef int IpcSemaphoreId;       /* semaphore ID returned by semget(2) */
 #define PGSemaMagic        537     /* must be less than SEMVMX */
 
 
+static PGSemaphore sharedSemas; /* array of PGSemaphoreData in shared memory */
+static int numSharedSemas;     /* number of PGSemaphoreDatas used so far */
+static int maxSharedSemas;     /* allocated size of PGSemaphoreData array */
 static IpcSemaphoreId *mySemaSets;     /* IDs of sema sets acquired so far */
 static int numSemaSets;        /* number of sema sets acquired so far */
 static int maxSemaSets;        /* allocated size of mySemaSets array */
@@ -273,6 +283,15 @@ IpcSemaphoreCreate(int numSems)
 }
 
 
+/*
+ * Report amount of shared memory needed for semaphores
+ */
+Size
+PGSemaphoreShmemSize(int maxSemas)
+{
+   return mul_size(maxSemas, sizeof(PGSemaphoreData));
+}
+
 /*
  * PGReserveSemaphores --- initialize semaphore support
  *
@@ -287,12 +306,26 @@ IpcSemaphoreCreate(int numSems)
  * zero will be passed.
  *
  * In the SysV implementation, we acquire semaphore sets on-demand; the
- * maxSemas parameter is just used to size the array that keeps track of
- * acquired sets for subsequent releasing.
+ * maxSemas parameter is just used to size the arrays.  There is an array
+ * of PGSemaphoreData structs in shared memory, and a postmaster-local array
+ * with one entry per SysV semaphore set, which we use for releasing the
+ * semaphore sets when done.  (This design ensures that postmaster shutdown
+ * doesn't rely on the contents of shared memory, which a failed backend might
+ * have clobbered.)
  */
 void
 PGReserveSemaphores(int maxSemas, int port)
 {
+   /*
+    * We must use ShmemAllocUnlocked(), since the spinlock protecting
+    * ShmemAlloc() won't be ready yet.  (This ordering is necessary when we
+    * are emulating spinlocks with semaphores.)
+    */
+   sharedSemas = (PGSemaphore)
+       ShmemAllocUnlocked(PGSemaphoreShmemSize(maxSemas));
+   numSharedSemas = 0;
+   maxSharedSemas = maxSemas;
+
    maxSemaSets = (maxSemas + SEMAS_PER_SET - 1) / SEMAS_PER_SET;
    mySemaSets = (IpcSemaphoreId *)
        malloc(maxSemaSets * sizeof(IpcSemaphoreId));
@@ -323,11 +356,13 @@ ReleaseSemaphores(int status, Datum arg)
 /*
  * PGSemaphoreCreate
  *
- * Initialize a PGSemaphore structure to represent a sema with count 1
+ * Allocate a PGSemaphore structure with initial count 1
  */
-void
-PGSemaphoreCreate(PGSemaphore sema)
+PGSemaphore
+PGSemaphoreCreate(void)
 {
+   PGSemaphore sema;
+
    /* Can't do this in a backend, because static state is postmaster's */
    Assert(!IsUnderPostmaster);
 
@@ -340,11 +375,17 @@ PGSemaphoreCreate(PGSemaphore sema)
        numSemaSets++;
        nextSemaNumber = 0;
    }
+   /* Use the next shared PGSemaphoreData */
+   if (numSharedSemas >= maxSharedSemas)
+       elog(PANIC, "too many semaphores created");
+   sema = &sharedSemas[numSharedSemas++];
    /* Assign the next free semaphore in the current set */
    sema->semId = mySemaSets[numSemaSets - 1];
    sema->semNum = nextSemaNumber++;
    /* Initialize it to count 1 */
    IpcSemaphoreInitialize(sema->semId, sema->semNum, 1);
+
+   return sema;
 }
 
 /*
index c688210e241d1509af37d950f383f85ec76e8179..c8b12bedbee844f1dde70462920dc043b36f4ad4 100644 (file)
@@ -23,6 +23,17 @@ static int   maxSems;            /* allocated size of mySemaSet array */
 
 static void ReleaseSemaphores(int code, Datum arg);
 
+
+/*
+ * Report amount of shared memory needed for semaphores
+ */
+Size
+PGSemaphoreShmemSize(int maxSemas)
+{
+   /* No shared memory needed on Windows */
+   return 0;
+}
+
 /*
  * PGReserveSemaphores --- initialize semaphore support
  *
@@ -62,10 +73,10 @@ ReleaseSemaphores(int code, Datum arg)
 /*
  * PGSemaphoreCreate
  *
- * Initialize a PGSemaphore structure to represent a sema with count 1
+ * Allocate a PGSemaphore structure with initial count 1
  */
-void
-PGSemaphoreCreate(PGSemaphore sema)
+PGSemaphore
+PGSemaphoreCreate(void)
 {
    HANDLE      cur_handle;
    SECURITY_ATTRIBUTES sec_attrs;
@@ -86,12 +97,14 @@ PGSemaphoreCreate(PGSemaphore sema)
    if (cur_handle)
    {
        /* Successfully done */
-       *sema = cur_handle;
        mySemSet[numSems++] = cur_handle;
    }
    else
        ereport(PANIC,
-               (errmsg("could not create semaphore: error code %lu", GetLastError())));
+               (errmsg("could not create semaphore: error code %lu",
+                       GetLastError())));
+
+   return (PGSemaphore) cur_handle;
 }
 
 /*
@@ -106,7 +119,8 @@ PGSemaphoreReset(PGSemaphore sema)
     * There's no direct API for this in Win32, so we have to ratchet the
     * semaphore down to 0 with repeated trylock's.
     */
-   while (PGSemaphoreTryLock(sema));
+   while (PGSemaphoreTryLock(sema))
+        /* loop */ ;
 }
 
 /*
@@ -127,7 +141,7 @@ PGSemaphoreLock(PGSemaphore sema)
     * pending signals are serviced.
     */
    wh[0] = pgwin32_signal_event;
-   wh[1] = *sema;
+   wh[1] = sema;
 
    /*
     * As in other implementations of PGSemaphoreLock, we need to check for
@@ -182,9 +196,10 @@ PGSemaphoreLock(PGSemaphore sema)
 void
 PGSemaphoreUnlock(PGSemaphore sema)
 {
-   if (!ReleaseSemaphore(*sema, 1, NULL))
+   if (!ReleaseSemaphore(sema, 1, NULL))
        ereport(FATAL,
-               (errmsg("could not unlock semaphore: error code %lu", GetLastError())));
+               (errmsg("could not unlock semaphore: error code %lu",
+                       GetLastError())));
 }
 
 /*
@@ -197,7 +212,7 @@ PGSemaphoreTryLock(PGSemaphore sema)
 {
    DWORD       ret;
 
-   ret = WaitForSingleObject(*sema, 0);
+   ret = WaitForSingleObject(sema, 0);
 
    if (ret == WAIT_OBJECT_0)
    {
@@ -213,7 +228,8 @@ PGSemaphoreTryLock(PGSemaphore sema)
 
    /* Otherwise we are in trouble */
    ereport(FATAL,
-   (errmsg("could not try-lock semaphore: error code %lu", GetLastError())));
+           (errmsg("could not try-lock semaphore: error code %lu",
+                   GetLastError())));
 
    /* keep compiler quiet */
    return false;
index 16dc075a3a1abbb31f9c67b70560dee44e12d4be..535f6c4e5a05ad2de860f89ae1a7ed770a3b49b9 100644 (file)
@@ -484,7 +484,7 @@ typedef struct
    VariableCache ShmemVariableCache;
    Backend    *ShmemBackendArray;
 #ifndef HAVE_SPINLOCKS
-   PGSemaphore SpinlockSemaArray;
+   PGSemaphore *SpinlockSemaArray;
 #endif
    int         NamedLWLockTrancheRequests;
    NamedLWLockTranche *NamedLWLockTrancheArray;
index 01bddcea16c4471af840db90deb2d21dbeeb017d..29febb46c474f73fde00e7bf4416a16ea2b7f590 100644 (file)
@@ -102,6 +102,10 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
        Size        size;
        int         numSemas;
 
+       /* Compute number of semaphores we'll need */
+       numSemas = ProcGlobalSemas();
+       numSemas += SpinlockSemas();
+
        /*
         * Size of the Postgres shared-memory block is estimated via
         * moderately-accurate estimates for the big hogs, plus 100K for the
@@ -112,6 +116,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
         * need to be so careful during the actual allocation phase.
         */
        size = 100000;
+       size = add_size(size, PGSemaphoreShmemSize(numSemas));
        size = add_size(size, SpinlockSemaSize());
        size = add_size(size, hash_estimate_size(SHMEM_INDEX_SIZE,
                                                 sizeof(ShmemIndexEnt)));
@@ -166,9 +171,15 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
        /*
         * Create semaphores
         */
-       numSemas = ProcGlobalSemas();
-       numSemas += SpinlockSemas();
        PGReserveSemaphores(numSemas, port);
+
+       /*
+        * If spinlocks are disabled, initialize emulation layer (which
+        * depends on semaphores, so the order is important here).
+        */
+#ifndef HAVE_SPINLOCKS
+       SpinlockSemaInit();
+#endif
    }
    else
    {
index e5d487dbb74cc8cd1b29ffc95e251c5a43813cc4..bf38470f01d92204d93d9a9632f0df402b88461e 100644 (file)
@@ -522,7 +522,7 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
        for (;;)
        {
            /* acts as a read barrier */
-           PGSemaphoreLock(&proc->sem);
+           PGSemaphoreLock(proc->sem);
            if (!proc->procArrayGroupMember)
                break;
            extraWaits++;
@@ -532,7 +532,7 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
 
        /* Fix semaphore count for any absorbed wakeups */
        while (extraWaits-- > 0)
-           PGSemaphoreUnlock(&proc->sem);
+           PGSemaphoreUnlock(proc->sem);
        return;
    }
 
@@ -591,7 +591,7 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
        proc->procArrayGroupMember = false;
 
        if (proc != MyProc)
-           PGSemaphoreUnlock(&proc->sem);
+           PGSemaphoreUnlock(proc->sem);
    }
 }
 
index cc3af2d6156b61d8d708a7df125fe0aa6b2e46e5..a51619407f4824503f300d42f8f925268372fc9f 100644 (file)
@@ -117,36 +117,22 @@ InitShmemAllocation(void)
    Assert(shmhdr != NULL);
 
    /*
-    * If spinlocks are disabled, initialize emulation layer.  We have to do
-    * the space allocation the hard way, since obviously ShmemAlloc can't be
-    * called yet.
+    * Initialize the spinlock used by ShmemAlloc.  We must use
+    * ShmemAllocUnlocked, since obviously ShmemAlloc can't be called yet.
     */
-#ifndef HAVE_SPINLOCKS
-   {
-       PGSemaphore spinsemas;
+   ShmemLock = (slock_t *) ShmemAllocUnlocked(sizeof(slock_t));
 
-       spinsemas = (PGSemaphore) (((char *) shmhdr) + shmhdr->freeoffset);
-       shmhdr->freeoffset += MAXALIGN(SpinlockSemaSize());
-       SpinlockSemaInit(spinsemas);
-       Assert(shmhdr->freeoffset <= shmhdr->totalsize);
-   }
-#endif
+   SpinLockInit(ShmemLock);
 
    /*
-    * Initialize the spinlock used by ShmemAlloc; we have to do this the hard
-    * way, too, for the same reasons as above.
+    * Allocations after this point should go through ShmemAlloc, which
+    * expects to allocate everything on cache line boundaries.  Make sure the
+    * first allocation begins on a cache line boundary.
     */
-   ShmemLock = (slock_t *) (((char *) shmhdr) + shmhdr->freeoffset);
-   shmhdr->freeoffset += MAXALIGN(sizeof(slock_t));
-   Assert(shmhdr->freeoffset <= shmhdr->totalsize);
-
-   /* Make sure the first allocation begins on a cache line boundary. */
    aligned = (char *)
        (CACHELINEALIGN((((char *) shmhdr) + shmhdr->freeoffset)));
    shmhdr->freeoffset = aligned - (char *) shmhdr;
 
-   SpinLockInit(ShmemLock);
-
    /* ShmemIndex can't be set up yet (need LWLocks first) */
    shmhdr->index = NULL;
    ShmemIndex = (HTAB *) NULL;
@@ -229,6 +215,45 @@ ShmemAllocNoError(Size size)
    return newSpace;
 }
 
+/*
+ * ShmemAllocUnlocked -- allocate max-aligned chunk from shared memory
+ *
+ * Allocate space without locking ShmemLock.  This should be used for,
+ * and only for, allocations that must happen before ShmemLock is ready.
+ *
+ * We consider maxalign, rather than cachealign, sufficient here.
+ */
+void *
+ShmemAllocUnlocked(Size size)
+{
+   Size        newStart;
+   Size        newFree;
+   void       *newSpace;
+
+   /*
+    * Ensure allocated space is adequately aligned.
+    */
+   size = MAXALIGN(size);
+
+   Assert(ShmemSegHdr != NULL);
+
+   newStart = ShmemSegHdr->freeoffset;
+
+   newFree = newStart + size;
+   if (newFree > ShmemSegHdr->totalsize)
+       ereport(ERROR,
+               (errcode(ERRCODE_OUT_OF_MEMORY),
+                errmsg("out of shared memory (%zu bytes requested)",
+                       size)));
+   ShmemSegHdr->freeoffset = newFree;
+
+   newSpace = (void *) ((char *) ShmemBase + newStart);
+
+   Assert(newSpace == (void *) MAXALIGN(newSpace));
+
+   return newSpace;
+}
+
 /*
  * ShmemAddrIsValid -- test if an address refers to shared memory
  *
index ffb2f72953c5d915c2ed20c5992a7c2d80539ea8..03c4c7825eb37c070bcee8f8ba03f0d564254b69 100644 (file)
@@ -1012,7 +1012,7 @@ LWLockWakeup(LWLock *lock)
         */
        pg_write_barrier();
        waiter->lwWaiting = false;
-       PGSemaphoreUnlock(&waiter->sem);
+       PGSemaphoreUnlock(waiter->sem);
    }
 }
 
@@ -1129,7 +1129,7 @@ LWLockDequeueSelf(LWLock *lock)
         */
        for (;;)
        {
-           PGSemaphoreLock(&MyProc->sem);
+           PGSemaphoreLock(MyProc->sem);
            if (!MyProc->lwWaiting)
                break;
            extraWaits++;
@@ -1139,7 +1139,7 @@ LWLockDequeueSelf(LWLock *lock)
         * Fix the process wait semaphore's count for any absorbed wakeups.
         */
        while (extraWaits-- > 0)
-           PGSemaphoreUnlock(&MyProc->sem);
+           PGSemaphoreUnlock(MyProc->sem);
    }
 
 #ifdef LOCK_DEBUG
@@ -1283,7 +1283,7 @@ LWLockAcquire(LWLock *lock, LWLockMode mode)
 
        for (;;)
        {
-           PGSemaphoreLock(&proc->sem);
+           PGSemaphoreLock(proc->sem);
            if (!proc->lwWaiting)
                break;
            extraWaits++;
@@ -1320,7 +1320,7 @@ LWLockAcquire(LWLock *lock, LWLockMode mode)
     * Fix the process wait semaphore's count for any absorbed wakeups.
     */
    while (extraWaits-- > 0)
-       PGSemaphoreUnlock(&proc->sem);
+       PGSemaphoreUnlock(proc->sem);
 
    return result;
 }
@@ -1444,7 +1444,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 
            for (;;)
            {
-               PGSemaphoreLock(&proc->sem);
+               PGSemaphoreLock(proc->sem);
                if (!proc->lwWaiting)
                    break;
                extraWaits++;
@@ -1481,7 +1481,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
     * Fix the process wait semaphore's count for any absorbed wakeups.
     */
    while (extraWaits-- > 0)
-       PGSemaphoreUnlock(&proc->sem);
+       PGSemaphoreUnlock(proc->sem);
 
    if (mustwait)
    {
@@ -1662,7 +1662,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 
        for (;;)
        {
-           PGSemaphoreLock(&proc->sem);
+           PGSemaphoreLock(proc->sem);
            if (!proc->lwWaiting)
                break;
            extraWaits++;
@@ -1692,7 +1692,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
     * Fix the process wait semaphore's count for any absorbed wakeups.
     */
    while (extraWaits-- > 0)
-       PGSemaphoreUnlock(&proc->sem);
+       PGSemaphoreUnlock(proc->sem);
 
    /*
     * Now okay to allow cancel/die interrupts.
@@ -1759,7 +1759,7 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
        /* check comment in LWLockWakeup() about this barrier */
        pg_write_barrier();
        waiter->lwWaiting = false;
-       PGSemaphoreUnlock(&waiter->sem);
+       PGSemaphoreUnlock(waiter->sem);
    }
 }
 
index 276261bd7b379f6fbfac9c23d905d34aacb3842f..e9555f2d8fc8e2e58d3d1ec22b11b9b58d199d1a 100644 (file)
@@ -224,7 +224,7 @@ InitProcGlobal(void)
         */
        if (i < MaxBackends + NUM_AUXILIARY_PROCS)
        {
-           PGSemaphoreCreate(&(procs[i].sem));
+           procs[i].sem = PGSemaphoreCreate();
            InitSharedLatch(&(procs[i].procLatch));
            LWLockInitialize(&(procs[i].backendLock), LWTRANCHE_PROC);
        }
@@ -420,7 +420,7 @@ InitProcess(void)
     * be careful and reinitialize its value here.  (This is not strictly
     * necessary anymore, but seems like a good idea for cleanliness.)
     */
-   PGSemaphoreReset(&MyProc->sem);
+   PGSemaphoreReset(MyProc->sem);
 
    /*
     * Arrange to clean up at backend exit.
@@ -575,7 +575,7 @@ InitAuxiliaryProcess(void)
     * be careful and reinitialize its value here.  (This is not strictly
     * necessary anymore, but seems like a good idea for cleanliness.)
     */
-   PGSemaphoreReset(&MyProc->sem);
+   PGSemaphoreReset(MyProc->sem);
 
    /*
     * Arrange to clean up at process exit.
index 50391414305f9d99fb017ebf891f9e5d8cba7414..a6510a07e94052cbe5f099926f48c5311f375007 100644 (file)
 #include "postgres.h"
 
 #include "storage/pg_sema.h"
+#include "storage/shmem.h"
 #include "storage/spin.h"
 
 
 #ifndef HAVE_SPINLOCKS
-PGSemaphore SpinlockSemaArray;
+PGSemaphore *SpinlockSemaArray;
 #endif
 
 /*
@@ -37,7 +38,7 @@ PGSemaphore SpinlockSemaArray;
 Size
 SpinlockSemaSize(void)
 {
-   return SpinlockSemas() * sizeof(PGSemaphoreData);
+   return SpinlockSemas() * sizeof(PGSemaphore);
 }
 
 #ifdef HAVE_SPINLOCKS
@@ -67,16 +68,24 @@ SpinlockSemas(void)
 }
 
 /*
- * Initialize semaphores.
+ * Initialize spinlock emulation.
+ *
+ * This must be called after PGReserveSemaphores().
  */
-extern void
-SpinlockSemaInit(PGSemaphore spinsemas)
+void
+SpinlockSemaInit(void)
 {
-   int         i;
+   PGSemaphore *spinsemas;
    int         nsemas = SpinlockSemas();
+   int         i;
 
+   /*
+    * We must use ShmemAllocUnlocked(), since the spinlock protecting
+    * ShmemAlloc() obviously can't be ready yet.
+    */
+   spinsemas = (PGSemaphore *) ShmemAllocUnlocked(SpinlockSemaSize());
    for (i = 0; i < nsemas; ++i)
-       PGSemaphoreCreate(&spinsemas[i]);
+       spinsemas[i] = PGSemaphoreCreate();
    SpinlockSemaArray = spinsemas;
 }
 
@@ -109,7 +118,7 @@ s_unlock_sema(volatile slock_t *lock)
 
    if (lockndx <= 0 || lockndx > NUM_SPINLOCK_SEMAPHORES)
        elog(ERROR, "invalid spinlock number: %d", lockndx);
-   PGSemaphoreUnlock(&SpinlockSemaArray[lockndx - 1]);
+   PGSemaphoreUnlock(SpinlockSemaArray[lockndx - 1]);
 }
 
 bool
@@ -128,7 +137,7 @@ tas_sema(volatile slock_t *lock)
    if (lockndx <= 0 || lockndx > NUM_SPINLOCK_SEMAPHORES)
        elog(ERROR, "invalid spinlock number: %d", lockndx);
    /* Note that TAS macros return 0 if *success* */
-   return !PGSemaphoreTryLock(&SpinlockSemaArray[lockndx - 1]);
+   return !PGSemaphoreTryLock(SpinlockSemaArray[lockndx - 1]);
 }
 
 #endif   /* !HAVE_SPINLOCKS */
index 2c9418320e6dba2e2207f7d8bd66100b4afdf6ab..63546ebc96a854f2f0fd916663a374f44ec376d4 100644 (file)
 #define PG_SEMA_H
 
 /*
- * PGSemaphoreData and pointer type PGSemaphore are the data structure
- * representing an individual semaphore.  The contents of PGSemaphoreData
- * vary across implementations and must never be touched by platform-
- * independent code.  PGSemaphoreData structures are always allocated
- * in shared memory (to support implementations where the data changes during
- * lock/unlock).
+ * struct PGSemaphoreData and pointer type PGSemaphore are the data structure
+ * representing an individual semaphore.  The contents of PGSemaphoreData vary
+ * across implementations and must never be touched by platform-independent
+ * code; hence, PGSemaphoreData is declared as an opaque struct here.
  *
- * pg_config.h must define exactly one of the USE_xxx_SEMAPHORES symbols.
+ * However, Windows is sufficiently unlike our other ports that it doesn't
+ * seem worth insisting on ABI compatibility for Windows too.  Hence, on
+ * that platform just define PGSemaphore as HANDLE.
  */
-
-#ifdef USE_NAMED_POSIX_SEMAPHORES
-
-#include <semaphore.h>
-
-typedef sem_t *PGSemaphoreData;
-#endif
-
-#ifdef USE_UNNAMED_POSIX_SEMAPHORES
-
-#include <semaphore.h>
-
-typedef sem_t PGSemaphoreData;
-#endif
-
-#ifdef USE_SYSV_SEMAPHORES
-
-typedef struct PGSemaphoreData
-{
-   int         semId;          /* semaphore set identifier */
-   int         semNum;         /* semaphore number within set */
-} PGSemaphoreData;
-#endif
-
-#ifdef USE_WIN32_SEMAPHORES
-
-typedef HANDLE PGSemaphoreData;
+#ifndef USE_WIN32_SEMAPHORES
+typedef struct PGSemaphoreData *PGSemaphore;
+#else
+typedef HANDLE PGSemaphore;
 #endif
 
-typedef PGSemaphoreData *PGSemaphore;
 
+/* Report amount of shared memory needed */
+extern Size PGSemaphoreShmemSize(int maxSemas);
 
 /* Module initialization (called during postmaster start or shmem reinit) */
 extern void PGReserveSemaphores(int maxSemas, int port);
 
-/* Initialize a PGSemaphore structure to represent a sema with count 1 */
-extern void PGSemaphoreCreate(PGSemaphore sema);
+/* Allocate a PGSemaphore structure with initial count 1 */
+extern PGSemaphore PGSemaphoreCreate(void);
 
 /* Reset a previously-initialized PGSemaphore to have count 0 */
 extern void PGSemaphoreReset(PGSemaphore sema);
index 6fa71253d86d781db145ba88fbcfa49404fb6fc8..0344f4277c37914ef356ab56dccb0922257192fb 100644 (file)
@@ -87,7 +87,7 @@ struct PGPROC
    SHM_QUEUE   links;          /* list link if process is in a list */
    PGPROC    **procgloballist; /* procglobal list that owns this PGPROC */
 
-   PGSemaphoreData sem;        /* ONE semaphore to sleep on */
+   PGSemaphore sem;            /* ONE semaphore to sleep on */
    int         waitStatus;     /* STATUS_WAITING, STATUS_OK or STATUS_ERROR */
 
    Latch       procLatch;      /* generic latch for process */
@@ -116,7 +116,7 @@ struct PGPROC
    proclist_node lwWaitLink;   /* position in LW lock wait list */
 
    /* Support for condition variables. */
-   proclist_node   cvWaitLink; /* position in CV wait list */
+   proclist_node cvWaitLink;   /* position in CV wait list */
 
    /* Info about lock the process is currently waiting for, if any. */
    /* waitLock and waitProcLock are NULL if not currently waiting. */
index 2560e6c6da5764c3a118ba278568b0b6022d27da..e4faebf2b63aff402a0482390629f9e1fab04b40 100644 (file)
@@ -36,6 +36,7 @@ extern void InitShmemAccess(void *seghdr);
 extern void InitShmemAllocation(void);
 extern void *ShmemAlloc(Size size);
 extern void *ShmemAllocNoError(Size size);
+extern void *ShmemAllocUnlocked(Size size);
 extern bool ShmemAddrIsValid(const void *addr);
 extern void InitShmemIndex(void);
 extern HTAB *ShmemInitHash(const char *name, long init_size, long max_size,
index 50412258b9e5944ae9e5dc58cb8f148f704aac5a..b95c9bc33342fe27584ca0a2074575dc7bd40685 100644 (file)
@@ -70,8 +70,8 @@ extern int    SpinlockSemas(void);
 extern Size SpinlockSemaSize(void);
 
 #ifndef HAVE_SPINLOCKS
-extern void SpinlockSemaInit(PGSemaphore);
-extern PGSemaphore SpinlockSemaArray;
+extern void SpinlockSemaInit(void);
+extern PGSemaphore *SpinlockSemaArray;
 #endif
 
 #endif   /* SPIN_H */