Wait free LW_SHARED acquisition

Started by Andres Freundover 12 years ago117 messages
#1Andres Freund
andres@2ndquadrant.com
1 attachment(s)

Hello,

We have had several customers running postgres on bigger machines report
problems on busy systems. Most recently one where a fully cached
workload completely stalled in s_lock()s due to the *shared* lwlock
acquisition in BufferAlloc() around the buffer partition lock.

Increasing the padding to a full cacheline helps making the partitioning
of the partition space actually effective (before it's essentially
halved on a read-mostly workload), but that still leaves one with very
hot spinlocks.

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

That's rougly 400%.

So, nice little improvement. Unless - not entirely unlikely - I fucked
up and it's fast because it's broken.

Anyway, here's the algorith I chose to implement:
The basic idea is to have a single 'uint32 lockcount' instead of the
former 'char exclusive' and 'int shared' and to use an atomic increment
to acquire the lock. That's fairly easy to do for rw-spinlocks, but a
lot harder for something like LWLocks that want to wait in the OS.

Exlusive lock acquisition:

Use an atomic compare-and-exchange on the lockcount variable swapping in
EXCLUSIVE_LOCK/1<<31/0x80000000 if and only if the current value of
lockcount is 0. If the swap was not successfull, we have to wait.

Shared lock acquisition:

Use an atomic add (lock xadd) to the lockcount variable to add 1. If the
value is bigger than EXCLUSIVE_LOCK we know that somebody actually has
an exclusive lock, and we back out by atomically decrementing by 1
again.
If so, we have to wait for the exlusive locker to release the lock.

Queueing & Wakeup:

Whenever we don't get a shared/exclusive lock we us nearly the same
queuing mechanism as we currently do. While we probably could make it
lockless as well, the queue currently is still protected by the good old
spinlock.

Relase:

Use a atomic decrement to release the lock. If the new value is zero (we
get that atomically), we know we have to release waiters.

And the real world:

Now, as you probably have noticed, naively doing the above has two
glaring race conditions:

1) too-quick-for-queueing:
We try to lock using the atomic operations and notice that we have to
wait. Unfortunately until we have finished queuing, the former locker
very well might have already finished it's work.

2) spurious failed locks:
Due to the logic of backing out of shared locks after we unconditionally
added a 1 to lockcount, we might have prevented another exclusive locker
from getting the lock:
1) Session A: LWLockAcquire(LW_EXCLUSIVE) - success
2) Session B: LWLockAcquire(LW_SHARED) - lockcount += 1
3) Session B: LWLockAcquire(LW_SHARED) - oops, bigger than EXCLUSIVE_LOCK
4) Session B: LWLockRelease()
5) Session C: LWLockAcquire(LW_EXCLUSIVE) - check if lockcount = 0, no. wait.
6) Session B: LWLockAcquire(LW_SHARED) - lockcount -= 1
7) Session B: LWLockAcquire(LW_SHARED) - wait

So now we can have both B) and C) waiting on a lock that nobody is
holding anymore. Not good.

The solution:
We use a two phased attempt at locking:
Phase 1: Try to do it atomically, if we succeed, nice
Phase 2: Add us too the waitqueue of the lock
Phase 3: Try to grab the lock again, if we succeed, remove ourselves
from the queue
Phase 4: Sleep till wakeup, goto Phase 1

This protects us against both problems from above:
1) Nobody can release too quick, before we're queued, after Phase 2 since we're already
queued.
2) If somebody spuriously got blocked from acquiring the lock, they will
get queued in Phase 2 and we can wake them up if neccessary.

Now, there are lots of tiny details to handle additionally to those, but
those seem better handled by looking at the code?
- The above algorithm only works for LWLockAcquire, not directly for
LWLockAcquireConditional where we don't want to wait. In that case we
just need to retry acquiring the lock until we're sure we didn't
disturb anybody in doing so.
- we can get removed from the queue of waiters in Phase 3, before we remove
ourselves. In that case we need to absorb the wakeup.
- Spurious locks can prevent us from recognizing a lock that's free
during release. Solve it by checking for existing waiters whenever an
exlusive lock is released.

I've done a couple of off-hand benchmarks and so far I can confirm that
everything using lots of shared locks benefits greatly and everything
else doesn't really change much. So far I've seen mostly some slight
improvements in exclusive lock heavy workloads, but those were pretty
small.
It's also very important to mention that those speedups are only there
on multi-socket machines. From what I've benchmarked so far in LW_SHARED
heavy workloads with 1 socket you get ~5-10%, 2 sockets 20-30% and
finally and nicely for 4 sockets: 350-400%.
While I did assume the difference would be bigger on 4 socket machines
than on my older 2 socket workstation (that's where the 20-30% come
from) I have to admit, I was surprised by the difference on the 4 socket
machine.

Does anybody see fundamental problems with the algorithm? The
implementation sure isn't ready for several reasons, but I don't want to
go ahead and spend lots of time on it before hearing some more voices.

So what's todo? The file header tells us:
* - revive pure-spinlock implementation
* - abstract away atomic ops, we really only need a few.
* - CAS
* - LOCK XADD
* - convert PGPROC->lwWaitLink to ilist.h slist or even dlist.
* - remove LWLockWakeup dealing with MyProc
* - overhaul the mask offsets, make SHARED/EXCLUSIVE_LOCK_MASK wider, MAX_BACKENDS

Currently only gcc is supported because I used its
__sync_fetch_and_add(), __sync_fetch_and_sub() and
__sync_val_compare_and_swap() are used. There have been reports about
__sync_fetch_and_sub() not getting properly optimized with gcc < 4.8,
perhaps we need to replace it by _and_add(-val). Given the low amount of
primitives required, it should be adaptable to most newer compilers.

Comments? Fundamental flaws? 8 socket machines?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v01-Wait-free-LW_SHARED-lwlock-acquiration.patchtext/x-patch; charset=us-asciiDownload
>From 8189689b71e5cd8129587345aa6e7b6e0231c6f8 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Tue, 24 Sep 2013 03:04:44 +0200
Subject: [PATCH] Wait free LW_SHARED lwlock acquiration

---
 src/backend/access/transam/xlog.c |   9 +-
 src/backend/storage/lmgr/lwlock.c | 863 +++++++++++++++++++++++++++-----------
 2 files changed, 617 insertions(+), 255 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index fc495d6..07e510b 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -1620,7 +1620,8 @@ WALInsertSlotAcquireOne(int slotno)
 			break;				/* got the lock */
 
 		Assert(slot->owner != MyProc);
-
+		Assert(!proc->lwWaiting);
+		Assert(proc->lwWaitLink == NULL);
 		/*
 		 * Add myself to wait queue.
 		 */
@@ -1630,7 +1631,10 @@ WALInsertSlotAcquireOne(int slotno)
 		if (slot->head == NULL)
 			slot->head = proc;
 		else
+		{
+			Assert(slot->tail->lwWaitLink == NULL);
 			slot->tail->lwWaitLink = proc;
+		}
 		slot->tail = proc;
 
 		/* Can release the mutex now */
@@ -1741,6 +1745,9 @@ WaitOnSlot(volatile XLogInsertSlot *slot, XLogRecPtr waitptr)
 
 		Assert(slot->owner != MyProc);
 
+		Assert(!proc->lwWaiting);
+		Assert(proc->lwWaitLink == NULL);
+
 		/*
 		 * Add myself to wait queue.
 		 */
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 4f88d3f..c7fffec 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -17,6 +17,14 @@
  * IDENTIFICATION
  *	  src/backend/storage/lmgr/lwlock.c
  *
+ * TODO:
+ * - convert lwWaitLink to ilist.h slist or even dlist.
+ * - remove LWLockWakeup dealing with MyProc
+ * - revive pure-spinlock implementation
+ * - abstract away atomic ops, we really only need a few.
+ *   - CAS
+ *   - LOCK XADD
+ * - overhaul the mask offsets, make SHARED/EXCLUSIVE_LOCK_MASK wider, MAX_BACKENDS
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -27,22 +35,29 @@
 #include "commands/async.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
+#include "storage/barrier.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/spin.h"
 
-
 /* We use the ShmemLock spinlock to protect LWLockAssign */
 extern slock_t *ShmemLock;
 
+#define EXCLUSIVE_LOCK_FINALIZED ((uint32)1<<31)
+#define EXCLUSIVE_LOCK_MASK (~(uint32)((1<<16)-1))
+/* XXX: must be bigger than MAX_BACKENDS */
+#define SHARED_LOCK_MASK ((uint32)((1<<16)-1))
 
 typedef struct LWLock
 {
 	slock_t		mutex;			/* Protects LWLock and queue of PGPROCs */
+	uint32		lockcount;		/* magic */
 	bool		releaseOK;		/* T if ok to release waiters */
-	char		exclusive;		/* # of exclusive holders (0 or 1) */
-	int			shared;			/* # of shared holders (0..MaxBackends) */
+	uint32		waiters;
+#ifdef LWLOCK_DEBUG
+	PGPROC	   *owner;
+#endif
 	PGPROC	   *head;			/* head of list of waiting PGPROCs */
 	PGPROC	   *tail;			/* tail of list of waiting PGPROCs */
 	/* tail is undefined when head is NULL */
@@ -60,7 +75,7 @@ typedef struct LWLock
  * LWLock is between 16 and 32 bytes on all known platforms, so these two
  * cases are sufficient.
  */
-#define LWLOCK_PADDED_SIZE	(sizeof(LWLock) <= 16 ? 16 : 32)
+#define LWLOCK_PADDED_SIZE	64
 
 typedef union LWLockPadded
 {
@@ -75,7 +90,6 @@ typedef union LWLockPadded
  */
 NON_EXEC_STATIC LWLockPadded *LWLockArray = NULL;
 
-
 /*
  * We use this structure to keep track of locked LWLocks for release
  * during error recovery.  The maximum size could be determined at runtime
@@ -84,12 +98,21 @@ NON_EXEC_STATIC LWLockPadded *LWLockArray = NULL;
  */
 #define MAX_SIMUL_LWLOCKS	100
 
+struct LWLockHandle
+{
+	LWLockId lock;
+	LWLockMode mode;
+};
+
 static int	num_held_lwlocks = 0;
-static LWLockId held_lwlocks[MAX_SIMUL_LWLOCKS];
+static struct LWLockHandle held_lwlocks[MAX_SIMUL_LWLOCKS];
 
 static int	lock_addin_request = 0;
 static bool lock_addin_request_allowed = true;
 
+/* read value from memory, does *not* have any barrier semantics */
+#define pg_atomic_read(atomic)	(*(volatile uint32 *)&(atomic))
+
 #ifdef LWLOCK_STATS
 static int	counts_for_pid = 0;
 static int *sh_acquire_counts;
@@ -102,24 +125,27 @@ static int *spin_delay_counts;
 bool		Trace_lwlocks = false;
 
 inline static void
-PRINT_LWDEBUG(const char *where, LWLockId lockid, const volatile LWLock *lock)
+PRINT_LWDEBUG(const char *where, LWLockId lockid, const volatile LWLock *lock, LWLockMode mode)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%d): excl %d shared %d head %p rOK %d",
-			 where, (int) lockid,
-			 (int) lock->exclusive, lock->shared, lock->head,
+		elog(LOG, "%s(%d)%u: excl %u shared %u head %p waiters %u rOK %d",
+		     where, (int) lockid, mode,
+		     (lock->lockcount & EXCLUSIVE_LOCK_MASK) >> 31,
+		     lock->lockcount & SHARED_LOCK_MASK,
+		     lock->head,
+		     lock->waiters,
 			 (int) lock->releaseOK);
 }
 
 inline static void
-LOG_LWDEBUG(const char *where, LWLockId lockid, const char *msg)
+LOG_LWDEBUG(const char *where, LWLockId lockid, LWLockMode mode, const char *msg)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%d): %s", where, (int) lockid, msg);
+		elog(LOG, "%s(%d)%u: %s", where, (int) lockid, mode, msg);
 }
 #else							/* not LOCK_DEBUG */
-#define PRINT_LWDEBUG(a,b,c)
-#define LOG_LWDEBUG(a,b,c)
+#define PRINT_LWDEBUG(a,b,c,d) (void)0
+#define LOG_LWDEBUG(a,b,c,d) (void)0
 #endif   /* LOCK_DEBUG */
 
 #ifdef LWLOCK_STATS
@@ -285,8 +311,8 @@ CreateLWLocks(void)
 	{
 		SpinLockInit(&lock->lock.mutex);
 		lock->lock.releaseOK = true;
-		lock->lock.exclusive = 0;
-		lock->lock.shared = 0;
+		lock->lock.lockcount = 0;
+		lock->lock.waiters = 0;
 		lock->lock.head = NULL;
 		lock->lock.tail = NULL;
 	}
@@ -329,6 +355,364 @@ LWLockAssign(void)
 	return result;
 }
 
+/*
+ * Internal function handling the atomic manipulation of lock->lockcount.
+ *
+ * 'double_check' = true means that we try to check more carefully
+ * against causing somebody else to spuriously believe the lock is
+ * already taken, although we're just about to back out of it.
+ */
+static inline bool
+LWLockAttemptLock(volatile LWLock* lock, LWLockMode mode, bool double_check, bool *potentially_spurious)
+{
+	bool mustwait;
+	uint32 oldstate;
+
+	if (mode == LW_EXCLUSIVE)
+	{
+		pg_read_barrier();
+		/* check without CAS, way cheaper, frequently locked otherwise */
+		if (pg_atomic_read(lock->lockcount) != 0)
+			mustwait = true;
+		/*
+		 * ok, no can do. Between the pg_atomic_read() above and the
+		 * CAS somebody else acquired the lock.
+		 */
+		else if (__sync_val_compare_and_swap(&lock->lockcount,
+		                                     0, EXCLUSIVE_LOCK_FINALIZED) != 0)
+		{
+			mustwait = true;
+		}
+		/* yipeyyahee */
+		else
+		{
+			mustwait = false;
+#ifdef LWLOCK_DEBUG
+			lock->owner = MyProc;
+#endif
+		}
+	}
+	else
+	{
+		/*
+		 * Do an unlocked check first, useful if potentially spurious
+		 * results have a noticeable cost
+		 */
+		if (double_check)
+		{
+			pg_read_barrier();
+			if (pg_atomic_read(lock->lockcount) >= EXCLUSIVE_LOCK_FINALIZED)
+			{
+				mustwait = true;
+				goto out;
+			}
+		}
+
+		/*
+		 * Acquire the share lock unconditionally using an atomic
+		 * addition. We might have to back out again if it turns out
+		 * somebody else has an exclusive lock.
+		 */
+		oldstate = __sync_fetch_and_add(&lock->lockcount, 1);
+
+		/*
+		 * ok, somebody else holds the lock exclusively. We need to
+		 * back away from the shared lock, since we don't actually
+		 * hold it right now. Since there's a window between lockcount
+		 * += 1 and logcount -= 1 the previous exclusive locker could
+		 * have released and another exclusive locker could have seen
+		 * our +1. We need to signal that to the upper layers so they
+		 * can deal with the race condition.
+		 */
+		if (oldstate >= EXCLUSIVE_LOCK_FINALIZED)
+		{
+			/*
+			 * FIXME: check return value if (double_check), it's not
+			 * spurious if still exclusively locked.
+			 */
+			__sync_fetch_and_sub(&lock->lockcount, 1);
+			mustwait = true;
+			*potentially_spurious = true;
+		}
+		/* yipeyyahee */
+		else
+			mustwait = false;
+	}
+
+out:
+	return mustwait;
+}
+
+/*
+ * Wakeup all the lockers that currently have a chance to run.
+ */
+static void
+LWLockWakeup(volatile LWLock* lock, LWLockId lockid, LWLockMode mode)
+{
+	/*
+	 * Remove the to-be-awakened PGPROCs from the queue.
+	 */
+	bool		releaseOK = true;
+	PGPROC	   *cur;
+	PGPROC	   *last = NULL;
+	PGPROC	   *next = NULL;
+	PGPROC	  **wakeup;
+	size_t		nwakeup = 0;
+	size_t		wakeup_slots;
+	bool		woke_up_shared = false;
+	int			i;
+
+	/* Acquire mutex.  Time spent holding mutex should be short! */
+	SpinLockAcquire(&lock->mutex);
+
+	Assert((lock->head != NULL && lock->tail != NULL) ||
+	       lock->head == NULL);
+
+	cur = lock->head;
+
+	if (cur == NULL)
+	{
+#ifdef LOCK_DEBUG
+		/*
+		 * this can legally happen since we increase lock->waiters in
+		 * LWLockQueueLock before queueing, but it can also indicate a
+		 * bug, so it's usefull to display this when tracing.
+		 */
+		if (pg_atomic_read(lock->waiters) > 0)
+		{
+			PRINT_LWDEBUG("LWLockWakeup - no head, but waiters", lockid, lock, 0);
+		}
+#endif
+		/* nothing to do, can't be out of date because of the spinlock */
+		goto unlock;
+	}
+
+	wakeup_slots = pg_atomic_read(lock->waiters);
+	wakeup = alloca(wakeup_slots * sizeof(PGPROC *));
+
+	while (cur != NULL)
+	{
+		/* panic because others will be stuck in a semaphore */
+		if (nwakeup >= wakeup_slots)
+			elog(PANIC, "too many waiters");
+
+		/* wakeup all shared locks we find, unless we've found an exclusive lock first */
+		if (cur != MyProc && cur->lwWaitMode == LW_SHARED)
+		{
+			woke_up_shared = true;
+			wakeup[nwakeup++] = cur;
+			/*
+			 * don't want to wakeup again before any of the woken up
+			 * entries got scheduled.
+			 */
+			releaseOK = false;
+		}
+		/*
+		 * wakeup all UNTIL_FREE entries until we find a shared lock,
+		 * even if there were shared locks before
+		 */
+		else if (cur != MyProc && cur->lwWaitMode == LW_WAIT_UNTIL_FREE)
+		{
+			wakeup[nwakeup++] = cur;
+		}
+
+		/*
+		 * Exlusive lock. Wake this one up, but no further ones since
+		 * they wouldn't be able to acquire the lock anyway until the
+		 * exclusive lock releases.
+		 */
+		else if (cur != MyProc && cur->lwWaitMode == LW_EXCLUSIVE && !woke_up_shared)
+		{
+			wakeup[nwakeup++] = cur;
+
+			/* repoint lock->head since we're removing the current one */
+			if (last == NULL)
+				lock->head = cur->lwWaitLink;
+			/* unlink current node from chain */
+			else
+				last->lwWaitLink = cur->lwWaitLink;
+
+			/*
+			 * removing the last node, we need to repoint ->tail,
+			 * otherwise appending won't work.
+			 */
+			if (cur == lock->tail)
+				lock->tail = last;
+
+			Assert((lock->head != NULL && lock->tail != NULL) ||
+			       lock->head == NULL);
+
+			cur->lwWaitLink = NULL;
+			/*
+			 * don't want to wakeup again before any of the woken up
+			 * entries got scheduled.
+			 */
+			releaseOK = false;
+			break;
+		}
+		/* won't be woken up, but we want to look at the later nodes */
+		else
+		{
+			last = cur;
+			cur = cur->lwWaitLink;
+			continue;
+		}
+
+		/* remove current from linked list */
+
+		/* current is the first node */
+		if (last == NULL)
+			lock->head = cur->lwWaitLink;
+		else
+			last->lwWaitLink = cur->lwWaitLink;
+
+		/* current is the end of the list, repoint tail */
+		if (cur == lock->tail)
+			lock->tail = last;
+
+		next = cur->lwWaitLink;
+		/* mark us not being part of the list, but do not yet wake up */
+		cur->lwWaitLink = NULL;
+		cur = next;
+	}
+
+	Assert((lock->head != NULL && lock->tail != NULL) ||
+	       lock->head == NULL);
+
+unlock:
+	lock->releaseOK = releaseOK;
+
+	/* We are done updating shared state of the lock queue. */
+	SpinLockRelease(&lock->mutex);
+
+	/*
+	 * Awaken any waiters I removed from the queue.
+	 */
+	for (i = 0; i < nwakeup; i++)
+	{
+		PGPROC *proc = wakeup[i];
+
+		Assert(proc != MyProc);
+		Assert(lock->head != proc);
+		Assert(lock->tail != proc);
+
+		/*
+		 * unset waiting, doing so only now allows some more error
+		 * checking to be done elsewhere
+		 */
+		proc->lwWaiting = false;
+		PGSemaphoreUnlock(&proc->sem);
+	}
+}
+
+/*
+ * Add ourselves to the end of the queue. Mode can be
+ * LW_WAIT_UNTIL_FREE here!
+ */
+static inline void
+LWLockQueueSelf(volatile LWLock *lock, LWLockMode mode)
+{
+	/*
+	 * If we don't have a PGPROC structure, there's no way to wait. This
+	 * should never occur, since MyProc should only be null during shared
+	 * memory initialization.
+	 */
+	if (MyProc == NULL)
+		elog(PANIC, "cannot wait without a PGPROC structure");
+
+	__sync_fetch_and_add(&lock->waiters, 1);
+
+	SpinLockAcquire(&lock->mutex);
+
+	/* consistent list state */
+	Assert((lock->head != NULL && lock->tail != NULL) ||
+	       lock->head == NULL);
+	/* we can only be part of one queue at a time */
+	Assert(!MyProc->lwWaiting);
+	/* quick and dirty check for repeated queueing */
+	Assert(lock->head != MyProc);
+	Assert(lock->tail != MyProc);
+
+	MyProc->lwWaiting = true;
+	MyProc->lwWaitMode = mode;
+	MyProc->lwWaitLink = NULL;
+
+	/* no list there yet */
+	if (lock->head == NULL)
+		lock->head = MyProc;
+	/* to the end with it */
+	else
+	{
+		Assert(lock->tail != NULL);
+		Assert(lock->tail->lwWaitLink == NULL);
+
+		lock->tail->lwWaitLink = MyProc;
+		Assert(lock->tail->lwWaitLink != lock->tail);
+	}
+	lock->tail = MyProc;
+
+	/* Can release the mutex now */
+	SpinLockRelease(&lock->mutex);
+}
+
+/*
+ * Go ahead and remove ourselves from somewhere on the waitlist after
+ * we've discovered that we don't actually need to be on it after some
+ * more checking. Somebody else might have already woken us up, in
+ * that case return false.
+ */
+static inline bool
+LWLockUnqueueSelf(volatile LWLock *lock)
+{
+	PGPROC *proc;
+	PGPROC *prev = NULL;
+	bool	found = false;
+
+	SpinLockAcquire(&lock->mutex);
+	proc = lock->head;
+
+	while (proc != NULL)
+	{
+		if (proc == MyProc)
+		{
+			/* removing first element */
+			if (prev == NULL)
+				lock->head = proc->lwWaitLink;
+			/* any other */
+			else
+			{
+				prev->lwWaitLink = proc->lwWaitLink;
+				Assert(prev->lwWaitLink != prev);
+			}
+
+			/* removing last element, repoint tail */
+			if (proc == lock->tail)
+				lock->tail = prev;
+
+			proc->lwWaitLink = NULL;
+			proc->lwWaiting = false;
+			found = true;
+			break;
+		}
+		prev = proc;
+		proc = proc->lwWaitLink;
+	}
+
+	Assert((lock->head != NULL && lock->tail != NULL) ||
+	       lock->head == NULL);
+
+	SpinLockRelease(&lock->mutex);
+
+	/*
+	 * We cannot still be part of a queue even if somebody else woke
+	 * us up, but we still can be marked as lwWaiting.
+	 */
+	Assert(MyProc->lwWaitLink == NULL);
+
+	if (found)
+		__sync_fetch_and_sub(&lock->waiters, 1);
+	return found;
+}
 
 /*
  * LWLockAcquire - acquire a lightweight lock in the specified mode
@@ -342,10 +726,11 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
 	PGPROC	   *proc = MyProc;
-	bool		retry = false;
 	int			extraWaits = 0;
+	bool		potentially_spurious;
+	uint32		iterations = 0;
 
-	PRINT_LWDEBUG("LWLockAcquire", lockid, lock);
+	PRINT_LWDEBUG("LWLockAcquire", lockid, lock, mode);
 
 #ifdef LWLOCK_STATS
 	/* Set up local count state first time through in a given process */
@@ -395,64 +780,69 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 	for (;;)
 	{
 		bool		mustwait;
-
 		/* Acquire mutex.  Time spent holding mutex should be short! */
 #ifdef LWLOCK_STATS
 		spin_delay_counts[lockid] += SpinLockAcquire(&lock->mutex);
 #else
-		SpinLockAcquire(&lock->mutex);
 #endif
 
-		/* If retrying, allow LWLockRelease to release waiters again */
-		if (retry)
-			lock->releaseOK = true;
+		/* try to grab the lock the first time, we're not in the waitqueue yet */
+		mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious);
+
+		if (!mustwait)
+			break;				/* got the lock */
+
+		/*
+		 * Ok, at this point we couldn't grab the lock on the first
+		 * try. We cannot simply queue ourselves to the end of the
+		 * list and wait to be woken up because by now the lock could
+		 * long have been released. Instead add us to the queue and
+		 * try to grab the lock again. If we suceed we need to revert
+		 * the queuing and be happy, otherwise we recheck the lock. If
+		 * we still couldn't grab it, we know that the other lock will
+		 * see our queue entries when releasing since they existed
+		 * before we checked for the lock.
+		 */
+
+		/* add to the queue */
+		LWLockQueueSelf(lock, mode);
+
+		/* we're now guaranteed to be woken up if neccessary */
+		mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious);
 
-		/* If I can get the lock, do so quickly. */
-		if (mode == LW_EXCLUSIVE)
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
 		{
-			if (lock->exclusive == 0 && lock->shared == 0)
+			if (!LWLockUnqueueSelf(lock))
 			{
-				lock->exclusive++;
-				mustwait = false;
+				/*
+				 * somebody else unqueued us and has or will wake us
+				 * up. Wait for the correct wakeup, otherwise our
+				 * ->lwWaiting might get at some unconvenient point
+				 * later, and releaseOk wouldn't be managed correctly.
+				 */
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+				lock->releaseOK = true;
 			}
-			else
-				mustwait = true;
+			PRINT_LWDEBUG("LWLockAcquire undo queue", lockid, lock, mode);
+			break;
 		}
 		else
 		{
-			if (lock->exclusive == 0)
-			{
-				lock->shared++;
-				mustwait = false;
-			}
-			else
-				mustwait = true;
+			PRINT_LWDEBUG("LWLockAcquire waiting 4", lockid, lock, mode);
 		}
 
-		if (!mustwait)
-			break;				/* got the lock */
-
 		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait. This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
+		 * NB: There's no need to deal with spurious lock attempts
+		 * here. Anyone we prevented from acquiring the lock will
+		 * enqueue themselves using the same protocol we used here.
 		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
-
-		proc->lwWaiting = true;
-		proc->lwWaitMode = mode;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
-
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
 
 		/*
 		 * Wait until awakened.
@@ -466,7 +856,7 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockAcquire", lockid, "waiting");
+		LOG_LWDEBUG("LWLockAcquire", lockid, mode, "waiting");
 
 #ifdef LWLOCK_STATS
 		block_counts[lockid]++;
@@ -482,28 +872,34 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 				break;
 			extraWaits++;
 		}
+		lock->releaseOK = true;
 
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(lockid, mode);
 
-		LOG_LWDEBUG("LWLockAcquire", lockid, "awakened");
+		LOG_LWDEBUG("LWLockAcquire", lockid, mode, "awakened");
 
 		/* Now loop back and try to acquire lock again. */
-		retry = true;
+		__sync_fetch_and_sub(&lock->waiters, 1);
+		iterations++;
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(lockid, mode);
 
 	/* Add lock to list of locks held by this backend */
-	held_lwlocks[num_held_lwlocks++] = lockid;
+	held_lwlocks[num_held_lwlocks].lock = lockid;
+	held_lwlocks[num_held_lwlocks++].mode = mode;
 
 	/*
 	 * Fix the process wait semaphore's count for any absorbed wakeups.
 	 */
 	while (extraWaits-- > 0)
 		PGSemaphoreUnlock(&proc->sem);
+
+	/* some minor consistency checks */
+	Assert(MyProc == NULL || MyProc->lwWaitLink == NULL);
+	Assert(MyProc == NULL || !MyProc->lwWaiting);
+	Assert(MyProc == NULL || lock->head != MyProc);
+	Assert(MyProc == NULL || lock->tail != MyProc);
 }
 
 /*
@@ -518,8 +914,9 @@ LWLockConditionalAcquire(LWLockId lockid, LWLockMode mode)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
 	bool		mustwait;
+	bool		potentially_spurious;
 
-	PRINT_LWDEBUG("LWLockConditionalAcquire", lockid, lock);
+	PRINT_LWDEBUG("LWLockConditionalAcquire", lockid, lock, mode);
 
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
@@ -531,49 +928,51 @@ LWLockConditionalAcquire(LWLockId lockid, LWLockMode mode)
 	 * manipulations of data structures in shared memory.
 	 */
 	HOLD_INTERRUPTS();
-
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
+retry:
+	/*
+	 * passing 'true' to check more carefully to avoid potential
+	 * spurious acquirations
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, true, &potentially_spurious);
 
 	if (mustwait)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockConditionalAcquire", lockid, "failed");
+		LOG_LWDEBUG("LWLockConditionalAcquire", lockid, mode, "failed");
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(lockid, mode);
+
+		/*
+		 * We ran into an exclusive lock and might have blocked
+		 * another exclusive lock from taking a shot because it took a
+		 * time to back off. Retry till we are either sure we didn't
+		 * block somebody (because somebody else securely has the
+		 * lock) or till we got it.
+		 *
+		 * We cannot rely on the two-step lock-acquiration protocol as
+		 * in LWLockAcquire because we're not using it.
+		 *
+		 * Retry until we're sure no spurious acquiration happened.
+		 */
+		if (potentially_spurious)
+		{
+			SPIN_DELAY();
+			goto retry;
+		}
 	}
 	else
 	{
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lockid;
+		held_lwlocks[num_held_lwlocks].lock = lockid;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(lockid, mode);
 	}
 
+	Assert(MyProc == NULL || MyProc->lwWaitLink == NULL);
+	Assert(MyProc == NULL || !MyProc->lwWaiting);
+	Assert(MyProc == NULL || lock->head != MyProc);
+	Assert(MyProc == NULL || lock->tail != MyProc);
+
 	return !mustwait;
 }
 
@@ -598,8 +997,10 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 	PGPROC	   *proc = MyProc;
 	bool		mustwait;
 	int			extraWaits = 0;
+	bool		potentially_spurious_first;
+	bool		potentially_spurious_second;
 
-	PRINT_LWDEBUG("LWLockAcquireOrWait", lockid, lock);
+	PRINT_LWDEBUG("LWLockAcquireOrWait", lockid, lock, mode);
 
 #ifdef LWLOCK_STATS
 	/* Set up local count state first time through in a given process */
@@ -618,84 +1019,66 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
+	/*
+	 * NB: We're using nearly the same twice-in-a-row lock acquiration
+	 * protocol as LWLockAcquire(). Check it's comments for details.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious_first);
 
 	if (mustwait)
 	{
-		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait.  This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
-		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
-
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
+		mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious_second);
 
-		/*
-		 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
-		 * wakups, because we share the semaphore with ProcWaitForSignal.
-		 */
-		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, "waiting");
+		if (mustwait)
+		{
+			/*
+			 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
+			 * wakups, because we share the semaphore with ProcWaitForSignal.
+			 */
+			LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "waiting");
 
 #ifdef LWLOCK_STATS
-		block_counts[lockid]++;
+			block_counts[lockid]++;
 #endif
+			TRACE_POSTGRESQL_LWLOCK_WAIT_START(lockid, mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_START(lockid, mode);
+			for (;;)
+			{
+				/* "false" means cannot accept cancel/die interrupt here. */
+				PGSemaphoreLock(&proc->sem, false);
+				if (!proc->lwWaiting)
+					break;
+				extraWaits++;
+			}
 
-		for (;;)
-		{
-			/* "false" means cannot accept cancel/die interrupt here. */
-			PGSemaphoreLock(&proc->sem, false);
-			if (!proc->lwWaiting)
-				break;
-			extraWaits++;
-		}
+			Assert(lock->head != MyProc);
+			Assert(lock->tail != MyProc);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(lockid, mode);
+			__sync_fetch_and_sub(&lock->waiters, 1);
 
-		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, "awakened");
-	}
-	else
-	{
-		/* We are done updating shared state of the lock itself. */
-		SpinLockRelease(&lock->mutex);
+			TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(lockid, mode);
+
+			LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "awakened");
+		}
+		else
+		{
+			/* got lock in the second attempt, undo queueing */
+			if (!LWLockUnqueueSelf(lock))
+			{
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+			}
+
+			/* FIXME: don't need that anymore? */
+			//LWLockWakeup(lock, lockid, mode);
+		}
 	}
 
 	/*
@@ -708,16 +1091,23 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, "failed");
+		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "failed");
 		TRACE_POSTGRESQL_LWLOCK_WAIT_UNTIL_FREE_FAIL(lockid, mode);
 	}
 	else
 	{
+		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "suceeded");
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lockid;
+		held_lwlocks[num_held_lwlocks].lock = lockid;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_WAIT_UNTIL_FREE(lockid, mode);
 	}
 
+	Assert(MyProc == NULL || MyProc->lwWaitLink == NULL);
+	Assert(MyProc == NULL || !MyProc->lwWaiting);
+	Assert(MyProc == NULL || lock->head != MyProc);
+	Assert(MyProc == NULL || lock->tail != MyProc);
+
 	return !mustwait;
 }
 
@@ -728,11 +1118,11 @@ void
 LWLockRelease(LWLockId lockid)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
-	PGPROC	   *head;
-	PGPROC	   *proc;
 	int			i;
-
-	PRINT_LWDEBUG("LWLockRelease", lockid, lock);
+	LWLockMode	mode;
+	uint32		lockcount;
+	bool		check_waiters;
+	bool		have_waiters = false;
 
 	/*
 	 * Remove lock from list of locks held.  Usually, but not always, it will
@@ -740,8 +1130,11 @@ LWLockRelease(LWLockId lockid)
 	 */
 	for (i = num_held_lwlocks; --i >= 0;)
 	{
-		if (lockid == held_lwlocks[i])
+		if (lockid == held_lwlocks[i].lock)
+		{
+			mode = held_lwlocks[i].mode;
 			break;
+		}
 	}
 	if (i < 0)
 		elog(ERROR, "lock %d is not held", (int) lockid);
@@ -749,100 +1142,63 @@ LWLockRelease(LWLockId lockid)
 	for (; i < num_held_lwlocks; i++)
 		held_lwlocks[i] = held_lwlocks[i + 1];
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
+	PRINT_LWDEBUG("LWLockRelease", lockid, lock, mode);
 
-	/* Release my hold on lock */
-	if (lock->exclusive > 0)
-		lock->exclusive--;
-	else
-	{
-		Assert(lock->shared > 0);
-		lock->shared--;
-	}
+	pg_read_barrier();
 
-	/*
-	 * See if I need to awaken any waiters.  If I released a non-last shared
-	 * hold, there cannot be anything to do.  Also, do not awaken any waiters
-	 * if someone has already awakened waiters that haven't yet acquired the
-	 * lock.
-	 */
-	head = lock->head;
-	if (head != NULL)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
-		{
-			/*
-			 * Remove the to-be-awakened PGPROCs from the queue.
-			 */
-			bool		releaseOK = true;
+#ifdef NOT_ANYMORE
+	if (pg_atomic_read(lock->waiters) > 0)
+		have_waiters = true;
+#endif
 
-			proc = head;
+	/* Release my hold on lock, both are a full barrier */
+	if (mode == LW_EXCLUSIVE)
+		lockcount = __sync_sub_and_fetch(&lock->lockcount, EXCLUSIVE_LOCK_FINALIZED);
+	else
+		lockcount = __sync_sub_and_fetch(&lock->lockcount, 1);
 
-			/*
-			 * First wake up any backends that want to be woken up without
-			 * acquiring the lock.
-			 */
-			while (proc->lwWaitMode == LW_WAIT_UNTIL_FREE && proc->lwWaitLink)
-				proc = proc->lwWaitLink;
+	/* nobody else can have that kind of lock */
+	Assert(lockcount < EXCLUSIVE_LOCK_FINALIZED);
 
-			/*
-			 * If the front waiter wants exclusive lock, awaken him only.
-			 * Otherwise awaken as many waiters as want shared access.
-			 */
-			if (proc->lwWaitMode != LW_EXCLUSIVE)
-			{
-				while (proc->lwWaitLink != NULL &&
-					   proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
-				{
-					if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
-						releaseOK = false;
-					proc = proc->lwWaitLink;
-				}
-			}
-			/* proc is now the last PGPROC to be released */
-			lock->head = proc->lwWaitLink;
-			proc->lwWaitLink = NULL;
+	/*
+	 * Anybody we need to wakeup needs to have started queueing before
+	 * we removed ourselves from the queue and the __sync_ operations
+	 * above are full barriers.
+	 */
 
-			/*
-			 * Prevent additional wakeups until retryer gets to run. Backends
-			 * that are just waiting for the lock to become free don't retry
-			 * automatically.
-			 */
-			if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
-				releaseOK = false;
+	if (pg_atomic_read(lock->waiters) > 0)
+		have_waiters = true;
+
+	/* we're still waiting for backends to get scheduled, don't release again */
+	if (!lock->releaseOK)
+		check_waiters = false;
+	/* grant permission to run, even if a spurious share lock increases logcount */
+	else if (mode == LW_EXCLUSIVE && have_waiters)
+		check_waiters = true;
+	/* nobody has this locked anymore, potential exclusive lockers get a chance */
+	else if (lockcount == 0 && have_waiters)
+		check_waiters = true;
+	/* nobody queued or not free */
+	else
+		check_waiters = false;
 
-			lock->releaseOK = releaseOK;
-		}
-		else
-		{
-			/* lock is still held, can't awaken anything */
-			head = NULL;
-		}
+	if (check_waiters)
+	{
+		PRINT_LWDEBUG("LWLockRelease releasing", lockid, lock, mode);
+		LWLockWakeup(lock, lockid, mode);
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_RELEASE(lockid);
 
 	/*
-	 * Awaken any waiters I removed from the queue.
-	 */
-	while (head != NULL)
-	{
-		LOG_LWDEBUG("LWLockRelease", lockid, "release waiter");
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
-	}
-
-	/*
 	 * Now okay to allow cancel/die interrupts.
 	 */
 	RESUME_INTERRUPTS();
+
+	Assert(MyProc == NULL || MyProc->lwWaitLink == NULL);
+	Assert(MyProc == NULL || !MyProc->lwWaiting);
+	Assert(MyProc == NULL || lock->head != MyProc);
+	Assert(MyProc == NULL || lock->tail != MyProc);
 }
 
 
@@ -862,7 +1218,7 @@ LWLockReleaseAll(void)
 	{
 		HOLD_INTERRUPTS();		/* match the upcoming RESUME_INTERRUPTS */
 
-		LWLockRelease(held_lwlocks[num_held_lwlocks - 1]);
+		LWLockRelease(held_lwlocks[num_held_lwlocks - 1].lock);
 	}
 }
 
@@ -870,8 +1226,7 @@ LWLockReleaseAll(void)
 /*
  * LWLockHeldByMe - test whether my process currently holds a lock
  *
- * This is meant as debug support only.  We do not distinguish whether the
- * lock is held shared or exclusive.
+ * This is meant as debug support only.
  */
 bool
 LWLockHeldByMe(LWLockId lockid)
@@ -880,7 +1235,7 @@ LWLockHeldByMe(LWLockId lockid)
 
 	for (i = 0; i < num_held_lwlocks; i++)
 	{
-		if (held_lwlocks[i] == lockid)
+		if (held_lwlocks[i].lock == lockid)
 			return true;
 	}
 	return false;
-- 
1.8.4.21.g992c386.dirty

#2Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#1)
Re: Wait free LW_SHARED acquisition

On Thu, Sep 26, 2013 at 3:55 PM, Andres Freund <andres@2ndquadrant.com> wrote:

We have had several customers running postgres on bigger machines report
problems on busy systems. Most recently one where a fully cached
workload completely stalled in s_lock()s due to the *shared* lwlock
acquisition in BufferAlloc() around the buffer partition lock.

That's unfortunate. I saw someone complain about what sounds like
exactly the same issue on Twitter yesterday:

https://twitter.com/Roguelazer/status/382706273927446528

I tried to engage with him, but was unsuccessful.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#3Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#2)
Re: Wait free LW_SHARED acquisition

On 2013-09-26 16:56:30 -0700, Peter Geoghegan wrote:

On Thu, Sep 26, 2013 at 3:55 PM, Andres Freund <andres@2ndquadrant.com> wrote:

We have had several customers running postgres on bigger machines report
problems on busy systems. Most recently one where a fully cached
workload completely stalled in s_lock()s due to the *shared* lwlock
acquisition in BufferAlloc() around the buffer partition lock.

That's unfortunate. I saw someone complain about what sounds like
exactly the same issue on Twitter yesterday:

Well, fortunately there's a solution, as presented here ;)

There's another bottleneck in the heaps of PinBuffer() calls in such
workloads, that present themselves after fixing the lwlock contention,
at least in my tests. I think I see a solution there, but let's fix this
first though ;)

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#4Heikki Linnakangas
hlinnakangas@vmware.com
In reply to: Andres Freund (#1)
Re: Wait free LW_SHARED acquisition

On 27.09.2013 01:55, Andres Freund wrote:

Hello,

We have had several customers running postgres on bigger machines report
problems on busy systems. Most recently one where a fully cached
workload completely stalled in s_lock()s due to the *shared* lwlock
acquisition in BufferAlloc() around the buffer partition lock.

Increasing the padding to a full cacheline helps making the partitioning
of the partition space actually effective (before it's essentially
halved on a read-mostly workload), but that still leaves one with very
hot spinlocks.

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

How does that compare with simply increasing NUM_BUFFER_PARTITIONS?

- Heikki

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#5Andres Freund
andres@2ndquadrant.com
In reply to: Heikki Linnakangas (#4)
Re: Wait free LW_SHARED acquisition

Hi,

On 2013-09-27 10:14:46 +0300, Heikki Linnakangas wrote:

On 27.09.2013 01:55, Andres Freund wrote:

We have had several customers running postgres on bigger machines report
problems on busy systems. Most recently one where a fully cached
workload completely stalled in s_lock()s due to the *shared* lwlock
acquisition in BufferAlloc() around the buffer partition lock.

Increasing the padding to a full cacheline helps making the partitioning
of the partition space actually effective (before it's essentially
halved on a read-mostly workload), but that still leaves one with very
hot spinlocks.

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

How does that compare with simply increasing NUM_BUFFER_PARTITIONS?

Heaps better. In the case causing this investigation lots of the pages
with hot spinlocks were the simply the same ones over and over again,
partitioning the lockspace won't help much there.
That's not exactly an uncommon scenario since often enough there's a
small amount of data hit very frequently and lots more that is accessed
only infrequently. E.g. recently inserted data and such tends to be very hot.

I can run a test on the 4 socket machine if it's unused, but on my 2
socket workstation the benefits of at least our simulation of the
original workloads the improvements were marginal after increasing the
padding to a full cacheline.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#6Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#5)
Re: Wait free LW_SHARED acquisition

On 2013-09-27 09:21:05 +0200, Andres Freund wrote:

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

How does that compare with simply increasing NUM_BUFFER_PARTITIONS?

Heaps better. In the case causing this investigation lots of the pages
with hot spinlocks were the simply the same ones over and over again,
partitioning the lockspace won't help much there.
That's not exactly an uncommon scenario since often enough there's a
small amount of data hit very frequently and lots more that is accessed
only infrequently. E.g. recently inserted data and such tends to be very hot.

I can run a test on the 4 socket machine if it's unused, but on my 2
socket workstation the benefits of at least our simulation of the
original workloads the improvements were marginal after increasing the
padding to a full cacheline.

Ok, was free:

padding + 16 partitions:
tps = 147884.648416

padding + 32 partitions:
tps = 141777.841125

padding + 64 partitions:
tps = 141561.539790

padding + 16 partitions + new lwlocks
tps = 601895.580903 (yeha, still reproduces after some sleep!)

Now, the other numbers were best-of-three, these aren't, but I think
it's pretty clear that you're not going to see the same benefits. I am
not surprised...
The current implementation of lwlocks will frequently block others, both
during acquiration and release of locks. What's even worse, trying to
fruitlessly acquire a spinlock will often prevent releasing it because
we need the spinlock during release.
With the proposed algorithm, even if we need the spinlock during release
of an lwlock because there are queued PGPROCs, we will acquire that
spinlock only after already having released the lock...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#7Heikki Linnakangas
hlinnakangas@vmware.com
In reply to: Andres Freund (#5)
Re: Wait free LW_SHARED acquisition

On 27.09.2013 10:21, Andres Freund wrote:

Hi,

On 2013-09-27 10:14:46 +0300, Heikki Linnakangas wrote:

On 27.09.2013 01:55, Andres Freund wrote:

We have had several customers running postgres on bigger machines report
problems on busy systems. Most recently one where a fully cached
workload completely stalled in s_lock()s due to the *shared* lwlock
acquisition in BufferAlloc() around the buffer partition lock.

Increasing the padding to a full cacheline helps making the partitioning
of the partition space actually effective (before it's essentially
halved on a read-mostly workload), but that still leaves one with very
hot spinlocks.

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

How does that compare with simply increasing NUM_BUFFER_PARTITIONS?

Heaps better. In the case causing this investigation lots of the pages
with hot spinlocks were the simply the same ones over and over again,
partitioning the lockspace won't help much there.
That's not exactly an uncommon scenario since often enough there's a
small amount of data hit very frequently and lots more that is accessed
only infrequently. E.g. recently inserted data and such tends to be very hot.

I see. So if only a few buffers are really hot, I'm assuming the problem
isn't just the buffer partition lock, but also the spinlock on the
buffer header, and the buffer content lwlock. Yeah, improving LWLocks
would be a nice wholesale solution to that. I don't see any fundamental
flaw in your algorithm. Nevertheless, I'm going to throw in a couple of
other ideas:

* Keep a small 4-5 entry cache of buffer lookups in each backend of most
recently accessed buffers. Before searching for a buffer in the
SharedBufHash, check the local cache.

* To pin a buffer, use an atomic fetch-and-add instruction to increase
the refcount. PinBuffer() also increases usage_count, but you could do
that without holding a lock; it doesn't need to be accurate.

One problem with your patch is going to be to make it also work without
the CAS and fetch-and-add instructions. Those are probably present in
all the architectures we support, but it'll take some effort to get the
architecture-specific code done. Until it's all done, it would be good
to be able to fall back to plain spinlocks, which we already have. Also,
when someone ports PostgreSQL to a new architecture in the future, it
would be helpful if you wouldn't need to write all the
architecture-specific code immediately to get it to compile.

Did you benchmark your patch against the compare-and-swap patch I posted
earlier?
(/messages/by-id/519A3587.80603@vmware.com). Just
on a theoretical level, I would assume your patch to scale better since
it uses fetch-and-add instead of compare-and-swap for acquiring a shared
lock. But in practice it might be a wash.

- Heikki

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#8Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#6)
Re: Wait free LW_SHARED acquisition

On 2013-09-27 09:57:07 +0200, Andres Freund wrote:

On 2013-09-27 09:21:05 +0200, Andres Freund wrote:

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

How does that compare with simply increasing NUM_BUFFER_PARTITIONS?

Heaps better. In the case causing this investigation lots of the pages
with hot spinlocks were the simply the same ones over and over again,
partitioning the lockspace won't help much there.
That's not exactly an uncommon scenario since often enough there's a
small amount of data hit very frequently and lots more that is accessed
only infrequently. E.g. recently inserted data and such tends to be very hot.

I can run a test on the 4 socket machine if it's unused, but on my 2
socket workstation the benefits of at least our simulation of the
original workloads the improvements were marginal after increasing the
padding to a full cacheline.

Ok, was free:

padding + 16 partitions:
tps = 147884.648416

padding + 32 partitions:
tps = 141777.841125

padding + 64 partitions:
tps = 141561.539790

padding + 16 partitions + new lwlocks
tps = 601895.580903 (yeha, still reproduces after some sleep!)

Pgbench numbers for writes on the machine (fsync = off,
synchronous_commit = off):
padding + 16 partitions:
tps = 8903.532163
vs
padding + 16 partitions + new lwlocks
tps = 13324.080491

So, on bigger machines the advantages seem to be there for writes as
well...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#9Andres Freund
andres@2ndquadrant.com
In reply to: Heikki Linnakangas (#7)
Re: Wait free LW_SHARED acquisition

On 2013-09-27 11:11:56 +0300, Heikki Linnakangas wrote:

On 27.09.2013 10:21, Andres Freund wrote:

Heaps better. In the case causing this investigation lots of the pages
with hot spinlocks were the simply the same ones over and over again,
partitioning the lockspace won't help much there.
That's not exactly an uncommon scenario since often enough there's a
small amount of data hit very frequently and lots more that is accessed
only infrequently. E.g. recently inserted data and such tends to be very hot.

I see. So if only a few buffers are really hot, I'm assuming the problem
isn't just the buffer partition lock, but also the spinlock on the buffer
header, and the buffer content lwlock. Yeah, improving LWLocks would be a
nice wholesale solution to that. I don't see any fundamental flaw in your
algorithm. Nevertheless, I'm going to throw in a couple of other ideas:

* Keep a small 4-5 entry cache of buffer lookups in each backend of most
recently accessed buffers. Before searching for a buffer in the
SharedBufHash, check the local cache.

I thought about that as well, but you'd either need to revalidate after
pinning the buffers, or keep them pinned.
I had a very hacky implementation of that, but it just make the buffer
content locks the top profile spots. Similar issue there.

It might be worthwile to do this nonetheless - lock xadd; certainly
isn't cheap, even due it's cheaper than a full spinnlock - but it's not
trivial.

* To pin a buffer, use an atomic fetch-and-add instruction to increase the
refcount. PinBuffer() also increases usage_count, but you could do that
without holding a lock; it doesn't need to be accurate.

Yes, Pin/UnpinBuffer() are the primary contention points after this
patch. I want to tackle them, but that seems like a separate thing to
do.

I think we should be able to get rid of most or even all LockBufHdr()
calls by
a) introducing PinBufferInternal() which increase pins but not
usage count using an atomic increment. That can replace locking headers
in many cases.
b) make PinBuffer() increment pin and usagecount using a single 64bit
atomic add if available and fit flags in there as well. Then back off
the usagecount if it's too high or even wraps around, that doesn't hurt
much, we're pinned in that moment.

One problem with your patch is going to be to make it also work without the
CAS and fetch-and-add instructions. Those are probably present in all the
architectures we support, but it'll take some effort to get the
architecture-specific code done. Until it's all done, it would be good to be
able to fall back to plain spinlocks, which we already have. Also, when
someone ports PostgreSQL to a new architecture in the future, it would be
helpful if you wouldn't need to write all the architecture-specific code
immediately to get it to compile.

I think most recent compilers have intrinsics for stuff for operations
like that. I quite like the API provided by gcc for this kind of stuff,
I think we should model an internal wrapper API similarly. I don't see
any new platforming coming that has a compiler without intrinsics?

But yes, you're right, I think we need a spinlock based fallback for
now. Even if it's just because nobody of us can verify the
implementations on the more obsolete platforms we claim to support.I
just didn't see it as a priority in the PoC.

Did you benchmark your patch against the compare-and-swap patch I posted
earlier? (/messages/by-id/519A3587.80603@vmware.com).
Just on a theoretical level, I would assume your patch to scale better since
it uses fetch-and-add instead of compare-and-swap for acquiring a shared
lock. But in practice it might be a wash.

I've tried compare-and-swap for shared acquisition and it performed
worse, didn't try your patch though as you seemed to have concluded it's
a wash after doing the unlocked test.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#10Florian Pflug
fgp@phlo.org
In reply to: Andres Freund (#1)
Re: Wait free LW_SHARED acquisition

On Sep27, 2013, at 00:55 , Andres Freund <andres@2ndquadrant.com> wrote:

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

That's rougly 400%.

Interesting. I played with pretty much the same idea two years or so ago.
At the time, I compared a few different LWLock implementations. Those
were AFAIR

A) Vanilla LWLocks
B) A + an atomic-increment fast path, very similar to your proposal
C) B but with a partitioned atomic-increment counter to further
reduce cache-line contention
D) A with the spinlock-based queue replaced by a lockless queue

At the time, the improvements seemed to be negligible - they looked great
in synthetic benchmarks of just the locking code, but didn't translate
to improved TPS numbers. Though I think the only version that ever got
tested on more than a handful of cores was C…

My (rather hacked together) benchmarking code can be found here: https://github.com/fgp/lockbench.
The different LWLock implementations live in the various pg_lwlock_* subfolders.

Here's a pointer into the relevant thread: /messages/by-id/651002C1-2EC1-4731-9B29-99217CB36653@phlo.org

best regards,
Florian Pflug

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#11Andres Freund
andres@2ndquadrant.com
In reply to: Florian Pflug (#10)
Re: Wait free LW_SHARED acquisition

On 2013-09-27 14:46:50 +0200, Florian Pflug wrote:

On Sep27, 2013, at 00:55 , Andres Freund <andres@2ndquadrant.com> wrote:

So the goal is to have LWLockAcquire(LW_SHARED) never block unless
somebody else holds an exclusive lock. To produce enough appetite for
reading the rest of the long mail, here's some numbers on a
pgbench -j 90 -c 90 -T 60 -S (-i -s 10) on a 4xE5-4620

master + padding: tps = 146904.451764
master + padding + lwlock: tps = 590445.927065

That's rougly 400%.

Interesting. I played with pretty much the same idea two years or so ago.
At the time, I compared a few different LWLock implementations. Those
were AFAIR

A) Vanilla LWLocks
B) A + an atomic-increment fast path, very similar to your proposal
C) B but with a partitioned atomic-increment counter to further
reduce cache-line contention
D) A with the spinlock-based queue replaced by a lockless queue

At the time, the improvements seemed to be negligible - they looked great
in synthetic benchmarks of just the locking code, but didn't translate
to improved TPS numbers. Though I think the only version that ever got
tested on more than a handful of cores was C…

I think you really need multi-socket systems to see the big benefits
from this. My laptop barely shows any improvements, while my older 2
socket workstation already shows some in workloads that have more
contention than pgbench -S.

From a quick look, you didn't have any sleeping queueing in at least one
of the variants in there? In my tests, that was tremendously important
to improve scaling if there was any contention. Which is not surprising
in the end, because otherwise you essentially have rw-spinlocks which
really aren't suitable for many of the lwlocks we use.

Getting the queueing semantics, including releaseOK, right was what took
me a good amount of time, the atomic ops part was pretty quick...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#12Bernd Helmle
mailings@oopsware.de
In reply to: Andres Freund (#6)
Re: Wait free LW_SHARED acquisition

--On 27. September 2013 09:57:07 +0200 Andres Freund
<andres@2ndquadrant.com> wrote:

Ok, was free:

padding + 16 partitions:
tps = 147884.648416

padding + 32 partitions:
tps = 141777.841125

padding + 64 partitions:
tps = 141561.539790

padding + 16 partitions + new lwlocks
tps = 601895.580903 (yeha, still reproduces after some sleep!)

Hmm, out of interest and since i have access to a (atm) free DL580 G7 (4x
E7-4800 10core) i've tried your bench against this machine and got this
(best of three):

HEAD (default):

tps = 181738.607247 (including connections establishing)
tps = 182665.993063 (excluding connections establishing)

HEAD (padding + 16 partitions + your lwlocks patch applied):

tps = 269328.259833 (including connections establishing)
tps = 270685.666091 (excluding connections establishing)

So, still an improvement but far away from what you got. Do you have some
other tweaks in your setup?

--
Thanks

Bernd

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#13Andres Freund
andres@2ndquadrant.com
In reply to: Bernd Helmle (#12)
Re: Wait free LW_SHARED acquisition

Hi,

On 2013-09-30 18:54:11 +0200, Bernd Helmle wrote:

HEAD (default):

tps = 181738.607247 (including connections establishing)
tps = 182665.993063 (excluding connections establishing)

HEAD (padding + 16 partitions + your lwlocks patch applied):

tps = 269328.259833 (including connections establishing)
tps = 270685.666091 (excluding connections establishing)

So, still an improvement but far away from what you got. Do you have some
other tweaks in your setup?

The only relevant setting changed was -c shared_buffers=1GB, no other
patches applied. At which scale did you pgbench -i?

Your processors are a different microarchitecture, I guess that could
also explain some of the difference. Any chance you could run a perf record -ag
(after compiling with -fno-omit-frame-pointer)?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#14Bernd Helmle
mailings@oopsware.de
In reply to: Andres Freund (#13)
Re: Wait free LW_SHARED acquisition

--On 30. September 2013 19:00:06 +0200 Andres Freund
<andres@2ndquadrant.com> wrote:

HEAD (default):

tps = 181738.607247 (including connections establishing)
tps = 182665.993063 (excluding connections establishing)

HEAD (padding + 16 partitions + your lwlocks patch applied):

tps = 269328.259833 (including connections establishing)
tps = 270685.666091 (excluding connections establishing)

So, still an improvement but far away from what you got. Do you have some
other tweaks in your setup?

The only relevant setting changed was -c shared_buffers=1GB, no other
patches applied. At which scale did you pgbench -i?

I've used a scale factor of 10 (i recall you've mentioned using the same
upthread...).

Okay, i've used 2GB shared buffers, repeating with your setting i get a far
more noticable speedup:

tps = 346292.008580 (including connections establishing)
tps = 347997.073595 (excluding connections establishing)

Here's the perf output:

+   4.34%      207112      postgres  postgres                 [.] 
AllocSetAlloc
+   4.07%      194476      postgres  libc-2.13.so             [.] 0x127b33
+   2.59%      123471      postgres  postgres                 [.] 
SearchCatCache
+   2.49%      118974       pgbench  libc-2.13.so             [.] 0x11aaef
+   2.48%      118263      postgres  postgres                 [.] 
GetSnapshotData
+   2.46%      117646      postgres  postgres                 [.] 
base_yyparse
+   2.02%       96546      postgres  postgres                 [.] 
MemoryContextAllocZeroAligned
+   1.58%       75326      postgres  postgres                 [.] 
AllocSetFreeIndex
+   1.23%       58587      postgres  postgres                 [.] 
hash_search_with_hash_value
+   1.01%       48391      postgres  postgres                 [.] palloc
+   0.93%       44258      postgres  postgres                 [.] 
LWLockAttemptLock
+   0.91%       43575       pgbench  libc-2.13.so             [.] free
+   0.89%       42484      postgres  postgres                 [.] 
nocachegetattr
+   0.89%       42378      postgres  postgres                 [.] core_yylex
+   0.88%       42001      postgres  postgres                 [.] 
_bt_compare
+   0.84%       39997      postgres  postgres                 [.] 
expression_tree_walker
+   0.76%       36533      postgres  postgres                 [.] 
ScanKeywordLookup
+   0.74%       35515       pgbench  libc-2.13.so             [.] malloc
+   0.64%       30715      postgres  postgres                 [.] 
LWLockRelease
+   0.56%       26779      postgres  postgres                 [.] 
fmgr_isbuiltin
+   0.54%       25681       pgbench  [kernel.kallsyms]        [k] _spin_lock
+   0.48%       22836      postgres  postgres                 [.] new_list
+   0.48%       22700      postgres  postgres                 [.] hash_any
+   0.47%       22378      postgres  postgres                 [.] 
FunctionCall2Coll
+   0.46%       22095      postgres  postgres                 [.] pfree
+   0.44%       20929      postgres  postgres                 [.] palloc0
+   0.43%       20592      postgres  postgres                 [.] 
AllocSetFree
+   0.40%       19495      postgres  [unknown]                [.] 0x81cf2f
+   0.40%       19247      postgres  postgres                 [.] 
hash_uint32
+   0.38%       18227      postgres  postgres                 [.] PinBuffer
+   0.38%       18022       pgbench  [kernel.kallsyms]        [k] do_select

--
Thanks

Bernd

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#15Merlin Moncure
mmoncure@gmail.com
In reply to: Bernd Helmle (#14)
Re: Wait free LW_SHARED acquisition

On Mon, Sep 30, 2013 at 5:28 PM, Bernd Helmle <mailings@oopsware.de> wrote:

--On 30. September 2013 19:00:06 +0200 Andres Freund
<andres@2ndquadrant.com> wrote:

HEAD (default):

tps = 181738.607247 (including connections establishing)
tps = 182665.993063 (excluding connections establishing)

HEAD (padding + 16 partitions + your lwlocks patch applied):

tps = 269328.259833 (including connections establishing)
tps = 270685.666091 (excluding connections establishing)

So, still an improvement but far away from what you got. Do you have some
other tweaks in your setup?

The only relevant setting changed was -c shared_buffers=1GB, no other
patches applied. At which scale did you pgbench -i?

I've used a scale factor of 10 (i recall you've mentioned using the same
upthread...).

Okay, i've used 2GB shared buffers, repeating with your setting i get a far
more noticable speedup:

If Andres's patch passes muster it may end up causing us to
re-evaluate practices for the shared buffer setting. I was trying to
optimize buffer locking in the clock sweep using a different approach
and gave up after not being able to find useful test cases to
demonstrate an improvement. The main reason for this is that clock
sweep issues are masked by contention in the buffer mapping lwlocks
(as you guys noted). I *do* think clock sweep contention comes out in
some production workloads but so far have been elusive to produce in
synthetic testing. Ditto buffer pin contention (this has been
documented).

So I'm very excited about this patch. Right now in servers I
configure (even some very large ones) I set shared buffers to max 2gb
for various reasons. Something tells me that's about to change.

merlin

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#16Andres Freund
andres@2ndquadrant.com
In reply to: Bernd Helmle (#14)
Re: Wait free LW_SHARED acquisition

On 2013-10-01 00:28:55 +0200, Bernd Helmle wrote:

--On 30. September 2013 19:00:06 +0200 Andres Freund
<andres@2ndquadrant.com> wrote:

HEAD (default):

tps = 181738.607247 (including connections establishing)
tps = 182665.993063 (excluding connections establishing)

HEAD (padding + 16 partitions + your lwlocks patch applied):

tps = 269328.259833 (including connections establishing)
tps = 270685.666091 (excluding connections establishing)

So, still an improvement but far away from what you got. Do you have some
other tweaks in your setup?

The only relevant setting changed was -c shared_buffers=1GB, no other
patches applied. At which scale did you pgbench -i?

I've used a scale factor of 10 (i recall you've mentioned using the same
upthread...).

Okay, i've used 2GB shared buffers, repeating with your setting i get a far
more noticable speedup:

tps = 346292.008580 (including connections establishing)
tps = 347997.073595 (excluding connections establishing)

Could you send hierarchical profiles of both 1 and 2GB? It's curious
that the difference is that big... Even though they will be a bit big,
it'd be helpful if you pasted the output of "perf report --stdio", to
include the callers...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#17Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#1)
4 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.2

Hi,

On 2013-09-27 00:55:45 +0200, Andres Freund wrote:

So what's todo? The file header tells us:
* - revive pure-spinlock implementation
* - abstract away atomic ops, we really only need a few.
* - CAS
* - LOCK XADD
* - convert PGPROC->lwWaitLink to ilist.h slist or even dlist.
* - remove LWLockWakeup dealing with MyProc
* - overhaul the mask offsets, make SHARED/EXCLUSIVE_LOCK_MASK wider, MAX_BACKENDS

So, here's the next version of this patchset:
1) I've added an abstracted atomic ops implementation. Needs a fair
amount of work, also submitted as a separate CF entry. (Patch 1 & 2)
2) I've converted PGPROC->lwWaiting into a dlist. That makes a fair bit
of code easier to read and reduces the size of the patchset. Also
fixes a bug in the xlog-scalability code. (Patch 3)
3) Alvaro and I updated the comments in lwlock.c. (Patch 4)

I think 2) should be committable pretty soon. It's imo a pretty clear
win in readability. 1) will need a good bit of more work.

With regard to the scalable lwlock work, what's most needed now is a
good amount of testing.

Please note that you need to 'autoreconf' after applying the patchset. I
don't have a compatible autoconf version on this computer causing the
diff to be humongous if I include those changes.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-Try-to-make-HP-UX-s-ac-not-cause-warnings-about-unus.patchtext/x-patch; charset=us-asciiDownload
>From 71e929cb4de63f060464feb9d73cad1161807253 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Fri, 15 Nov 2013 16:52:02 +0100
Subject: [PATCH 1/4] Try to make HP-UX's ac++ not cause warnings about unused
 static inlines.

---
 src/template/hpux | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/src/template/hpux b/src/template/hpux
index ce4d93c..6349ae5 100644
--- a/src/template/hpux
+++ b/src/template/hpux
@@ -3,7 +3,9 @@
 CPPFLAGS="$CPPFLAGS -D_XOPEN_SOURCE_EXTENDED"
 
 if test "$GCC" != yes ; then
-  CC="$CC -Ae"
+  # -Ae enables C89/C99
+  # +W2177 should disable warnings about unused static inlines
+  CC="$CC -Ae +W2177"
   CFLAGS="+O2"
 fi
 
-- 
1.8.5.rc1.dirty

0002-Very-basic-atomic-ops-implementation.patchtext/x-patch; charset=iso-8859-1Download
From ea80fc0c988b1d51816b2fd4b9d2bba2e7aead05 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Wed, 6 Nov 2013 10:32:53 +0100
Subject: [PATCH 2/4] Very basic atomic ops implementation

Only gcc has been tested, stub implementations for
* msvc
* HUPX acc
* IBM xlc
* Sun/Oracle Sunpro compiler
are included.

All implementations only provide the bare minimum of operations for
now and rely on emulating more complex operations via
compare_and_swap().

Most of s_lock.h is removed and replaced by usage of the atomics ops.
---
 config/c-compiler.m4                             |  77 ++
 configure.in                                     |  19 +-
 src/backend/storage/lmgr/Makefile                |   2 +-
 src/backend/storage/lmgr/atomics.c               |  26 +
 src/backend/storage/lmgr/spin.c                  |   8 -
 src/include/c.h                                  |   7 +
 src/include/storage/atomics-arch-alpha.h         |  25 +
 src/include/storage/atomics-arch-amd64.h         |  56 ++
 src/include/storage/atomics-arch-arm.h           |  29 +
 src/include/storage/atomics-arch-hppa.h          |  17 +
 src/include/storage/atomics-arch-i386.h          |  96 +++
 src/include/storage/atomics-arch-ia64.h          |  27 +
 src/include/storage/atomics-arch-ppc.h           |  25 +
 src/include/storage/atomics-generic-acc.h        |  99 +++
 src/include/storage/atomics-generic-gcc.h        | 174 +++++
 src/include/storage/atomics-generic-msvc.h       |  67 ++
 src/include/storage/atomics-generic-sunpro.h     |  66 ++
 src/include/storage/atomics-generic-xlc.h        |  84 +++
 src/include/storage/atomics-generic.h            | 402 +++++++++++
 src/include/storage/atomics.h                    | 543 ++++++++++++++
 src/include/storage/barrier.h                    | 137 +---
 src/include/storage/s_lock.h                     | 880 +----------------------
 src/test/regress/expected/lock.out               |   8 +
 src/test/regress/input/create_function_1.source  |   5 +
 src/test/regress/output/create_function_1.source |   4 +
 src/test/regress/regress.c                       | 213 ++++++
 src/test/regress/sql/lock.sql                    |   5 +
 27 files changed, 2078 insertions(+), 1023 deletions(-)
 create mode 100644 src/backend/storage/lmgr/atomics.c
 create mode 100644 src/include/storage/atomics-arch-alpha.h
 create mode 100644 src/include/storage/atomics-arch-amd64.h
 create mode 100644 src/include/storage/atomics-arch-arm.h
 create mode 100644 src/include/storage/atomics-arch-hppa.h
 create mode 100644 src/include/storage/atomics-arch-i386.h
 create mode 100644 src/include/storage/atomics-arch-ia64.h
 create mode 100644 src/include/storage/atomics-arch-ppc.h
 create mode 100644 src/include/storage/atomics-generic-acc.h
 create mode 100644 src/include/storage/atomics-generic-gcc.h
 create mode 100644 src/include/storage/atomics-generic-msvc.h
 create mode 100644 src/include/storage/atomics-generic-sunpro.h
 create mode 100644 src/include/storage/atomics-generic-xlc.h
 create mode 100644 src/include/storage/atomics-generic.h
 create mode 100644 src/include/storage/atomics.h

diff --git a/config/c-compiler.m4 b/config/c-compiler.m4
index 4ba3236..c24cb59 100644
--- a/config/c-compiler.m4
+++ b/config/c-compiler.m4
@@ -289,3 +289,80 @@ if test x"$Ac_cachevar" = x"yes"; then
 fi
 undefine([Ac_cachevar])dnl
 ])# PGAC_PROG_CC_LDFLAGS_OPT
+
+AC_DEFUN([PGAC_HAVE_GCC__SYNC_INT32_ATOMICS],
+[AC_CACHE_CHECK(for __builtin_constant_p, pgac_cv__builtin_constant_p,
+[AC_TRY_COMPILE([static int x; static int y[__builtin_constant_p(x) ? x : 1];],
+[],
+[pgac_cv__builtin_constant_p=yes],
+[pgac_cv__builtin_constant_p=no])])
+if test x"$pgac_cv__builtin_constant_p" = xyes ; then
+AC_DEFINE(HAVE__BUILTIN_CONSTANT_P, 1,
+          [Define to 1 if your compiler understands __builtin_constant_p.])
+fi])# PGAC_C_BUILTIN_CONSTANT_P
+
+# PGAC_HAVE_GCC__SYNC_INT32_TAS
+# -------------------------
+# Check if the C compiler understands __sync_lock_test_and_set(),
+# and define HAVE_GCC__SYNC_INT32_TAS
+#
+# NB: There are platforms where test_and_set is available but compare_and_swap
+# is not, so test this separa
+AC_DEFUN([PGAC_HAVE_GCC__SYNC_INT32_TAS],
+[AC_CACHE_CHECK(for builtin locking functions, pgac_cv_gcc_sync_int32_tas,
+[AC_TRY_LINK([],
+  [int lock = 0;
+   __sync_lock_test_and_set(&lock, 1);
+   __sync_lock_release(&lock);],
+  [pgac_cv_gcc_sync_int32_tas="yes"],
+  [pgac_cv_gcc_sync_int32_tas="no"])])
+if test x"$pgac_cv_gcc_sync_int32_tas" = x"yes"; then
+  AC_DEFINE(HAVE_GCC__SYNC_INT32_TAS, 1, [Define to 1 if you have __sync_lock_test_and_set(int *) and friends.])
+fi])# PGAC_HAVE_GCC__SYNC_INT32_TAS
+
+# PGAC_HAVE_GCC__SYNC_INT32_CAS
+# -------------------------
+# Check if the C compiler understands __sync_compare_and_swap() for 32bit
+# types, and define HAVE_GCC__SYNC_INT32_CAS if so.
+AC_DEFUN([PGAC_HAVE_GCC__SYNC_INT32_CAS],
+[AC_CACHE_CHECK(for builtin __sync int32 atomic operations, pgac_cv_gcc_sync_int32_cas,
+[AC_TRY_LINK([],
+  [int val = 0;
+   __sync_val_compare_and_swap(&val, 0, 37);],
+  [pgac_cv_gcc_sync_int32_cas="yes"],
+  [pgac_cv_gcc_sync_int32_cas="no"])])
+if test x"$pgac_cv_gcc_sync_int32_cas" = x"yes"; then
+  AC_DEFINE(HAVE_GCC__SYNC_INT32_CAS, 1, [Define to 1 if you have __sync_compare_and_swap(int *, int, int).])
+fi])# PGAC_HAVE_GCC__SYNC_INT32_CAS
+
+# PGAC_HAVE_GCC__SYNC_INT64_CAS
+# -------------------------
+# Check if the C compiler understands __sync_compare_and_swap() for 64bit
+# types, and define HAVE_GCC__SYNC_INT64_CAS if so.
+AC_DEFUN([PGAC_HAVE_GCC__SYNC_INT64_CAS],
+[AC_CACHE_CHECK(for builtin __sync int64 atomic operations, pgac_cv_gcc_sync_int64_cas,
+[AC_TRY_LINK([],
+  [PG_INT64_TYPE lock = 0;
+   __sync_val_compare_and_swap(&lock, 0, (PG_INT64_TYPE) 37);],
+  [pgac_cv_gcc_sync_int64_cas="yes"],
+  [pgac_cv_gcc_sync_int64_cas="no"])])
+if test x"$pgac_cv_gcc_sync_int64_cas" = x"yes"; then
+  AC_DEFINE(HAVE_GCC__SYNC_INT64_CAS, 1, [Define to 1 if you have __sync_compare_and_swap(int64 *, int64, int64).])
+fi])# PGAC_HAVE_GCC__SYNC_INT64_CAS
+
+# PGAC_HAVE_GCC__ATOMIC_INT32_CAS
+# -------------------------
+
+# Check if the C compiler understands __atomic_compare_exchange_n() for 32bit
+# types, and define HAVE_GCC__ATOMIC_INT32_CAS if so.
+AC_DEFUN([PGAC_HAVE_GCC__ATOMIC_INT32_CAS],
+[AC_CACHE_CHECK(for builtin __atomic int32 atomic operations, pgac_cv_gcc_atomic_int32_cas,
+[AC_TRY_LINK([],
+  [int val = 0;
+   int expect = 0;
+   __atomic_compare_exchange_n(&val, &expect, 37, 0, __ATOMIC_SEQ_CST, __ATOMIC_RELAXED);],
+  [pgac_cv_gcc_atomic_int32_cas="yes"],
+  [pgac_cv_gcc_atomic_int32_cas="no"])])
+if test x"$pgac_cv_gcc_atomic_int32_cas" = x"yes"; then
+  AC_DEFINE(HAVE_GCC__ATOMIC_INT32_CAS, 1, [Define to 1 if you have __atomic_compare_exchange_n(int *, int *, int).])
+fi])# PGAC_HAVE_GCC__ATOMIC_INT32_CAS
diff --git a/configure.in b/configure.in
index 304399e..1fab189 100644
--- a/configure.in
+++ b/configure.in
@@ -19,7 +19,7 @@ m4_pattern_forbid(^PGAC_)dnl to catch undefined macros
 
 AC_INIT([PostgreSQL], [9.4devel], [pgsql-bugs@postgresql.org])
 
-m4_if(m4_defn([m4_PACKAGE_VERSION]), [2.63], [], [m4_fatal([Autoconf version 2.63 is required.
+m4_if(m4_defn([m4_PACKAGE_VERSION]), [2.69], [], [m4_fatal([Autoconf version 2.63 is required.
 Untested combinations of 'autoconf' and PostgreSQL versions are not
 recommended.  You can remove the check from 'configure.in' but it is then
 your responsibility whether the result works or not.])])
@@ -1453,17 +1453,6 @@ fi
 AC_CHECK_FUNCS([strtoll strtoq], [break])
 AC_CHECK_FUNCS([strtoull strtouq], [break])
 
-AC_CACHE_CHECK([for builtin locking functions], pgac_cv_gcc_int_atomics,
-[AC_TRY_LINK([],
-  [int lock = 0;
-   __sync_lock_test_and_set(&lock, 1);
-   __sync_lock_release(&lock);],
-  [pgac_cv_gcc_int_atomics="yes"],
-  [pgac_cv_gcc_int_atomics="no"])])
-if test x"$pgac_cv_gcc_int_atomics" = x"yes"; then
-  AC_DEFINE(HAVE_GCC_INT_ATOMICS, 1, [Define to 1 if you have __sync_lock_test_and_set(int *) and friends.])
-fi
-
 # Lastly, restore full LIBS list and check for readline/libedit symbols
 LIBS="$LIBS_including_readline"
 
@@ -1738,6 +1727,12 @@ AC_CHECK_TYPES([int8, uint8, int64, uint64], [], [],
 # C, but is missing on some old platforms.
 AC_CHECK_TYPES(sig_atomic_t, [], [], [#include <signal.h>])
 
+# Check for various atomic operations now that we have checked how to declare
+# 64bit integers.
+PGAC_HAVE_GCC__SYNC_INT32_TAS
+PGAC_HAVE_GCC__SYNC_INT32_CAS
+PGAC_HAVE_GCC__SYNC_INT64_CAS
+PGAC_HAVE_GCC__ATOMIC_INT32_CAS
 
 if test "$PORTNAME" != "win32"
 then
diff --git a/src/backend/storage/lmgr/Makefile b/src/backend/storage/lmgr/Makefile
index e12a854..be95d50 100644
--- a/src/backend/storage/lmgr/Makefile
+++ b/src/backend/storage/lmgr/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/storage/lmgr
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = lmgr.o lock.o proc.o deadlock.o lwlock.o spin.o s_lock.o predicate.o
+OBJS = atomics.o lmgr.o lock.o proc.o deadlock.o lwlock.o spin.o s_lock.o predicate.o
 
 include $(top_srcdir)/src/backend/common.mk
 
diff --git a/src/backend/storage/lmgr/atomics.c b/src/backend/storage/lmgr/atomics.c
new file mode 100644
index 0000000..af95153
--- /dev/null
+++ b/src/backend/storage/lmgr/atomics.c
@@ -0,0 +1,26 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics.c
+ *	   Non-Inline parts of the atomics implementation
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/lmgr/atomics.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+/*
+ * We want the functions below to be inline; but if the compiler doesn't
+ * support that, fall back on providing them as regular functions.	See
+ * STATIC_IF_INLINE in c.h.
+ */
+#define ATOMICS_INCLUDE_DEFINITIONS
+
+#include "storage/atomics.h"
+
+#ifdef PG_USE_ATOMICS_EMULATION
+#endif
diff --git a/src/backend/storage/lmgr/spin.c b/src/backend/storage/lmgr/spin.c
index f054be8..469dd84 100644
--- a/src/backend/storage/lmgr/spin.c
+++ b/src/backend/storage/lmgr/spin.c
@@ -86,14 +86,6 @@ s_unlock_sema(volatile slock_t *lock)
 	PGSemaphoreUnlock((PGSemaphore) lock);
 }
 
-bool
-s_lock_free_sema(volatile slock_t *lock)
-{
-	/* We don't currently use S_LOCK_FREE anyway */
-	elog(ERROR, "spin.c does not support S_LOCK_FREE()");
-	return false;
-}
-
 int
 tas_sema(volatile slock_t *lock)
 {
diff --git a/src/include/c.h b/src/include/c.h
index 6e19c6d..347f394 100644
--- a/src/include/c.h
+++ b/src/include/c.h
@@ -866,6 +866,13 @@ typedef NameData *Name;
 #define STATIC_IF_INLINE
 #endif   /* PG_USE_INLINE */
 
+#ifdef PG_USE_INLINE
+#define STATIC_IF_INLINE_DECLARE static inline
+#else
+#define STATIC_IF_INLINE_DECLARE extern
+#endif   /* PG_USE_INLINE */
+
+
 /* ----------------------------------------------------------------
  *				Section 8:	random stuff
  * ----------------------------------------------------------------
diff --git a/src/include/storage/atomics-arch-alpha.h b/src/include/storage/atomics-arch-alpha.h
new file mode 100644
index 0000000..06b7125
--- /dev/null
+++ b/src/include/storage/atomics-arch-alpha.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-alpha.h
+ *	  Atomic operations considerations specific to Alpha
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-alpha.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#if defined(__GNUC__)
+/*
+ * Unlike all other known architectures, Alpha allows dependent reads to be
+ * reordered, but we don't currently find it necessary to provide a conditional
+ * read barrier to cover that case.  We might need to add that later.
+ */
+#define pg_memory_barrier()		__asm__ __volatile__ ("mb" : : : "memory")
+#define pg_read_barrier()		__asm__ __volatile__ ("rmb" : : : "memory")
+#define pg_write_barrier()		__asm__ __volatile__ ("wmb" : : : "memory")
+#endif
diff --git a/src/include/storage/atomics-arch-amd64.h b/src/include/storage/atomics-arch-amd64.h
new file mode 100644
index 0000000..90f4abc
--- /dev/null
+++ b/src/include/storage/atomics-arch-amd64.h
@@ -0,0 +1,56 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-amd64.h
+ *	  Atomic operations considerations specific to intel/amd x86-64
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-amd64.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * x86_64 has similar ordering characteristics to i386.
+ *
+ * Technically, some x86-ish chips support uncached memory access and/or
+ * special instructions that are weakly ordered.  In those cases we'd need
+ * the read and write barriers to be lfence and sfence.  But since we don't
+ * do those things, a compiler barrier should be enough.
+ */
+#if defined(__INTEL_COMPILER)
+#	define pg_memory_barrier_impl()		_mm_mfence()
+#elif defined(__GNUC__)
+#	define pg_memory_barrier_impl()		\
+	__asm__ __volatile__ ("lock; addl $0,0(%%rsp)" : : : "memory")
+#endif
+
+#define pg_read_barrier_impl()		pg_compiler_barrier()
+#define pg_write_barrier_impl()		pg_compiler_barrier()
+
+#if defined(__GNUC__)
+
+#ifndef PG_HAS_SPIN_DELAY
+#define PG_HAS_SPIN_DELAY
+static __inline__ void
+pg_spin_delay_impl(void)
+{
+	/*
+	 * Adding a PAUSE in the spin delay loop is demonstrably a no-op on
+	 * Opteron, but it may be of some use on EM64T, so we keep it.
+	 */
+	__asm__ __volatile__(
+		" rep; nop			\n");
+}
+#endif
+
+#elif defined(WIN32_ONLY_COMPILER)
+static __forceinline void
+pg_spin_delay_impl(void)
+{
+	_mm_pause();
+}
+#endif
diff --git a/src/include/storage/atomics-arch-arm.h b/src/include/storage/atomics-arch-arm.h
new file mode 100644
index 0000000..a00f8f5
--- /dev/null
+++ b/src/include/storage/atomics-arch-arm.h
@@ -0,0 +1,29 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-arm.h
+ *	  Atomic operations considerations specific to ARM
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-arm.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/* intentionally no include guards, should only be included by atomics.h */
+#ifndef INSIDE_ATOMICS_H
+#error "should be included via atomics.h"
+#endif
+
+
+/*
+ * FIXME: Disable 32bit atomics for ARMs before v6 or error out
+ */
+
+/*
+ * 64 bit atomics on arm are implemented using kernel fallbacks and might be
+ * slow, so disable entirely for now.
+ */
+#define PG_DISABLE_64_BIT_ATOMICS
diff --git a/src/include/storage/atomics-arch-hppa.h b/src/include/storage/atomics-arch-hppa.h
new file mode 100644
index 0000000..8913801
--- /dev/null
+++ b/src/include/storage/atomics-arch-hppa.h
@@ -0,0 +1,17 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-alpha.h
+ *	  Atomic operations considerations specific to HPPA
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-hppa.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/* HPPA doesn't do either read or write reordering */
+#define pg_memory_barrier_impl()		pg_compiler_barrier_impl()
diff --git a/src/include/storage/atomics-arch-i386.h b/src/include/storage/atomics-arch-i386.h
new file mode 100644
index 0000000..151a14f
--- /dev/null
+++ b/src/include/storage/atomics-arch-i386.h
@@ -0,0 +1,96 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-i386.h
+ *	  Atomic operations considerations specific to intel x86
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-i386.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * i386 does not allow loads to be reordered with other loads, or stores to be
+ * reordered with other stores, but a load can be performed before a subsequent
+ * store.
+ *
+ * "lock; addl" has worked for longer than "mfence".
+ */
+
+#if defined(__INTEL_COMPILER)
+#define pg_memory_barrier_impl()		_mm_mfence()
+#elif defined(__GNUC__)
+#define pg_memory_barrier_impl()		\
+	__asm__ __volatile__ ("lock; addl $0,0(%%rsp)" : : : "memory")
+#endif
+
+#define pg_read_barrier_impl()		pg_compiler_barrier()
+#define pg_write_barrier_impl()		pg_compiler_barrier()
+
+#if defined(__GNUC__)
+
+/*
+ * Implement TAS if not available as a __sync builtin on very old gcc
+ * versions.
+ */
+#ifndef HAVE_GCC__SYNC_INT32_TAS
+static __inline__ int
+tas(volatile slock_t *lock)
+	register slock_t _res = 1;
+
+	__asm__ __volatile__(
+		"	lock			\n"
+		"	xchgb	%0,%1	\n"
+:		"+q"(_res), "+m"(*lock)
+:
+:		"memory", "cc");
+	return (int) _res;
+}
+#endif
+
+
+#ifndef PG_HAS_SPIN_DELAY
+#define PG_HAS_SPIN_DELAY
+static __inline__ void
+pg_spin_delay_impl(void)
+{
+	/*
+	 * This sequence is equivalent to the PAUSE instruction ("rep" is
+	 * ignored by old IA32 processors if the following instruction is
+	 * not a string operation); the IA-32 Architecture Software
+	 * Developer's Manual, Vol. 3, Section 7.7.2 describes why using
+	 * PAUSE in the inner loop of a spin lock is necessary for good
+	 * performance:
+	 *
+	 *     The PAUSE instruction improves the performance of IA-32
+	 *     processors supporting Hyper-Threading Technology when
+	 *     executing spin-wait loops and other routines where one
+	 *     thread is accessing a shared lock or semaphore in a tight
+	 *     polling loop. When executing a spin-wait loop, the
+	 *     processor can suffer a severe performance penalty when
+	 *     exiting the loop because it detects a possible memory order
+	 *     violation and flushes the core processor's pipeline. The
+	 *     PAUSE instruction provides a hint to the processor that the
+	 *     code sequence is a spin-wait loop. The processor uses this
+	 *     hint to avoid the memory order violation and prevent the
+	 *     pipeline flush. In addition, the PAUSE instruction
+	 *     de-pipelines the spin-wait loop to prevent it from
+	 *     consuming execution resources excessively.
+	 */
+	__asm__ __volatile__(
+		" rep; nop			\n");
+}
+#endif
+
+#elif defined(WIN32_ONLY_COMPILER)
+static __forceinline void
+pg_spin_delay_impl(void)
+{
+	/* See comment for gcc code. Same code, MASM syntax */
+	__asm rep nop;
+}
+#endif
diff --git a/src/include/storage/atomics-arch-ia64.h b/src/include/storage/atomics-arch-ia64.h
new file mode 100644
index 0000000..2863431
--- /dev/null
+++ b/src/include/storage/atomics-arch-ia64.h
@@ -0,0 +1,27 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-ia64.h
+ *	  Atomic operations considerations specific to intel itanium
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-ia64.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * Itanium is weakly ordered, so read and write barriers require a full
+ * fence.
+ */
+#if defined(__INTEL_COMPILER)
+#	define pg_memory_barrier_impl()		__mf()
+#elif defined(__GNUC__)
+#	define pg_memory_barrier_impl()		__asm__ __volatile__ ("mf" : : : "memory")
+#elif defined(__hpux)
+#	define pg_compiler_barrier_impl()	_Asm_sched_fence()
+#	define pg_memory_barrier_impl()		_Asm_mf()
+#endif
diff --git a/src/include/storage/atomics-arch-ppc.h b/src/include/storage/atomics-arch-ppc.h
new file mode 100644
index 0000000..f785c99
--- /dev/null
+++ b/src/include/storage/atomics-arch-ppc.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-arch-ppc.h
+ *	  Atomic operations considerations specific to PowerPC
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * NOTES:
+ *
+ * src/include/storage/atomics-arch-ppc.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#if defined(__GNUC__)
+/*
+ * lwsync orders loads with respect to each other, and similarly with stores.
+ * But a load can be performed before a subsequent store, so sync must be used
+ * for a full memory barrier.
+ */
+#define pg_memory_barrier_impl()	__asm__ __volatile__ ("sync" : : : "memory")
+#define pg_read_barrier_impl()		__asm__ __volatile__ ("lwsync" : : : "memory")
+#define pg_write_barrier_impl()		__asm__ __volatile__ ("lwsync" : : : "memory")
+#endif
diff --git a/src/include/storage/atomics-generic-acc.h b/src/include/storage/atomics-generic-acc.h
new file mode 100644
index 0000000..b1c9200
--- /dev/null
+++ b/src/include/storage/atomics-generic-acc.h
@@ -0,0 +1,99 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-generic-acc.h
+ *	  Atomic operations support when using HPs acc on HPUX
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * NOTES:
+ *
+ * Documentation:
+ * * inline assembly for Itanium-based HP-UX:
+ *   http://h21007.www2.hp.com/portal/download/files/unprot/Itanium/inline_assem_ERS.pdf
+ * * Implementing Spinlocks on the Intel ® Itanium ® Architecture and PA-RISC
+ *   http://h21007.www2.hp.com/portal/download/files/unprot/itanium/spinlocks.pdf
+ *
+ * Itanium only supports a small set of numbers (6, -8, -4, -1, 1, 4, 8, 16)
+ * for atomic add/sub, so we just implement everything but compare_exchange via
+ * the compare_exchange fallbacks in atomics-generic.h.
+ *
+ * src/include/storage/atomics-generic-acc.h
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include <machine/sys/inline.h>
+
+/* IA64 always has 32/64 bit atomics */
+
+typedef struct pg_atomic_uint32
+{
+	volatile uint32 value;
+} pg_atomic_uint32;
+
+typedef pg_atomic_uint32 pg_atomic_flag;
+
+typedef struct pg_atomic_uint64
+{
+	volatile uint64 value;
+} pg_atomic_uint64;
+
+/* declare outside the defined(PG_USE_INLINE) for visibility */
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+
+
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+#define MINOR_FENCE (_Asm_fence) (_UP_CALL_FENCE | _UP_SYS_FENCE | \
+								 _DOWN_CALL_FENCE | _DOWN_SYS_FENCE )
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32_impl(volatile pg_atomic_uint32 *ptr,
+									uint32 *expected, uint32 newval)
+{
+	bool	ret;
+	uint32	current;
+
+	_Asm_mov_to_ar(_AREG_CCV, *expected, MINOR_FENCE);
+	/*
+	 * We want a barrier, not just release/acquire semantics.
+	 */
+	_Asm_mf();
+	/*
+	 * Notes:
+	 * DOWN_MEM_FENCE | _UP_MEM_FENCE prevents reordering by the compiler
+	 */
+	current =  _Asm_cmpxchg(_SZ_W, /* word */
+							_SEM_REL,
+							&ptr->value,
+							newval, _LDHINT_NONE,
+							_DOWN_MEM_FENCE | _UP_MEM_FENCE);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64_impl(volatile pg_atomic_uint64 *ptr,
+									uint64 *expected, uint64 newval)
+{
+	bool	ret;
+	uint64	current;
+
+	_Asm_mov_to_ar(_AREG_CCV, *expected, MINOR_FENCE);
+	_Asm_mf();
+	current =  _Asm_cmpxchg(_SZ_D, /* doubleword */
+							_SEM_REL,
+							&ptr->value,
+							newval, _LDHINT_NONE,
+							_DOWN_MEM_FENCE | _UP_MEM_FENCE);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+#undef MINOR_FENCE
+
+#endif /* defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS) */
diff --git a/src/include/storage/atomics-generic-gcc.h b/src/include/storage/atomics-generic-gcc.h
new file mode 100644
index 0000000..0cc0b16
--- /dev/null
+++ b/src/include/storage/atomics-generic-gcc.h
@@ -0,0 +1,174 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-generic-gcc.h
+ *	  Atomic operations, implemented using gcc (or compatible) intrinsics.
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * NOTES:
+ *
+ * Documentation:
+ * * Legacy __sync Built-in Functions for Atomic Memory Access
+ *   http://gcc.gnu.org/onlinedocs/gcc-4.8.2/gcc/_005f_005fsync-Builtins.html
+ * * Built-in functions for memory model aware atomic operations
+ *   http://gcc.gnu.org/onlinedocs/gcc-4.8.2/gcc/_005f_005fatomic-Builtins.html
+ *
+ * src/include/storage/atomics-generic-gcc.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/* intentionally no include guards, should only be included by atomics.h */
+#ifndef INSIDE_ATOMICS_H
+#error "should be included via atomics.h"
+#endif
+
+#define pg_compiler_barrier_impl()	__asm__ __volatile__("" ::: "memory");
+
+/*
+ * If we're on GCC 4.1.0 or higher, we should be able to get a memory
+ * barrier out of this compiler built-in.  But we prefer to rely on our
+ * own definitions where possible, and use this only as a fallback.
+ */
+#if !defined(pg_memory_barrier_impl)
+#	if defined(HAVE_GCC__PG_ATOMIC_INT64_CAS)
+#		define pg_memory_barrier_impl()		__atomic_thread_fence(__ATOMIC_SEQ_CST)
+#	elif (__GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 1))
+#		define pg_memory_barrier_impl()		__sync_synchronize()
+#	endif
+#endif /* !defined(pg_memory_barrier_impl) */
+
+#if !defined(pg_read_barrier_impl) && defined(HAVE_GCC__PG_ATOMIC_INT64_CAS)
+#		define pg_read_barrier_impl()		__atomic_thread_fence(__ATOMIC_ACQUIRE)
+#endif
+
+#if !defined(pg_write_barrier_impl) && defined(HAVE_GCC__PG_ATOMIC_INT64_CAS)
+#		define pg_write_barrier_impl()		__atomic_thread_fence(__ATOMIC_RELEASE)
+#endif
+
+#ifdef HAVE_GCC__SYNC_INT32_TAS
+typedef struct pg_atomic_flag
+{
+	volatile uint32 value;
+} pg_atomic_flag;
+
+#define PG_ATOMIC_INIT_FLAG(flag) {0}
+#endif
+
+/* declare outside the defined(PG_USE_INLINE) for visibility */
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+
+typedef struct pg_atomic_uint32
+{
+	volatile uint32 value;
+} pg_atomic_uint32;
+
+#ifdef HAVE_GCC__PG_ATOMIC_INT64_CAS
+/* declare outside the defined(PG_USE_INLINE) for visibility */
+#	define PG_HAVE_PG_ATOMIC_COMPARE_EXCHANGE_U64
+typedef struct pg_atomic_uint64
+{
+	volatile uint64 value;
+} pg_atomic_uint64;
+#endif /* HAVE_GCC__PG_ATOMIC_INT64_CAS */
+
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+#if !defined(PG_HAS_ATOMIC_TEST_SET_FLAG) && defined(HAVE_GCC__SYNC_INT32_TAS)
+#define PG_HAS_ATOMIC_TEST_SET_FLAG
+STATIC_IF_INLINE bool
+pg_atomic_test_set_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	uint32 ret;
+	/* XXX: only a acquire barrier, make it a full one for now */
+	pg_memory_barrier_impl();
+	/* some platform only support a 1 here */
+	ret = __sync_lock_test_and_set(&ptr->value, 1);
+	return ret == 0;
+}
+
+#define PG_HAS_ATOMIC_UNLOCKED_TEST_FLAG
+STATIC_IF_INLINE bool
+pg_atomic_unlocked_test_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	return ptr->value;
+}
+
+#define PG_HAS_ATOMIC_CLEAR_FLAG
+STATIC_IF_INLINE void
+pg_atomic_clear_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	/* XXX: only a release barrier, make it a full one for now */
+	pg_memory_barrier_impl();
+	__sync_lock_release(&ptr->value);
+}
+
+#define PG_HAS_ATOMIC_INIT_FLAG
+STATIC_IF_INLINE void
+pg_atomic_init_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	pg_atomic_clear_flag_impl(ptr);
+}
+
+#endif /* !defined(PG_HAS_ATOMIC_TEST_SET_FLAG) && defined(HAVE_GCC__SYNC_INT32_TAS) */
+
+#if defined(HAVE_GCC__ATOMIC_INT32_CAS)
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32_impl(volatile pg_atomic_uint32 *ptr,
+									uint32 *expected, uint32 newval)
+{
+	return __atomic_compare_exchange_n(&ptr->value, expected, newval, false,
+									   __ATOMIC_SEQ_CST, __ATOMIC_SEQ_CST);
+}
+
+#elif defined(HAVE_GCC__SYNC_INT32_CAS)
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32_impl(volatile pg_atomic_uint32 *ptr,
+									uint32 *expected, uint32 newval)
+{
+	bool	ret;
+	uint32	current;
+	current = __sync_val_compare_and_swap(&ptr->value, *expected, newval);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+#else
+#	error "strange configuration"
+#endif
+
+#ifdef HAVE_GCC__ATOMIC_INT64_CAS
+
+/* if __atomic is supported for 32 it's also supported for 64bit */
+#if defined(HAVE_GCC__ATOMIC_INT32_CAS)
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64_impl(volatile pg_atomic_uint64 *ptr,
+									uint64 *expected, uint64 newval)
+{
+	return __atomic_compare_exchange_n(&ptr->value, expected, newval, false,
+									   __ATOMIC_SEQ_CST, __ATOMIC_SEQ_CST);
+}
+
+#elif defined(HAVE_GCC__SYNC_INT64_CAS)
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64_impl(volatile pg_atomic_uint64 *ptr,
+									uint64 *expected, uint64 newval)
+{
+	bool	ret;
+	uint64	current;
+	current = __sync_val_compare_and_swap(&ptr->value, *expected, newval);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+#else
+#	error "strange configuration"
+#endif
+
+#endif /* HAVE_GCC__ATOMIC_INT64_CAS */
+
+#endif /* defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS) */
diff --git a/src/include/storage/atomics-generic-msvc.h b/src/include/storage/atomics-generic-msvc.h
new file mode 100644
index 0000000..724d9ee
--- /dev/null
+++ b/src/include/storage/atomics-generic-msvc.h
@@ -0,0 +1,67 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-generic-msvc.h
+ *	  Atomic operations support when using MSVC
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * NOTES:
+ *
+ * Documentation:
+ * * Interlocked Variable Access
+ *   http://msdn.microsoft.com/en-us/library/ms684122%28VS.85%29.aspx
+ *
+ * src/include/storage/atomics-generic-msvc.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#include <intrin.h>
+
+/* intentionally no include guards, should only be included by atomics.h */
+#ifndef INSIDE_ATOMICS_H
+#error "should be included via atomics.h"
+#endif
+
+/* Should work on both MSVC and Borland. */
+#pragma intrinsic(_ReadWriteBarrier)
+#define pg_compiler_barrier_impl()	_ReadWriteBarrier()
+
+#ifndef pg_memory_barrier_impl
+#define pg_memory_barrier_impl()		MemoryBarrier()
+#endif
+
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32_impl(volatile pg_atomic_uint32 *ptr,
+									uint32 *expected, uint32 newval)
+{
+	bool	ret;
+	uint32	current;
+	current = InterlockedCompareExchange(&ptr->value, newval, expected);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+/*
+ * FIXME: Only available on Vista/2003 onwards, add test for that.
+ *
+ * Only supported on >486, but we require XP as a minimum baseline, which
+ * doesn't support the 486, so we don't need to add checks for that case.
+ */
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64_impl(volatile pg_atomic_uint64 *ptr,
+									uint64 *expected, uint64 newval)
+{
+	bool	ret;
+	uint32	current;
+	current = InterlockedCompareExchange64(&ptr->value, newval, expected);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+#endif /* defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS) */
diff --git a/src/include/storage/atomics-generic-sunpro.h b/src/include/storage/atomics-generic-sunpro.h
new file mode 100644
index 0000000..d512409
--- /dev/null
+++ b/src/include/storage/atomics-generic-sunpro.h
@@ -0,0 +1,66 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-generic-sunpro.h
+ *	  Atomic operations for solaris' CC
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * NOTES:
+ *
+ * Documentation:
+ * * manpage for atomic_cas(3C)
+ *   http://www.unix.com/man-page/opensolaris/3c/atomic_cas/
+ *   http://docs.oracle.com/cd/E23824_01/html/821-1465/atomic-cas-3c.html
+ *
+ * src/include/storage/atomics-generic-sunpro.h
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include <atomic.h>
+
+typedef struct pg_atomic_uint32
+{
+	volatile uint32 value;
+} pg_atomic_uint32;
+
+typedef pg_atomic_uint32 pg_atomic_flag;
+
+typedef struct pg_atomic_uint64
+{
+	volatile uint64 value;
+} pg_atomic_uint64;
+
+/* declare outside the defined(PG_USE_INLINE) for visibility */
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32_impl(volatile pg_atomic_uint32 *ptr,
+									uint32 *expected, uint32 newval)
+{
+	bool	ret;
+	uint64	current;
+
+	current = atomic_cas_32(&ptr->value, *expected, newval);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64_impl(volatile pg_atomic_uint64 *ptr,
+									uint64 *expected, uint64 newval)
+{
+	bool	ret;
+	uint64	current;
+
+	current = atomic_cas_64(&ptr->value, *expected, newval);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+#endif
diff --git a/src/include/storage/atomics-generic-xlc.h b/src/include/storage/atomics-generic-xlc.h
new file mode 100644
index 0000000..6a47fe2
--- /dev/null
+++ b/src/include/storage/atomics-generic-xlc.h
@@ -0,0 +1,84 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-generic-xlc.h
+ *	  Atomic operations for IBM's CC
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * NOTES:
+ *
+ * Documentation:
+ * * Synchronization and atomic built-in functions
+ *   http://publib.boulder.ibm.com/infocenter/lnxpcomp/v8v101/topic/com.ibm.xlcpp8l.doc/compiler/ref/bif_sync.htm
+ *
+ * src/include/storage/atomics-generic-xlc.h
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include <atomic.h>
+
+typedef struct pg_atomic_uint32
+{
+	volatile uint32 value;
+} pg_atomic_uint32;
+
+typedef pg_atomic_uint32 pg_atomic_flag;
+
+typedef struct pg_atomic_uint64
+{
+	volatile uint64 value;
+} pg_atomic_uint64;
+
+/* declare outside the defined(PG_USE_INLINE) for visibility */
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+
+/* FIXME: check for 64bit mode, only supported there */
+#define PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32_impl(volatile pg_atomic_uint32 *ptr,
+									uint32 *expected, uint32 newval)
+{
+	bool	ret;
+	uint64	current;
+
+	/*
+	 * xlc's documentation tells us:
+	 * "If __compare_and_swap is used as a locking primitive, insert a call to
+	 * the __isync built-in function at the start of any critical sections."
+	 */
+	__isync();
+
+	/*
+	 * XXX: __compare_and_swap is defined to take signed parameters, but that
+	 * shouldn't matter since we don't perform any arithmetic operations.
+	 */
+	current = (uint32)__compare_and_swap((volatile int*)ptr->value,
+										 (int)*expected, (int)newval);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+
+#ifdef PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64_impl(volatile pg_atomic_uint64 *ptr,
+									uint64 *expected, uint64 newval)
+{
+	bool	ret;
+	uint64	current;
+
+	__isync();
+
+	current = (uint64)__compare_and_swaplp((volatile long*)ptr->value,
+										   (long)*expected, (long)newval);
+	ret = current == *expected;
+	*expected = current;
+	return ret;
+}
+#endif /* defined(PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64) */
+
+#endif
diff --git a/src/include/storage/atomics-generic.h b/src/include/storage/atomics-generic.h
new file mode 100644
index 0000000..eff8495
--- /dev/null
+++ b/src/include/storage/atomics-generic.h
@@ -0,0 +1,402 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics-generic.h
+ *	  Atomic operations.
+ *
+ * Portions Copyright (c) 2013, PostgreSQL Global Development Group
+ *
+ * src/include/storage/atomics-generic.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+/* intentionally no include guards, should only be included by atomics.h */
+#ifndef INSIDE_ATOMICS_H
+#	error "should be included via atomics.h"
+#endif
+
+/* Need a way to implement spinlocks */
+#if !defined(PG_HAVE_ATOMIC_TEST_SET_FLAG) && \
+	!defined(PG_HAVE_ATOMIC_EXCHANGE_U32) && \
+	!defined(PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32)
+#	error "postgres requires atomic ops for locking to be provided"
+#endif
+
+#ifndef pg_memory_barrier_impl
+#	error "postgres requires a memory barrier implementation"
+#endif
+
+#ifndef pg_compiler_barrier_impl
+#	error "postgres requires a compiler barrier implementation"
+#endif
+
+/*
+ * If read or write barriers are undefined, we upgrade them to full memory
+ * barriers.
+ */
+#if !defined(pg_read_barrier_impl)
+#	define pg_read_barrier_impl pg_memory_barrier_impl
+#endif
+#if !defined(pg_read_barrier_impl)
+#	define pg_write_barrier_impl pg_memory_barrier_impl
+#endif
+
+#ifndef PG_HAS_SPIN_DELAY
+#define PG_HAS_SPIN_DELAY
+#define pg_spin_delay_impl()	((void)0)
+#endif
+
+
+/* provide fallback */
+#ifndef PG_HAS_ATOMIC_TEST_SET_FLAG
+typedef pg_atomic_uint32 pg_atomic_flag;
+#define PG_ATOMIC_INIT_FLAG(flag) {0}
+#endif
+
+#if !defined(PG_DISABLE_64_BIT_ATOMICS) && defined(PG_HAS_ATOMIC_COMPARE_EXCHANGE_U64)
+#	define PG_HAVE_64_BIT_ATOMICS
+#endif
+
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+#ifndef PG_HAS_ATOMIC_READ_U32
+#define PG_HAS_ATOMIC_READ_U32
+STATIC_IF_INLINE uint32
+pg_atomic_read_u32_impl(volatile pg_atomic_uint32 *ptr)
+{
+	return ptr->value;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_WRITE_U32
+#define PG_HAS_ATOMIC_WRITE_U32
+STATIC_IF_INLINE void
+pg_atomic_write_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 val)
+{
+	ptr->value = val;
+}
+#endif
+
+/*
+ * provide fallback for test_and_set using atomic_exchange if available
+ */
+#if !defined(PG_HAS_ATOMIC_TEST_SET_FLAG) && defined(PG_HAVE_ATOMIC_EXCHANGE_U32)
+
+#define PG_HAS_ATOMIC_INIT_FLAG
+STATIC_IF_INLINE void
+pg_atomic_init_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	pg_atomic_write_u32_impl(ptr, 0);
+}
+
+#define PG_HAS_ATOMIC_TEST_SET_FLAG
+STATIC_IF_INLINE bool
+pg_atomic_test_set_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	return pg_atomic_exchange_u32_impl(ptr, &value, 1) == 0;
+}
+
+#define PG_HAS_ATOMIC_UNLOCKED_TEST_FLAG
+STATIC_IF_INLINE bool
+pg_atomic_unlocked_test_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	return (bool) pg_atomic_read_u32_impl(ptr, &value);
+}
+
+
+#define PG_HAS_ATOMIC_CLEAR_FLAG
+STATIC_IF_INLINE void
+pg_atomic_clear_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	/* XXX: release semantics suffice? */
+	pg_memory_barrier();
+	pg_atomic_write_u32_impl(ptr, &value, 0);
+}
+
+/*
+ * provide fallback for test_and_set using atomic_compare_exchange if
+ * available.
+ */
+#elif !defined(PG_HAS_ATOMIC_TEST_SET_FLAG) && defined(PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32)
+
+#define PG_HAS_ATOMIC_INIT_FLAG
+STATIC_IF_INLINE void
+pg_atomic_init_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	pg_atomic_write_u32_impl(ptr, 0);
+}
+
+#define PG_HAS_ATOMIC_TEST_SET_FLAG
+STATIC_IF_INLINE bool
+pg_atomic_test_set_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	uint32 value = 0;
+	return pg_atomic_compare_exchange_u32_impl(ptr, &value, 1);
+}
+
+#define PG_HAS_ATOMIC_UNLOCKED_TEST_FLAG
+STATIC_IF_INLINE bool
+pg_atomic_unlocked_test_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	return (bool) pg_atomic_read_u32_impl(ptr, &value);
+}
+
+#define PG_HAS_ATOMIC_CLEAR_FLAG
+STATIC_IF_INLINE void
+pg_atomic_clear_flag_impl(volatile pg_atomic_flag *ptr)
+{
+	/* XXX: release semantics suffice? */
+	pg_memory_barrier();
+	pg_atomic_write_u32_impl(ptr, &value, 0);
+}
+
+#elif !defined(PG_HAS_ATOMIC_TEST_SET_FLAG)
+#	error "No pg_atomic_test_and_set provided"
+#endif /* !defined(PG_HAS_ATOMIC_TEST_SET_FLAG) */
+
+
+#ifndef PG_HAS_ATOMIC_INIT_U32
+#define PG_HAS_ATOMIC_INIT_U32
+STATIC_IF_INLINE void
+pg_atomic_init_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 val_)
+{
+	pg_atomic_write_u32_impl(ptr, val_);
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_EXCHANGE_U32
+#define PG_HAS_ATOMIC_EXCHANGE_U32
+STATIC_IF_INLINE uint32
+pg_atomic_exchange_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 xchg_)
+{
+	uint32 old;
+	while (true)
+	{
+		old = pg_atomic_read_u32_impl(ptr);
+		if (pg_atomic_compare_exchange_u32_impl(ptr, &old, xchg_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_ADD_U32
+#define PG_HAS_ATOMIC_FETCH_ADD_U32
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_add_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 add_)
+{
+	uint32 old;
+	while (true)
+	{
+		old = pg_atomic_read_u32_impl(ptr);
+		if (pg_atomic_compare_exchange_u32_impl(ptr, &old, old + add_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_SUB_U32
+#define PG_HAS_ATOMIC_FETCH_SUB_U32
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_sub_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 sub_)
+{
+	uint32 old;
+	while (true)
+	{
+		old = pg_atomic_read_u32_impl(ptr);
+		if (pg_atomic_compare_exchange_u32_impl(ptr, &old, old - sub_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_AND_U32
+#define PG_HAS_ATOMIC_FETCH_AND_U32
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_and_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 and_)
+{
+	uint32 old;
+	while (true)
+	{
+		old = pg_atomic_read_u32_impl(ptr);
+		if (pg_atomic_compare_exchange_u32_impl(ptr, &old, old & and_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_OR_U32
+#define PG_HAS_ATOMIC_FETCH_OR_U32
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_or_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 or_)
+{
+	uint32 old;
+	while (true)
+	{
+		old = pg_atomic_read_u32_impl(ptr);
+		if (pg_atomic_compare_exchange_u32_impl(ptr, &old, old | or_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_ADD_FETCH_U32
+#define PG_HAS_ATOMIC_ADD_FETCH_U32
+STATIC_IF_INLINE uint32
+pg_atomic_add_fetch_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 add_)
+{
+	return pg_atomic_fetch_add_u32_impl(ptr, add_) + add_;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_SUB_FETCH_U32
+#define PG_HAS_ATOMIC_SUB_FETCH_U32
+STATIC_IF_INLINE uint32
+pg_atomic_sub_fetch_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 sub_)
+{
+	return pg_atomic_fetch_sub_u32_impl(ptr, sub_) - sub_;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_ADD_UNTIL_U32
+#define PG_HAS_ATOMIC_FETCH_ADD_UNTIL_U32
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_add_until_u32_impl(volatile pg_atomic_uint32 *ptr, uint32 add_, uint32 until)
+{
+	uint32 old;
+	while (true)
+	{
+		uint32 new_val;
+		old = pg_atomic_read_u32_impl(ptr);
+		if (old >= until)
+			break;
+
+		new_val = old + add_;
+		if (new_val > until)
+			new_val = until;
+
+		if (pg_atomic_compare_exchange_u32_impl(ptr, &old, new_val))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifdef PG_HAS_ATOMIC_COMPARE_EXCHANGE_U64
+
+#ifndef PG_HAS_ATOMIC_INIT_U64
+#define PG_HAS_ATOMIC_INIT_U64
+STATIC_IF_INLINE void
+pg_atomic_init_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 val_)
+{
+	pg_atomic_write_u64_impl(ptr, val_);
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_READ_U64
+#define PG_HAS_ATOMIC_READ_U64
+STATIC_IF_INLINE uint64
+pg_atomic_read_u64_impl(volatile pg_atomic_uint64 *ptr)
+{
+	return ptr->value;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_WRITE_U64
+#define PG_HAS_ATOMIC_WRITE_U64
+STATIC_IF_INLINE void
+pg_atomic_write_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 val)
+{
+	ptr->value = val;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_ADD_U64
+#define PG_HAS_ATOMIC_FETCH_ADD_U64
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_add_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 add_)
+{
+	uint64 old;
+	while (true)
+	{
+		old = pg_atomic_read_u64_impl(ptr);
+		if (pg_atomic_compare_exchange_64_impl(ptr, &old, old + add_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_SUB_U64
+#define PG_HAS_ATOMIC_FETCH_SUB_U64
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_sub_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 sub_)
+{
+	uint64 old;
+	while (true)
+	{
+		old = pg_atomic_read_u64_impl(ptr);
+		if (pg_atomic_compare_exchange_64_impl(ptr, &old, old - sub_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_AND_U64
+#define PG_HAS_ATOMIC_FETCH_AND_U64
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_and_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 and_)
+{
+	uint64 old;
+	while (true)
+	{
+		old = pg_atomic_read_64_impl(ptr);
+		if (pg_atomic_compare_exchange_u64_impl(ptr, &old, old & and_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_FETCH_OR_U64
+#define PG_HAS_ATOMIC_FETCH_OR_U64
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_or_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 or_)
+{
+	uint64 old;
+	while (true)
+	{
+		old = pg_atomic_read_u64_impl(ptr);
+		if (pg_atomic_compare_exchange_64_impl(ptr, &old, old | or_))
+			break;
+	}
+	return old;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_ADD_FETCH_U64
+#define PG_HAS_ATOMIC_ADD_FETCH_U64
+STATIC_IF_INLINE
+pg_atomic_add_fetch_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 add_)
+{
+	return pg_atomic_fetch_add_u64_impl(ptr, add_) + add_;
+}
+#endif
+
+#ifndef PG_HAS_ATOMIC_SUB_FETCH_U64
+#define PG_HAS_ATOMIC_SUB_FETCH_U64
+STATIC_IF_INLINE
+pg_atomic_sub_fetch_u64_impl(volatile pg_atomic_uint64 *ptr, uint64 sub_)
+{
+	return pg_atomic_fetch_sub_u64_impl(ptr, sub_) - sub_;
+}
+#endif
+
+#endif /* PG_HAS_ATOMIC_COMPARE_EXCHANGE_U64 */
+
+#endif /* defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS) */
diff --git a/src/include/storage/atomics.h b/src/include/storage/atomics.h
new file mode 100644
index 0000000..39ed336
--- /dev/null
+++ b/src/include/storage/atomics.h
@@ -0,0 +1,543 @@
+/*-------------------------------------------------------------------------
+ *
+ * atomics.h
+ *	  Atomic operations.
+ *
+ * Hardware and compiler dependent functions for manipulating memory
+ * atomically and dealing with cache coherency. Used to implement
+ * locking facilities and replacements.
+ *
+ * To bring up postgres on a platform/compiler at the very least
+ * either one of
+ * * pg_atomic_test_set_flag(), pg_atomic_init_flag(), pg_atomic_clear_flag()
+ * * pg_atomic_compare_exchange_u32()
+ * * pg_atomic_exchange_u32()
+ * and
+ * * pg_compiler_barrier(), pg_write_barrier(), pg_read_barrier()
+ * need to be implemented. There exist generic, hardware independent,
+ * implementations for several compilers which might be sufficient,
+ * although possibly not optimal, for a new platform.
+ *
+ * Use higher level functionality (lwlocks, spinlocks, heavyweight
+ * locks) whenever possible. Writing correct code using these
+ * facilities is hard.
+ *
+ * For an introduction to using memory barriers within the PostgreSQL backend,
+ * see src/backend/storage/lmgr/README.barrier
+ *
+ * Portions Copyright (c) 1996-2013, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/storage/atomics.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef ATOMICS_H
+#define ATOMICS_H
+
+#define INSIDE_ATOMICS_H
+
+/*
+ * Architecture specific implementations/considerations.
+ */
+#if defined(__arm__) || defined(__arm) || \
+	defined(__aarch64__) || defined(__aarch64)
+#	include "storage/atomics-arch-arm.h"
+#endif
+
+#if defined(__i386__) || defined(__i386)
+#	include "storage/atomics-arch-i368.h"
+#elif defined(__x86_64__)
+#	include "storage/atomics-arch-amd64.h"
+#elif defined(__ia64__) || defined(__ia64)
+#	include "storage/atomics-arch-amd64.h"
+#elif defined(__ppc__) || defined(__powerpc__) || defined(__ppc64__) || defined(__powerpc64__)
+#	include "storage/atomics-arch-ppc.h"
+#elif defined(__alpha) || defined(__alpha__)
+#	include "storage/atomics-arch-alpha.h"
+#elif defined(__hppa) || defined(__hppa__)
+#	include "storage/atomics-arch-hppa.h"
+#endif
+
+/*
+ * Compiler specific, but architecture independent implementations
+ */
+
+/* gcc or compatible, including icc */
+#if defined(HAVE_GCC__ATOMIC_INT32_CAS) || defined(HAVE_GCC__SYNC_INT32_CAS)
+#	include "storage/atomics-generic-gcc.h"
+#elif defined(WIN32_ONLY_COMPILER)
+#	include "storage/atomics-generic-msvc.h"
+#elif defined(__hpux) && defined(__ia64) && !defined(__GNUC__)
+#	include "storage/atomics-generic-acc.h"
+#elif defined(__SUNPRO_C) && defined(__ia64) && !defined(__GNUC__)
+#	include "storage/atomics-generic-sunpro.h"
+#elif (defined(__IBMC__) || defined(__IBMCPP__)) && !defined(__GNUC__)
+#	include "storage/atomics-generic-xlc.h"
+#else
+#	error "unsupported compiler"
+#endif
+
+/* if we have spinlocks, but not atomic ops, emulate them */
+#if defined(PG_HAVE_ATOMIC_TEST_SET_FLAG) && \
+	!defined(PG_HAVE_ATOMIC_EXCHANGE_U32)
+#	define PG_USE_ATOMICS_EMULATION
+#	include "storage/atomics-generic-spinlock.h"
+#endif
+
+
+/*
+ * Provide fallback implementations for operations that aren't provided if
+ * possible.
+ */
+#include "storage/atomics-generic.h"
+
+/*
+ * pg_compiler_barrier - prevent the compiler from moving code
+ *
+ * A compiler barrier need not (and preferably should not) emit any actual
+ * machine code, but must act as an optimization fence: the compiler must not
+ * reorder loads or stores to main memory around the barrier.  However, the
+ * CPU may still reorder loads or stores at runtime, if the architecture's
+ * memory model permits this.
+ */
+#define pg_compiler_barrier()	pg_compiler_barrier_impl()
+
+/*
+ * pg_memory_barrier - prevent the CPU from reordering memory access
+ *
+ * A memory barrier must act as a compiler barrier, and in addition must
+ * guarantee that all loads and stores issued prior to the barrier are
+ * completed before any loads or stores issued after the barrier.  Unless
+ * loads and stores are totally ordered (which is not the case on most
+ * architectures) this requires issuing some sort of memory fencing
+ * instruction.
+ */
+#define pg_memory_barrier()	pg_memory_barrier_impl()
+
+/*
+ * pg_(read|write)_barrier - prevent the CPU from reordering memory access
+ *
+ * A read barrier must act as a compiler barrier, and in addition must
+ * guarantee that any loads issued prior to the barrier are completed before
+ * any loads issued after the barrier.	Similarly, a write barrier acts
+ * as a compiler barrier, and also orders stores.  Read and write barriers
+ * are thus weaker than a full memory barrier, but stronger than a compiler
+ * barrier.  In practice, on machines with strong memory ordering, read and
+ * write barriers may require nothing more than a compiler barrier.
+ */
+#define pg_read_barrier()	pg_read_barrier_impl()
+#define pg_write_barrier()	pg_write_barrier_impl()
+
+/*
+ * Spinloop delay -
+ */
+#define pg_spin_delay()	pg_spin_delay_impl()
+
+
+/*
+ * pg_atomic_init_flag - initialize lock
+ *
+ * No barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_init_flag(volatile pg_atomic_flag *ptr);
+
+/*
+ * pg_atomic_test_and_set_flag - TAS()
+ *
+ * Full barrier semantics. (XXX? Only acquire?)
+ */
+STATIC_IF_INLINE_DECLARE bool
+pg_atomic_test_set_flag(volatile pg_atomic_flag *ptr);
+
+/*
+ * pg_atomic_unlocked_test_flag - TAS()
+ *
+ * No barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE bool
+pg_atomic_unlocked_test_flag(volatile pg_atomic_flag *ptr);
+
+/*
+ * pg_atomic_clear_flag - release lock set by TAS()
+ *
+ * Full barrier semantics. (XXX? Only release?)
+ */
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_clear_flag(volatile pg_atomic_flag *ptr);
+
+/*
+ * pg_atomic_init_u32 - initialize atomic variable
+ *
+ * Has to be done before any usage except when the atomic variable is declared
+ * statically in which case the variable is initialized to 0.
+ *
+ * No barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_init_u32(volatile pg_atomic_uint32 *ptr, uint32 val);
+
+/*
+ * pg_atomic_read_u32 - read from atomic variable
+ *
+ * No barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_read_u32(volatile pg_atomic_uint32 *ptr);
+
+/*
+ * pg_atomic_write_u32 - write to atomic variable
+ *
+ * No barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_write_u32(volatile pg_atomic_uint32 *ptr, uint32 val);
+
+/*
+ * pg_atomic_exchange_u32 - exchange newval with current value
+ *
+ * Returns the old value of 'ptr' before the swap.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_exchange_u32(volatile pg_atomic_uint32 *ptr, uint32 newval);
+
+/*
+ * pg_atomic_compare_exchange_u32 - CAS operation
+ *
+ * Atomically compare the current value of ptr with *expected and store newval
+ * iff ptr and *expected have the same value. The current value of *ptr will
+ * always be stored in *expected.
+ *
+ * Return whether the values have been exchanged.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE bool
+pg_atomic_compare_exchange_u32(volatile pg_atomic_uint32 *ptr,
+							   uint32 *expected, uint32 newval);
+
+/*
+ * pg_atomic_fetch_add_u32 - atomically add to variable
+ *
+ * Returns the the value of ptr before the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_fetch_add_u32(volatile pg_atomic_uint32 *ptr, uint32 add_);
+
+/*
+ * pg_atomic_fetch_sub_u32 - atomically subtract from variable
+ *
+ * Returns the the value of ptr before the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_fetch_sub_u32(volatile pg_atomic_uint32 *ptr, uint32 sub_);
+
+/*
+ * pg_atomic_fetch_and_u32 - atomically bit-and and_ with variable
+ *
+ * Returns the the value of ptr before the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_fetch_and_u32(volatile pg_atomic_uint32 *ptr, uint32 and_);
+
+/*
+ * pg_atomic_fetch_or_u32 - atomically bit-or or_ with variable
+ *
+ * Returns the the value of ptr before the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_fetch_or_u32(volatile pg_atomic_uint32 *ptr, uint32 or_);
+
+/*
+ * pg_atomic_add_fetch_u32 - atomically add to variable
+ *
+ * Returns the the value of ptr after the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_add_fetch_u32(volatile pg_atomic_uint32 *ptr, uint32 add_);
+
+/*
+ * pg_atomic_sub_fetch_u32 - atomically subtract from variable
+ *
+ * Returns the the value of ptr after the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_sub_fetch_u32(volatile pg_atomic_uint32 *ptr, uint32 sub_);
+
+/*
+ * pg_fetch_add_until_u32 - saturated addition to variable
+ *
+ * Returns the the value of ptr after the arithmetic operation.
+ *
+ * Full barrier semantics.
+ */
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_fetch_add_until_u32(volatile pg_atomic_uint32 *ptr, uint32 add_,
+							  uint32 until);
+
+/* ----
+ * The 64 bit operations have the same semantics as their 32bit counterparts if
+ * they are available.
+ * ---
+ */
+#ifdef PG_HAVE_64_BIT_ATOMICS
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_init_u64(volatile pg_atomic_uint64 *ptr, uint64 val_);
+
+STATIC_IF_INLINE_DECLARE uint32
+pg_atomic_read_u64(volatile pg_atomic_uint64 *ptr);
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_write_u64(volatile pg_atomic_uint64 *ptr, uint64 val);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_exchange_u64(volatile pg_atomic_uint64 *ptr, uint64 newval);
+
+STATIC_IF_INLINE_DECLARE bool
+pg_atomic_compare_exchange_u64(volatile pg_atomic_uint64 *ptr,
+							   uint32 *expected, uint64 newval);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_fetch_add_u64(volatile pg_atomic_uint64 *ptr, uint64 add_);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_fetch_sub_u64(volatile pg_atomic_uint64 *ptr, uint64 sub_);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_fetch_and_u64(volatile pg_atomic_uint64 *ptr, uint64 and_);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_fetch_or_u64(volatile pg_atomic_uint64 *ptr, uint64 or_);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_add_fetch_u64(volatile pg_atomic_uint64 *ptr, uint64 add_);
+
+STATIC_IF_INLINE_DECLARE uint64
+pg_atomic_sub_fetch_u64(volatile pg_atomic_uint64 *ptr, uint64 sub_);
+
+#endif /* PG_HAVE_64_BIT_ATOMICS */
+
+/*
+ * The following functions are wrapper functions around the platform specific
+ * implementation of the atomic operations performing common checks.
+ */
+#if defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS)
+
+#define CHECK_POINTER_ALIGNMENT(ptr, bndr) \
+	Assert(TYPEALIGN((uintptr_t)(ptr), bndr))
+
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_init_flag(volatile pg_atomic_flag *ptr)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, sizeof(*ptr));
+
+	return pg_atomic_clear_flag_impl(ptr);
+}
+
+STATIC_IF_INLINE_DECLARE bool
+pg_atomic_test_set_flag(volatile pg_atomic_flag *ptr)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, sizeof(*ptr));
+
+	return pg_atomic_test_set_flag_impl(ptr);
+}
+
+STATIC_IF_INLINE_DECLARE bool
+pg_atomic_unlocked_test_flag(volatile pg_atomic_flag *ptr)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, sizeof(*ptr));
+
+	return pg_atomic_unlocked_test_flag_impl(ptr);
+}
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_clear_flag(volatile pg_atomic_flag *ptr)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, sizeof(*ptr));
+
+	pg_atomic_clear_flag_impl(ptr);
+}
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_init_u32(volatile pg_atomic_uint32 *ptr, uint32 val)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+
+	pg_atomic_init_u32_impl(ptr, val);
+}
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_write_u32(volatile pg_atomic_uint32 *ptr, uint32 val)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+
+	pg_atomic_write_u32_impl(ptr, val);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_read_u32(volatile pg_atomic_uint32 *ptr)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_read_u32_impl(ptr);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_exchange_u32(volatile pg_atomic_uint32 *ptr, uint32 newval)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+
+	return pg_atomic_exchange_u32_impl(ptr, newval);
+}
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u32(volatile pg_atomic_uint32 *ptr,
+							   uint32 *expected, uint32 newval)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	CHECK_POINTER_ALIGNMENT(expected, 4);
+
+	return pg_atomic_compare_exchange_u32_impl(ptr, expected, newval);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_add_u32(volatile pg_atomic_uint32 *ptr, uint32 add_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_fetch_add_u32_impl(ptr, add_);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_sub_u32(volatile pg_atomic_uint32 *ptr, uint32 sub_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_fetch_sub_u32_impl(ptr, sub_);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_and_u32(volatile pg_atomic_uint32 *ptr, uint32 and_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_fetch_and_u32_impl(ptr, and_);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_or_u32(volatile pg_atomic_uint32 *ptr, uint32 or_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_fetch_or_u32_impl(ptr, or_);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_add_fetch_u32(volatile pg_atomic_uint32 *ptr, uint32 add_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_add_fetch_u32_impl(ptr, add_);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_sub_fetch_u32(volatile pg_atomic_uint32 *ptr, uint32 sub_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_sub_fetch_u32_impl(ptr, sub_);
+}
+
+STATIC_IF_INLINE uint32
+pg_atomic_fetch_add_until_u32(volatile pg_atomic_uint32 *ptr, uint32 add_,
+							  uint32 until)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 4);
+	return pg_atomic_fetch_add_until_u32_impl(ptr, add_, until);
+}
+
+#ifdef PG_HAVE_64_BIT_ATOMICS
+
+STATIC_IF_INLINE_DECLARE void
+pg_atomic_init_u64(volatile pg_atomic_uint64 *ptr, uint64 val)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+
+	pg_atomic_init_u64_impl(ptr, val);
+}
+
+STATIC_IF_INLINE uint64
+pg_atomic_read_u64(volatile pg_atomic_uint64 *ptr)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_read_u64_impl(ptr);
+}
+
+STATIC_IF_INLINE void
+pg_atomic_write_u64(volatile pg_atomic_uint64 *ptr, uint64 val)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_write_u64_impl(ptr, val);
+}
+
+STATIC_IF_INLINE uint64
+pg_atomic_exchange_u64(volatile pg_atomic_uint64 *ptr, uint64 newval)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+
+	return pg_atomic_exchange_u64_impl(ptr, newval);
+}
+
+STATIC_IF_INLINE bool
+pg_atomic_compare_exchange_u64(volatile pg_atomic_uint64 *ptr,
+							   uint64 *expected, uint64 newval)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	CHECK_POINTER_ALIGNMENT(expected, 8);
+	return pg_atomic_compare_exchange_u64_impl(ptr, expected, newval);
+}
+
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_add_u64(volatile pg_atomic_uint64 *ptr, uint64 add_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_fetch_add_u64_impl(ptr, add_);
+}
+
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_sub_u64(volatile pg_atomic_uint64 *ptr, uint64 sub_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_fetch_sub_u64_impl(ptr, sub_);
+}
+
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_and_u64(volatile pg_atomic_uint64 *ptr, uint64 and_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_fetch_and_u64_impl(ptr, and_);
+}
+
+STATIC_IF_INLINE uint64
+pg_atomic_fetch_or_u64(volatile pg_atomic_uint64 *ptr, uint64 or_)
+{
+	CHECK_POINTER_ALIGNMENT(ptr, 8);
+	return pg_atomic_fetch_or_u64_impl(ptr, or_);
+}
+#endif /* PG_HAVE_64_BIT_ATOMICS */
+
+#endif /* defined(PG_USE_INLINE) || defined(ATOMICS_INCLUDE_DEFINITIONS) */
+
+#undef INSIDE_ATOMICS_H
+
+#endif /* ATOMICS_H */
diff --git a/src/include/storage/barrier.h b/src/include/storage/barrier.h
index 16a9a0c..607a3c9 100644
--- a/src/include/storage/barrier.h
+++ b/src/include/storage/barrier.h
@@ -13,137 +13,12 @@
 #ifndef BARRIER_H
 #define BARRIER_H
 
-#include "storage/s_lock.h"
-
-extern slock_t dummy_spinlock;
-
-/*
- * A compiler barrier need not (and preferably should not) emit any actual
- * machine code, but must act as an optimization fence: the compiler must not
- * reorder loads or stores to main memory around the barrier.  However, the
- * CPU may still reorder loads or stores at runtime, if the architecture's
- * memory model permits this.
- *
- * A memory barrier must act as a compiler barrier, and in addition must
- * guarantee that all loads and stores issued prior to the barrier are
- * completed before any loads or stores issued after the barrier.  Unless
- * loads and stores are totally ordered (which is not the case on most
- * architectures) this requires issuing some sort of memory fencing
- * instruction.
- *
- * A read barrier must act as a compiler barrier, and in addition must
- * guarantee that any loads issued prior to the barrier are completed before
- * any loads issued after the barrier.	Similarly, a write barrier acts
- * as a compiler barrier, and also orders stores.  Read and write barriers
- * are thus weaker than a full memory barrier, but stronger than a compiler
- * barrier.  In practice, on machines with strong memory ordering, read and
- * write barriers may require nothing more than a compiler barrier.
- *
- * For an introduction to using memory barriers within the PostgreSQL backend,
- * see src/backend/storage/lmgr/README.barrier
- */
-
-#if defined(DISABLE_BARRIERS)
-
-/*
- * Fall through to the spinlock-based implementation.
- */
-#elif defined(__INTEL_COMPILER)
-
-/*
- * icc defines __GNUC__, but doesn't support gcc's inline asm syntax
- */
-#if defined(__ia64__) || defined(__ia64)
-#define pg_memory_barrier()		__mf()
-#elif defined(__i386__) || defined(__x86_64__)
-#define pg_memory_barrier()		_mm_mfence()
-#endif
-
-#define pg_compiler_barrier()	__memory_barrier()
-#elif defined(__GNUC__)
-
-/* This works on any architecture, since it's only talking to GCC itself. */
-#define pg_compiler_barrier()	__asm__ __volatile__("" : : : "memory")
-
-#if defined(__i386__)
-
 /*
- * i386 does not allow loads to be reordered with other loads, or stores to be
- * reordered with other stores, but a load can be performed before a subsequent
- * store.
- *
- * "lock; addl" has worked for longer than "mfence".
+ * This used to be a separate file, full of compiler/architecture
+ * dependent defines, but it's not included in the atomics.h
+ * infrastructure and just kept for backward compatibility.
  */
-#define pg_memory_barrier()		\
-	__asm__ __volatile__ ("lock; addl $0,0(%%esp)" : : : "memory")
-#define pg_read_barrier()		pg_compiler_barrier()
-#define pg_write_barrier()		pg_compiler_barrier()
-#elif defined(__x86_64__)		/* 64 bit x86 */
-
-/*
- * x86_64 has similar ordering characteristics to i386.
- *
- * Technically, some x86-ish chips support uncached memory access and/or
- * special instructions that are weakly ordered.  In those cases we'd need
- * the read and write barriers to be lfence and sfence.  But since we don't
- * do those things, a compiler barrier should be enough.
- */
-#define pg_memory_barrier()		\
-	__asm__ __volatile__ ("lock; addl $0,0(%%rsp)" : : : "memory")
-#define pg_read_barrier()		pg_compiler_barrier()
-#define pg_write_barrier()		pg_compiler_barrier()
-#elif defined(__ia64__) || defined(__ia64)
-
-/*
- * Itanium is weakly ordered, so read and write barriers require a full
- * fence.
- */
-#define pg_memory_barrier()		__asm__ __volatile__ ("mf" : : : "memory")
-#elif defined(__ppc__) || defined(__powerpc__) || defined(__ppc64__) || defined(__powerpc64__)
-
-/*
- * lwsync orders loads with respect to each other, and similarly with stores.
- * But a load can be performed before a subsequent store, so sync must be used
- * for a full memory barrier.
- */
-#define pg_memory_barrier()		__asm__ __volatile__ ("sync" : : : "memory")
-#define pg_read_barrier()		__asm__ __volatile__ ("lwsync" : : : "memory")
-#define pg_write_barrier()		__asm__ __volatile__ ("lwsync" : : : "memory")
-#elif defined(__alpha) || defined(__alpha__)	/* Alpha */
-
-/*
- * Unlike all other known architectures, Alpha allows dependent reads to be
- * reordered, but we don't currently find it necessary to provide a conditional
- * read barrier to cover that case.  We might need to add that later.
- */
-#define pg_memory_barrier()		__asm__ __volatile__ ("mb" : : : "memory")
-#define pg_read_barrier()		__asm__ __volatile__ ("rmb" : : : "memory")
-#define pg_write_barrier()		__asm__ __volatile__ ("wmb" : : : "memory")
-#elif defined(__hppa) || defined(__hppa__)		/* HP PA-RISC */
-
-/* HPPA doesn't do either read or write reordering */
-#define pg_memory_barrier()		pg_compiler_barrier()
-#elif __GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 1)
-
-/*
- * If we're on GCC 4.1.0 or higher, we should be able to get a memory
- * barrier out of this compiler built-in.  But we prefer to rely on our
- * own definitions where possible, and use this only as a fallback.
- */
-#define pg_memory_barrier()		__sync_synchronize()
-#endif
-#elif defined(__ia64__) || defined(__ia64)
-
-#define pg_compiler_barrier()	_Asm_sched_fence()
-#define pg_memory_barrier()		_Asm_mf()
-#elif defined(WIN32_ONLY_COMPILER)
-
-/* Should work on both MSVC and Borland. */
-#include <intrin.h>
-#pragma intrinsic(_ReadWriteBarrier)
-#define pg_compiler_barrier()	_ReadWriteBarrier()
-#define pg_memory_barrier()		MemoryBarrier()
-#endif
+#include "storage/atomics.h"
 
 /*
  * If we have no memory barrier implementation for this architecture, we
@@ -157,6 +32,10 @@ extern slock_t dummy_spinlock;
  * fence.  But all of our actual implementations seem OK in this regard.
  */
 #if !defined(pg_memory_barrier)
+#include "storage/s_lock.h"
+
+extern slock_t dummy_spinlock;
+
 #define pg_memory_barrier() \
 	do { S_LOCK(&dummy_spinlock); S_UNLOCK(&dummy_spinlock); } while (0)
 #endif
diff --git a/src/include/storage/s_lock.h b/src/include/storage/s_lock.h
index 7dcd5d9..73aebfa 100644
--- a/src/include/storage/s_lock.h
+++ b/src/include/storage/s_lock.h
@@ -21,10 +21,6 @@
  *	void S_UNLOCK(slock_t *lock)
  *		Unlock a previously acquired lock.
  *
- *	bool S_LOCK_FREE(slock_t *lock)
- *		Tests if the lock is free. Returns TRUE if free, FALSE if locked.
- *		This does *not* change the state of the lock.
- *
  *	void SPIN_DELAY(void)
  *		Delay operation to occur inside spinlock wait loop.
  *
@@ -94,879 +90,17 @@
 #ifndef S_LOCK_H
 #define S_LOCK_H
 
+#include "storage/atomics.h"
 #include "storage/pg_sema.h"
 
-#ifdef HAVE_SPINLOCKS	/* skip spinlocks if requested */
-
-
-#if defined(__GNUC__) || defined(__INTEL_COMPILER)
-/*************************************************************************
- * All the gcc inlines
- * Gcc consistently defines the CPU as __cpu__.
- * Other compilers use __cpu or __cpu__ so we test for both in those cases.
- */
-
-/*----------
- * Standard gcc asm format (assuming "volatile slock_t *lock"):
-
-	__asm__ __volatile__(
-		"	instruction	\n"
-		"	instruction	\n"
-		"	instruction	\n"
-:		"=r"(_res), "+m"(*lock)		// return register, in/out lock value
-:		"r"(lock)					// lock pointer, in input register
-:		"memory", "cc");			// show clobbered registers here
-
- * The output-operands list (after first colon) should always include
- * "+m"(*lock), whether or not the asm code actually refers to this
- * operand directly.  This ensures that gcc believes the value in the
- * lock variable is used and set by the asm code.  Also, the clobbers
- * list (after third colon) should always include "memory"; this prevents
- * gcc from thinking it can cache the values of shared-memory fields
- * across the asm code.  Add "cc" if your asm code changes the condition
- * code register, and also list any temp registers the code uses.
- *----------
- */
-
-
-#ifdef __i386__		/* 32-bit i386 */
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock) tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register slock_t _res = 1;
-
-	/*
-	 * Use a non-locking test before asserting the bus lock.  Note that the
-	 * extra test appears to be a small loss on some x86 platforms and a small
-	 * win on others; it's by no means clear that we should keep it.
-	 *
-	 * When this was last tested, we didn't have separate TAS() and TAS_SPIN()
-	 * macros.  Nowadays it probably would be better to do a non-locking test
-	 * in TAS_SPIN() but not in TAS(), like on x86_64, but no-one's done the
-	 * testing to verify that.  Without some empirical evidence, better to
-	 * leave it alone.
-	 */
-	__asm__ __volatile__(
-		"	cmpb	$0,%1	\n"
-		"	jne		1f		\n"
-		"	lock			\n"
-		"	xchgb	%0,%1	\n"
-		"1: \n"
-:		"+q"(_res), "+m"(*lock)
-:
-:		"memory", "cc");
-	return (int) _res;
-}
-
-#define SPIN_DELAY() spin_delay()
-
-static __inline__ void
-spin_delay(void)
-{
-	/*
-	 * This sequence is equivalent to the PAUSE instruction ("rep" is
-	 * ignored by old IA32 processors if the following instruction is
-	 * not a string operation); the IA-32 Architecture Software
-	 * Developer's Manual, Vol. 3, Section 7.7.2 describes why using
-	 * PAUSE in the inner loop of a spin lock is necessary for good
-	 * performance:
-	 *
-	 *     The PAUSE instruction improves the performance of IA-32
-	 *     processors supporting Hyper-Threading Technology when
-	 *     executing spin-wait loops and other routines where one
-	 *     thread is accessing a shared lock or semaphore in a tight
-	 *     polling loop. When executing a spin-wait loop, the
-	 *     processor can suffer a severe performance penalty when
-	 *     exiting the loop because it detects a possible memory order
-	 *     violation and flushes the core processor's pipeline. The
-	 *     PAUSE instruction provides a hint to the processor that the
-	 *     code sequence is a spin-wait loop. The processor uses this
-	 *     hint to avoid the memory order violation and prevent the
-	 *     pipeline flush. In addition, the PAUSE instruction
-	 *     de-pipelines the spin-wait loop to prevent it from
-	 *     consuming execution resources excessively.
-	 */
-	__asm__ __volatile__(
-		" rep; nop			\n");
-}
-
-#endif	 /* __i386__ */
-
-
-#ifdef __x86_64__		/* AMD Opteron, Intel EM64T */
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock) tas(lock)
-
-/*
- * On Intel EM64T, it's a win to use a non-locking test before the xchg proper,
- * but only when spinning.
- *
- * See also Implementing Scalable Atomic Locks for Multi-Core Intel(tm) EM64T
- * and IA32, by Michael Chynoweth and Mary R. Lee. As of this writing, it is
- * available at:
- * http://software.intel.com/en-us/articles/implementing-scalable-atomic-locks-for-multi-core-intel-em64t-and-ia32-architectures
- */
-#define TAS_SPIN(lock)    (*(lock) ? 1 : TAS(lock))
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register slock_t _res = 1;
-
-	__asm__ __volatile__(
-		"	lock			\n"
-		"	xchgb	%0,%1	\n"
-:		"+q"(_res), "+m"(*lock)
-:
-:		"memory", "cc");
-	return (int) _res;
-}
-
-#define SPIN_DELAY() spin_delay()
-
-static __inline__ void
-spin_delay(void)
-{
-	/*
-	 * Adding a PAUSE in the spin delay loop is demonstrably a no-op on
-	 * Opteron, but it may be of some use on EM64T, so we keep it.
-	 */
-	__asm__ __volatile__(
-		" rep; nop			\n");
-}
-
-#endif	 /* __x86_64__ */
-
-
-#if defined(__ia64__) || defined(__ia64)
-/*
- * Intel Itanium, gcc or Intel's compiler.
- *
- * Itanium has weak memory ordering, but we rely on the compiler to enforce
- * strict ordering of accesses to volatile data.  In particular, while the
- * xchg instruction implicitly acts as a memory barrier with 'acquire'
- * semantics, we do not have an explicit memory fence instruction in the
- * S_UNLOCK macro.  We use a regular assignment to clear the spinlock, and
- * trust that the compiler marks the generated store instruction with the
- * ".rel" opcode.
- *
- * Testing shows that assumption to hold on gcc, although I could not find
- * any explicit statement on that in the gcc manual.  In Intel's compiler,
- * the -m[no-]serialize-volatile option controls that, and testing shows that
- * it is enabled by default.
- */
-#define HAS_TEST_AND_SET
-
-typedef unsigned int slock_t;
-
-#define TAS(lock) tas(lock)
-
-/* On IA64, it's a win to use a non-locking test before the xchg proper */
-#define TAS_SPIN(lock)	(*(lock) ? 1 : TAS(lock))
-
-#ifndef __INTEL_COMPILER
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	long int	ret;
-
-	__asm__ __volatile__(
-		"	xchg4 	%0=%1,%2	\n"
-:		"=r"(ret), "+m"(*lock)
-:		"r"(1)
-:		"memory");
-	return (int) ret;
-}
-
-#else /* __INTEL_COMPILER */
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	int		ret;
-
-	ret = _InterlockedExchange(lock,1);	/* this is a xchg asm macro */
-
-	return ret;
-}
-
-#endif /* __INTEL_COMPILER */
-#endif	 /* __ia64__ || __ia64 */
-
-
-/*
- * On ARM, we use __sync_lock_test_and_set(int *, int) if available, and if
- * not fall back on the SWPB instruction.  SWPB does not work on ARMv6 or
- * later, so the compiler builtin is preferred if available.  Note also that
- * the int-width variant of the builtin works on more chips than other widths.
- */
-#if defined(__arm__) || defined(__arm)
-#define HAS_TEST_AND_SET
-
-#define TAS(lock) tas(lock)
-
-#ifdef HAVE_GCC_INT_ATOMICS
-
-typedef int slock_t;
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	return __sync_lock_test_and_set(lock, 1);
-}
-
-#define S_UNLOCK(lock) __sync_lock_release(lock)
-
-#else /* !HAVE_GCC_INT_ATOMICS */
-
-typedef unsigned char slock_t;
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register slock_t _res = 1;
-
-	__asm__ __volatile__(
-		"	swpb 	%0, %0, [%2]	\n"
-:		"+r"(_res), "+m"(*lock)
-:		"r"(lock)
-:		"memory");
-	return (int) _res;
-}
-
-#endif	 /* HAVE_GCC_INT_ATOMICS */
-#endif	 /* __arm__ */
-
-
-/*
- * On ARM64, we use __sync_lock_test_and_set(int *, int) if available.
- */
-#if defined(__aarch64__) || defined(__aarch64)
-#ifdef HAVE_GCC_INT_ATOMICS
-#define HAS_TEST_AND_SET
-
-#define TAS(lock) tas(lock)
-
-typedef int slock_t;
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	return __sync_lock_test_and_set(lock, 1);
-}
-
-#define S_UNLOCK(lock) __sync_lock_release(lock)
-
-#endif	 /* HAVE_GCC_INT_ATOMICS */
-#endif	 /* __aarch64__ */
-
-
-/* S/390 and S/390x Linux (32- and 64-bit zSeries) */
-#if defined(__s390__) || defined(__s390x__)
-#define HAS_TEST_AND_SET
-
-typedef unsigned int slock_t;
-
-#define TAS(lock)	   tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	int			_res = 0;
-
-	__asm__	__volatile__(
-		"	cs 	%0,%3,0(%2)		\n"
-:		"+d"(_res), "+m"(*lock)
-:		"a"(lock), "d"(1)
-:		"memory", "cc");
-	return _res;
-}
-
-#endif	 /* __s390__ || __s390x__ */
-
-
-#if defined(__sparc__)		/* Sparc */
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock) tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register slock_t _res;
-
-	/*
-	 *	See comment in /pg/backend/port/tas/solaris_sparc.s for why this
-	 *	uses "ldstub", and that file uses "cas".  gcc currently generates
-	 *	sparcv7-targeted binaries, so "cas" use isn't possible.
-	 */
-	__asm__ __volatile__(
-		"	ldstub	[%2], %0	\n"
-:		"=r"(_res), "+m"(*lock)
-:		"r"(lock)
-:		"memory");
-	return (int) _res;
-}
-
-#endif	 /* __sparc__ */
-
-
-/* PowerPC */
-#if defined(__ppc__) || defined(__powerpc__) || defined(__ppc64__) || defined(__powerpc64__)
-#define HAS_TEST_AND_SET
-
-typedef unsigned int slock_t;
-
-#define TAS(lock) tas(lock)
-
-/* On PPC, it's a win to use a non-locking test before the lwarx */
-#define TAS_SPIN(lock)	(*(lock) ? 1 : TAS(lock))
-
-/*
- * NOTE: per the Enhanced PowerPC Architecture manual, v1.0 dated 7-May-2002,
- * an isync is a sufficient synchronization barrier after a lwarx/stwcx loop.
- * On newer machines, we can use lwsync instead for better performance.
- */
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	slock_t _t;
-	int _res;
-
-	__asm__ __volatile__(
-#ifdef USE_PPC_LWARX_MUTEX_HINT
-"	lwarx   %0,0,%3,1	\n"
-#else
-"	lwarx   %0,0,%3		\n"
-#endif
-"	cmpwi   %0,0		\n"
-"	bne     1f			\n"
-"	addi    %0,%0,1		\n"
-"	stwcx.  %0,0,%3		\n"
-"	beq     2f         	\n"
-"1:	li      %1,1		\n"
-"	b		3f			\n"
-"2:						\n"
-#ifdef USE_PPC_LWSYNC
-"	lwsync				\n"
-#else
-"	isync				\n"
-#endif
-"	li      %1,0		\n"
-"3:						\n"
-
-:	"=&r"(_t), "=r"(_res), "+m"(*lock)
-:	"r"(lock)
-:	"memory", "cc");
-	return _res;
-}
-
-/*
- * PowerPC S_UNLOCK is almost standard but requires a "sync" instruction.
- * On newer machines, we can use lwsync instead for better performance.
- */
-#ifdef USE_PPC_LWSYNC
-#define S_UNLOCK(lock)	\
-do \
-{ \
-	__asm__ __volatile__ ("	lwsync \n"); \
-	*((volatile slock_t *) (lock)) = 0; \
-} while (0)
-#else
-#define S_UNLOCK(lock)	\
-do \
-{ \
-	__asm__ __volatile__ ("	sync \n"); \
-	*((volatile slock_t *) (lock)) = 0; \
-} while (0)
-#endif /* USE_PPC_LWSYNC */
-
-#endif /* powerpc */
-
-
-/* Linux Motorola 68k */
-#if (defined(__mc68000__) || defined(__m68k__)) && defined(__linux__)
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock) tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register int rv;
-
-	__asm__	__volatile__(
-		"	clrl	%0		\n"
-		"	tas		%1		\n"
-		"	sne		%0		\n"
-:		"=d"(rv), "+m"(*lock)
-:
-:		"memory", "cc");
-	return rv;
-}
-
-#endif	 /* (__mc68000__ || __m68k__) && __linux__ */
-
-
-/*
- * VAXen -- even multiprocessor ones
- * (thanks to Tom Ivar Helbekkmo)
- */
-#if defined(__vax__)
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock) tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register int	_res;
-
-	__asm__ __volatile__(
-		"	movl 	$1, %0			\n"
-		"	bbssi	$0, (%2), 1f	\n"
-		"	clrl	%0				\n"
-		"1: \n"
-:		"=&r"(_res), "+m"(*lock)
-:		"r"(lock)
-:		"memory");
-	return _res;
-}
-
-#endif	 /* __vax__ */
-
-#if defined(__alpha) || defined(__alpha__)	/* Alpha */
-/*
- * Correct multi-processor locking methods are explained in section 5.5.3
- * of the Alpha AXP Architecture Handbook, which at this writing can be
- * found at ftp://ftp.netbsd.org/pub/NetBSD/misc/dec-docs/index.html.
- * For gcc we implement the handbook's code directly with inline assembler.
- */
-#define HAS_TEST_AND_SET
-
-typedef unsigned long slock_t;
-
-#define TAS(lock)  tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register slock_t _res;
-
-	__asm__	__volatile__(
-		"	ldq		$0, %1	\n"
-		"	bne		$0, 2f	\n"
-		"	ldq_l	%0, %1	\n"
-		"	bne		%0, 2f	\n"
-		"	mov		1,  $0	\n"
-		"	stq_c	$0, %1	\n"
-		"	beq		$0, 2f	\n"
-		"	mb				\n"
-		"	br		3f		\n"
-		"2:	mov		1, %0	\n"
-		"3:					\n"
-:		"=&r"(_res), "+m"(*lock)
-:
-:		"memory", "0");
-	return (int) _res;
-}
-
-#define S_UNLOCK(lock)	\
-do \
-{\
-	__asm__ __volatile__ ("	mb \n"); \
-	*((volatile slock_t *) (lock)) = 0; \
-} while (0)
-
-#endif /* __alpha || __alpha__ */
-
-
-#if defined(__mips__) && !defined(__sgi)	/* non-SGI MIPS */
-/* Note: on SGI we use the OS' mutex ABI, see below */
-/* Note: R10000 processors require a separate SYNC */
-#define HAS_TEST_AND_SET
-
-typedef unsigned int slock_t;
-
-#define TAS(lock) tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register volatile slock_t *_l = lock;
-	register int _res;
-	register int _tmp;
-
-	__asm__ __volatile__(
-		"       .set push           \n"
-		"       .set mips2          \n"
-		"       .set noreorder      \n"
-		"       .set nomacro        \n"
-		"       ll      %0, %2      \n"
-		"       or      %1, %0, 1   \n"
-		"       sc      %1, %2      \n"
-		"       xori    %1, 1       \n"
-		"       or      %0, %0, %1  \n"
-		"       sync                \n"
-		"       .set pop              "
-:		"=&r" (_res), "=&r" (_tmp), "+R" (*_l)
-:
-:		"memory");
-	return _res;
-}
-
-/* MIPS S_UNLOCK is almost standard but requires a "sync" instruction */
-#define S_UNLOCK(lock)	\
-do \
-{ \
-	__asm__ __volatile__( \
-		"       .set push           \n" \
-		"       .set mips2          \n" \
-		"       .set noreorder      \n" \
-		"       .set nomacro        \n" \
-		"       sync                \n" \
-		"       .set pop              "); \
-	*((volatile slock_t *) (lock)) = 0; \
-} while (0)
-
-#endif /* __mips__ && !__sgi */
-
-
-#if defined(__m32r__) && defined(HAVE_SYS_TAS_H)	/* Renesas' M32R */
-#define HAS_TEST_AND_SET
-
-#include <sys/tas.h>
-
-typedef int slock_t;
-
-#define TAS(lock) tas(lock)
-
-#endif /* __m32r__ */
-
-
-#if defined(__sh__)				/* Renesas' SuperH */
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock) tas(lock)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	register int _res;
-
-	/*
-	 * This asm is coded as if %0 could be any register, but actually SuperH
-	 * restricts the target of xor-immediate to be R0.  That's handled by
-	 * the "z" constraint on _res.
-	 */
-	__asm__ __volatile__(
-		"	tas.b @%2    \n"
-		"	movt  %0     \n"
-		"	xor   #1,%0  \n"
-:		"=z"(_res), "+m"(*lock)
-:		"r"(lock)
-:		"memory", "t");
-	return _res;
-}
-
-#endif	 /* __sh__ */
-
-
-/* These live in s_lock.c, but only for gcc */
-
-
-#if defined(__m68k__) && !defined(__linux__)	/* non-Linux Motorola 68k */
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-#endif
-
-
-#endif	/* defined(__GNUC__) || defined(__INTEL_COMPILER) */
-
-
-
-/*
- * ---------------------------------------------------------------------
- * Platforms that use non-gcc inline assembly:
- * ---------------------------------------------------------------------
- */
-
-#if !defined(HAS_TEST_AND_SET)	/* We didn't trigger above, let's try here */
-
-
-#if defined(USE_UNIVEL_CC)		/* Unixware compiler */
-#define HAS_TEST_AND_SET
-
-typedef unsigned char slock_t;
-
-#define TAS(lock)	tas(lock)
-
-asm int
-tas(volatile slock_t *s_lock)
-{
-/* UNIVEL wants %mem in column 1, so we don't pg_indent this file */
-%mem s_lock
-	pushl %ebx
-	movl s_lock, %ebx
-	movl $255, %eax
-	lock
-	xchgb %al, (%ebx)
-	popl %ebx
-}
-
-#endif	 /* defined(USE_UNIVEL_CC) */
-
-
-#if defined(__alpha) || defined(__alpha__)	/* Tru64 Unix Alpha compiler */
-/*
- * The Tru64 compiler doesn't support gcc-style inline asm, but it does
- * have some builtin functions that accomplish much the same results.
- * For simplicity, slock_t is defined as long (ie, quadword) on Alpha
- * regardless of the compiler in use.  LOCK_LONG and UNLOCK_LONG only
- * operate on an int (ie, longword), but that's OK as long as we define
- * S_INIT_LOCK to zero out the whole quadword.
- */
-#define HAS_TEST_AND_SET
-
-typedef unsigned long slock_t;
-
-#include <alpha/builtins.h>
-#define S_INIT_LOCK(lock)  (*(lock) = 0)
-#define TAS(lock)		   (__LOCK_LONG_RETRY((lock), 1) == 0)
-#define S_UNLOCK(lock)	   __UNLOCK_LONG(lock)
-
-#endif	 /* __alpha || __alpha__ */
-
-
-#if defined(__hppa) || defined(__hppa__)	/* HP PA-RISC, GCC and HP compilers */
-/*
- * HP's PA-RISC
- *
- * See src/backend/port/hpux/tas.c.template for details about LDCWX.  Because
- * LDCWX requires a 16-byte-aligned address, we declare slock_t as a 16-byte
- * struct.  The active word in the struct is whichever has the aligned address;
- * the other three words just sit at -1.
- *
- * When using gcc, we can inline the required assembly code.
- */
-#define HAS_TEST_AND_SET
-
-typedef struct
-{
-	int			sema[4];
-} slock_t;
-
-#define TAS_ACTIVE_WORD(lock)	((volatile int *) (((uintptr_t) (lock) + 15) & ~15))
-
-#if defined(__GNUC__)
-
-static __inline__ int
-tas(volatile slock_t *lock)
-{
-	volatile int *lockword = TAS_ACTIVE_WORD(lock);
-	register int lockval;
-
-	__asm__ __volatile__(
-		"	ldcwx	0(0,%2),%0	\n"
-:		"=r"(lockval), "+m"(*lockword)
-:		"r"(lockword)
-:		"memory");
-	return (lockval == 0);
-}
-
-#endif /* __GNUC__ */
-
-#define S_UNLOCK(lock)	(*TAS_ACTIVE_WORD(lock) = -1)
-
-#define S_INIT_LOCK(lock) \
-	do { \
-		volatile slock_t *lock_ = (lock); \
-		lock_->sema[0] = -1; \
-		lock_->sema[1] = -1; \
-		lock_->sema[2] = -1; \
-		lock_->sema[3] = -1; \
-	} while (0)
-
-#define S_LOCK_FREE(lock)	(*TAS_ACTIVE_WORD(lock) != 0)
-
-#endif	 /* __hppa || __hppa__ */
-
-
-#if defined(__hpux) && defined(__ia64) && !defined(__GNUC__)
-/*
- * HP-UX on Itanium, non-gcc compiler
- *
- * We assume that the compiler enforces strict ordering of loads/stores on
- * volatile data (see comments on the gcc-version earlier in this file).
- * Note that this assumption does *not* hold if you use the
- * +Ovolatile=__unordered option on the HP-UX compiler, so don't do that.
- *
- * See also Implementing Spinlocks on the Intel Itanium Architecture and
- * PA-RISC, by Tor Ekqvist and David Graves, for more information.  As of
- * this writing, version 1.0 of the manual is available at:
- * http://h21007.www2.hp.com/portal/download/files/unprot/itanium/spinlocks.pdf
- */
-#define HAS_TEST_AND_SET
-
-typedef unsigned int slock_t;
-
-#include <ia64/sys/inline.h>
-#define TAS(lock) _Asm_xchg(_SZ_W, lock, 1, _LDHINT_NONE)
-/* On IA64, it's a win to use a non-locking test before the xchg proper */
-#define TAS_SPIN(lock)	(*(lock) ? 1 : TAS(lock))
-
-#endif	/* HPUX on IA64, non gcc */
-
-#if defined(_AIX)	/* AIX */
-/*
- * AIX (POWER)
- */
-#define HAS_TEST_AND_SET
-
-#include <sys/atomic_op.h>
-
-typedef int slock_t;
-
-#define TAS(lock)			_check_lock((slock_t *) (lock), 0, 1)
-#define S_UNLOCK(lock)		_clear_lock((slock_t *) (lock), 0)
-#endif	 /* _AIX */
-
-
-/* These are in s_lock.c */
-
-#if defined(__SUNPRO_C) && (defined(__i386) || defined(__x86_64__) || defined(__sparc__) || defined(__sparc))
-#define HAS_TEST_AND_SET
-
-#if defined(__i386) || defined(__x86_64__) || defined(__sparcv9) || defined(__sparcv8plus)
-typedef unsigned int slock_t;
-#else
-typedef unsigned char slock_t;
-#endif
-
-extern slock_t pg_atomic_cas(volatile slock_t *lock, slock_t with,
-									  slock_t cmp);
-
-#define TAS(a) (pg_atomic_cas((a), 1, 0) != 0)
-#endif
-
-
-#ifdef WIN32_ONLY_COMPILER
-typedef LONG slock_t;
-
-#define HAS_TEST_AND_SET
-#define TAS(lock) (InterlockedCompareExchange(lock, 1, 0))
-
-#define SPIN_DELAY() spin_delay()
-
-/* If using Visual C++ on Win64, inline assembly is unavailable.
- * Use a _mm_pause instrinsic instead of rep nop.
- */
-#if defined(_WIN64)
-static __forceinline void
-spin_delay(void)
-{
-	_mm_pause();
-}
-#else
-static __forceinline void
-spin_delay(void)
-{
-	/* See comment for gcc code. Same code, MASM syntax */
-	__asm rep nop;
-}
-#endif
-
-#endif
-
-
-#endif	/* !defined(HAS_TEST_AND_SET) */
-
-
-/* Blow up if we didn't have any way to do spinlocks */
-#ifndef HAS_TEST_AND_SET
-#error PostgreSQL does not have native spinlock support on this platform.  To continue the compilation, rerun configure using --disable-spinlocks.  However, performance will be poor.  Please report this to pgsql-bugs@postgresql.org.
-#endif
-
-
-#else	/* !HAVE_SPINLOCKS */
-
-
-/*
- * Fake spinlock implementation using semaphores --- slow and prone
- * to fall foul of kernel limits on number of semaphores, so don't use this
- * unless you must!  The subroutines appear in spin.c.
- */
-typedef PGSemaphoreData slock_t;
-
-extern bool s_lock_free_sema(volatile slock_t *lock);
-extern void s_unlock_sema(volatile slock_t *lock);
-extern void s_init_lock_sema(volatile slock_t *lock);
-extern int	tas_sema(volatile slock_t *lock);
-
-#define S_LOCK_FREE(lock)	s_lock_free_sema(lock)
-#define S_UNLOCK(lock)	 s_unlock_sema(lock)
-#define S_INIT_LOCK(lock)	s_init_lock_sema(lock)
-#define TAS(lock)	tas_sema(lock)
-
-
-#endif	/* HAVE_SPINLOCKS */
-
-
-/*
- * Default Definitions - override these above as needed.
- */
-
-#if !defined(S_LOCK)
+typedef pg_atomic_flag slock_t;
+#define TAS(lock)			(!pg_atomic_test_set_flag(lock))
+#define TAS_SPIN(lock)		(pg_atomic_unlocked_test_flag(lock) ? 1 : TAS(lock))
+#define S_UNLOCK(lock)		pg_atomic_clear_flag(lock)
+#define S_INIT_LOCK(lock)	pg_atomic_init_flag(lock)
+#define SPIN_DELAY()		pg_spin_delay()
 #define S_LOCK(lock) \
 	(TAS(lock) ? s_lock((lock), __FILE__, __LINE__) : 0)
-#endif	 /* S_LOCK */
-
-#if !defined(S_LOCK_FREE)
-#define S_LOCK_FREE(lock)	(*(lock) == 0)
-#endif	 /* S_LOCK_FREE */
-
-#if !defined(S_UNLOCK)
-#define S_UNLOCK(lock)		(*((volatile slock_t *) (lock)) = 0)
-#endif	 /* S_UNLOCK */
-
-#if !defined(S_INIT_LOCK)
-#define S_INIT_LOCK(lock)	S_UNLOCK(lock)
-#endif	 /* S_INIT_LOCK */
-
-#if !defined(SPIN_DELAY)
-#define SPIN_DELAY()	((void) 0)
-#endif	 /* SPIN_DELAY */
-
-#if !defined(TAS)
-extern int	tas(volatile slock_t *lock);		/* in port/.../tas.s, or
-												 * s_lock.c */
-
-#define TAS(lock)		tas(lock)
-#endif	 /* TAS */
-
-#if !defined(TAS_SPIN)
-#define TAS_SPIN(lock)	TAS(lock)
-#endif	 /* TAS_SPIN */
-
 
 /*
  * Platform-independent out-of-line support routines
diff --git a/src/test/regress/expected/lock.out b/src/test/regress/expected/lock.out
index 0d7c3ba..fd27344 100644
--- a/src/test/regress/expected/lock.out
+++ b/src/test/regress/expected/lock.out
@@ -60,3 +60,11 @@ DROP TABLE lock_tbl2;
 DROP TABLE lock_tbl1;
 DROP SCHEMA lock_schema1 CASCADE;
 DROP ROLE regress_rol_lock1;
+-- atomic ops tests
+RESET search_path;
+SELECT test_atomic_ops();
+ test_atomic_ops 
+-----------------
+ t
+(1 row)
+
diff --git a/src/test/regress/input/create_function_1.source b/src/test/regress/input/create_function_1.source
index aef1518..1fded84 100644
--- a/src/test/regress/input/create_function_1.source
+++ b/src/test/regress/input/create_function_1.source
@@ -57,6 +57,11 @@ CREATE FUNCTION make_tuple_indirect (record)
         AS '@libdir@/regress@DLSUFFIX@'
         LANGUAGE C STRICT;
 
+CREATE FUNCTION test_atomic_ops()
+    RETURNS bool
+    AS '@libdir@/regress@DLSUFFIX@'
+    LANGUAGE C;
+
 -- Things that shouldn't work:
 
 CREATE FUNCTION test1 (int) RETURNS int LANGUAGE SQL
diff --git a/src/test/regress/output/create_function_1.source b/src/test/regress/output/create_function_1.source
index 9761d12..9926c90 100644
--- a/src/test/regress/output/create_function_1.source
+++ b/src/test/regress/output/create_function_1.source
@@ -51,6 +51,10 @@ CREATE FUNCTION make_tuple_indirect (record)
         RETURNS record
         AS '@libdir@/regress@DLSUFFIX@'
         LANGUAGE C STRICT;
+CREATE FUNCTION test_atomic_ops()
+    RETURNS bool
+    AS '@libdir@/regress@DLSUFFIX@'
+    LANGUAGE C;
 -- Things that shouldn't work:
 CREATE FUNCTION test1 (int) RETURNS int LANGUAGE SQL
     AS 'SELECT ''not an integer'';';
diff --git a/src/test/regress/regress.c b/src/test/regress/regress.c
index 3bd8a15..1db996a 100644
--- a/src/test/regress/regress.c
+++ b/src/test/regress/regress.c
@@ -16,6 +16,7 @@
 #include "commands/trigger.h"
 #include "executor/executor.h"
 #include "executor/spi.h"
+#include "storage/atomics.h"
 #include "utils/builtins.h"
 #include "utils/geo_decls.h"
 #include "utils/rel.h"
@@ -40,6 +41,7 @@ extern int	oldstyle_length(int n, text *t);
 extern Datum int44in(PG_FUNCTION_ARGS);
 extern Datum int44out(PG_FUNCTION_ARGS);
 extern Datum make_tuple_indirect(PG_FUNCTION_ARGS);
+extern Datum test_atomic_ops(PG_FUNCTION_ARGS);
 
 #ifdef PG_MODULE_MAGIC
 PG_MODULE_MAGIC;
@@ -829,3 +831,214 @@ make_tuple_indirect(PG_FUNCTION_ARGS)
 
 	PG_RETURN_HEAPTUPLEHEADER(newtup->t_data);
 }
+
+
+static pg_atomic_flag global_flag = PG_ATOMIC_INIT_FLAG();
+
+static void
+test_atomic_flag(void)
+{
+	pg_atomic_flag flag;
+
+	/* check the global flag */
+	if (pg_atomic_unlocked_test_flag(&global_flag))
+		elog(ERROR, "global_flag: unduly set");
+
+	if (!pg_atomic_test_set_flag(&global_flag))
+		elog(ERROR, "global_flag: couldn't set");
+
+	if (pg_atomic_test_set_flag(&global_flag))
+		elog(ERROR, "global_flag: set spuriously #1");
+
+	pg_atomic_clear_flag(&global_flag);
+
+	if (!pg_atomic_test_set_flag(&global_flag))
+		elog(ERROR, "global_flag: couldn't set");
+
+	/* check the local flag */
+	pg_atomic_init_flag(&flag);
+
+	if (pg_atomic_unlocked_test_flag(&flag))
+		elog(ERROR, "flag: unduly set");
+
+	if (!pg_atomic_test_set_flag(&flag))
+		elog(ERROR, "flag: couldn't set");
+
+	if (pg_atomic_test_set_flag(&flag))
+		elog(ERROR, "flag: set spuriously #2");
+
+	pg_atomic_clear_flag(&flag);
+
+	if (!pg_atomic_test_set_flag(&flag))
+		elog(ERROR, "flag: couldn't set");
+}
+
+#ifdef PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+static void
+test_atomic_uint32(void)
+{
+	pg_atomic_uint32 var;
+	uint32 expected;
+	int i;
+
+	pg_atomic_init_u32(&var, 0);
+
+	if (pg_atomic_read_u32(&var) != 0)
+		elog(ERROR, "atomic_read_u32() #1 wrong");
+
+	pg_atomic_write_u32(&var, 3);
+
+	if (pg_atomic_read_u32(&var) != 3)
+		elog(ERROR, "atomic_read_u32() #2 wrong");
+
+	if (pg_atomic_fetch_add_u32(&var, 1) != 3)
+		elog(ERROR, "atomic_fetch_add_u32() #1 wrong");
+
+	if (pg_atomic_fetch_sub_u32(&var, 1) != 4)
+		elog(ERROR, "atomic_fetch_sub_u32() #1 wrong");
+
+	if (pg_atomic_sub_fetch_u32(&var, 3) != 0)
+		elog(ERROR, "atomic_sub_fetch_u32() #1 wrong");
+
+	if (pg_atomic_add_fetch_u32(&var, 10) != 10)
+		elog(ERROR, "atomic_add_fetch_u32() #1 wrong");
+
+	if (pg_atomic_exchange_u32(&var, 5) != 10)
+		elog(ERROR, "pg_atomic_exchange_u32() #1 wrong");
+
+	if (pg_atomic_exchange_u32(&var, 0) != 5)
+		elog(ERROR, "pg_atomic_exchange_u32() #0 wrong");
+
+	/* fail exchange because of old expected */
+	expected = 10;
+	if (pg_atomic_compare_exchange_u32(&var, &expected, 1))
+		elog(ERROR, "atomic_compare_exchange_u32() changed value spuriously");
+
+	/* CAS is allowed to fail due to interrupts, try a couple of times */
+	for (i = 0; i < 1000; i++)
+	{
+		expected = 0;
+		if (!pg_atomic_compare_exchange_u32(&var, &expected, 1))
+			break;
+	}
+	if (i == 1000)
+		elog(ERROR, "atomic_compare_exchange_u32() never succeeded");
+	if (pg_atomic_read_u32(&var) != 1)
+		elog(ERROR, "atomic_compare_exchange_u32() didn't set value properly");
+
+	pg_atomic_write_u32(&var, 0);
+
+	/* try setting flagbits */
+	if (pg_atomic_fetch_or_u32(&var, 1) & 1)
+		elog(ERROR, "pg_atomic_fetch_or_u32() #1 wrong");
+
+	if (!(pg_atomic_fetch_or_u32(&var, 2) & 1))
+		elog(ERROR, "pg_atomic_fetch_or_u32() #2 wrong");
+
+	if (pg_atomic_read_u32(&var) != 3)
+		elog(ERROR, "invalid result after pg_atomic_fetch_or_u32()");
+
+	/* try clearing flagbits */
+	if ((pg_atomic_fetch_and_u32(&var, ~2) & 3) != 3)
+		elog(ERROR, "pg_atomic_fetch_and_u32() #1 wrong");
+
+	if (pg_atomic_fetch_and_u32(&var, ~1) != 1)
+		elog(ERROR, "pg_atomic_fetch_and_u32() #2 wrong: is %u",
+			 pg_atomic_read_u32(&var));
+	/* no bits set anymore */
+	if (pg_atomic_fetch_and_u32(&var, ~0) != 0)
+		elog(ERROR, "pg_atomic_fetch_and_u32() #3 wrong");
+}
+#endif /* PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32 */
+
+#ifdef PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+static void
+test_atomic_uint64(void)
+{
+	pg_atomic_uint64 var;
+	uint64 expected;
+	int i;
+
+	pg_atomic_init_u64(&var, 0);
+
+	if (pg_atomic_read_u64(&var) != 0)
+		elog(ERROR, "atomic_read_u64() #1 wrong");
+
+	pg_atomic_write_u64(&var, 3);
+
+	if (pg_atomic_read_u64(&var) != 3)
+		elog(ERROR, "atomic_read_u64() #2 wrong");
+
+	if (pg_atomic_fetch_add_u64(&var, 1) != 3)
+		elog(ERROR, "atomic_fetch_add_u64() #1 wrong");
+
+	if (pg_atomic_fetch_sub_u64(&var, 1) != 4)
+		elog(ERROR, "atomic_fetch_sub_u64() #1 wrong");
+
+	if (pg_atomic_sub_fetch_u64(&var, 3) != 0)
+		elog(ERROR, "atomic_sub_fetch_u64() #1 wrong");
+
+	if (pg_atomic_add_fetch_u64(&var, 10) != 10)
+		elog(ERROR, "atomic_add_fetch_u64() #1 wrong");
+
+	if (pg_atomic_exchange_u64(&var, 5) != 10)
+		elog(ERROR, "pg_atomic_exchange_u64() #1 wrong");
+
+	if (pg_atomic_exchange_u64(&var, 0) != 5)
+		elog(ERROR, "pg_atomic_exchange_u64() #0 wrong");
+
+	/* fail exchange because of old expected */
+	expected = 10;
+	if (pg_atomic_compare_exchange_u64(&var, &expected, 1))
+		elog(ERROR, "atomic_compare_exchange_u64() changed value spuriously");
+
+	/* CAS is allowed to fail due to interrupts, try a couple of times */
+	for (i = 0; i < 100; i++)
+	{
+		expected = 0;
+		if (!pg_atomic_compare_exchange_u64(&var, &expected, 1))
+			break;
+	}
+	if (i == 100)
+		elog(ERROR, "atomic_compare_exchange_u64() never succeeded");
+	if (pg_atomic_read_u64(&var) != 1)
+		elog(ERROR, "atomic_compare_exchange_u64() didn't set value properly");
+
+	pg_atomic_write_u64(&var, 0);
+
+	/* try setting flagbits */
+	if (pg_atomic_fetch_or_u64(&var, 1) & 1)
+		elog(ERROR, "pg_atomic_fetch_or_u64() #1 wrong");
+
+	if (!(pg_atomic_fetch_or_u64(&var, 2) & 1))
+		elog(ERROR, "pg_atomic_fetch_or_u64() #2 wrong");
+
+	if (pg_atomic_read_u64(&var) != 3)
+		elog(ERROR, "invalid result after pg_atomic_fetch_or_u64()");
+
+	/* try clearing flagbits */
+	if ((pg_atomic_fetch_and_u64(&var, ~2) & 3) != 3)
+		elog(ERROR, "pg_atomic_fetch_and_u64() #1 wrong");
+
+	if (pg_atomic_fetch_and_u64(&var, ~1) != 1)
+		elog(ERROR, "pg_atomic_fetch_and_u64() #2 wrong: is %u",
+			 pg_atomic_read_u64(&var));
+	/* no bits set anymore */
+	if (pg_atomic_fetch_and_u64(&var, ~0) != 0)
+		elog(ERROR, "pg_atomic_fetch_and_u64() #3 wrong");
+}
+#endif /* PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64 */
+
+
+extern Datum test_atomic_ops(PG_FUNCTION_ARGS)
+{
+	test_atomic_flag();
+#ifdef PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U32
+	test_atomic_uint32();
+#endif
+#ifdef PG_HAVE_ATOMIC_COMPARE_EXCHANGE_U64
+	test_atomic_uint64();
+#endif
+
+	PG_RETURN_BOOL(true);
+}
diff --git a/src/test/regress/sql/lock.sql b/src/test/regress/sql/lock.sql
index dda212f..567e8bc 100644
--- a/src/test/regress/sql/lock.sql
+++ b/src/test/regress/sql/lock.sql
@@ -64,3 +64,8 @@ DROP TABLE lock_tbl2;
 DROP TABLE lock_tbl1;
 DROP SCHEMA lock_schema1 CASCADE;
 DROP ROLE regress_rol_lock1;
+
+
+-- atomic ops tests
+RESET search_path;
+SELECT test_atomic_ops();
-- 
1.8.5.rc1.dirty

0003-Convert-the-PGPROC-lwWaitLink-list-into-a-dlist-inst.patchtext/x-patch; charset=us-asciiDownload
>From 18eff6f07848d1e069e9552167285580b3005830 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Fri, 15 Nov 2013 19:00:33 +0100
Subject: [PATCH 3/4] Convert the PGPROC->lwWaitLink list into a dlist instead
 of open coding it.

Besides being shorter and much easier to read it:
* fixes a bug in xlog.c:WakeupWaiters() accidentally waking up
  exclusive lockers
* changes the logic in LWLockRelease() to release all shared lockers
  when waking up any.
* adds a memory pg_write_barrier() in the wakeup paths between
  dequeuing and unsetting ->lwWaiting. On a weakly ordered machine the
  current coding could lead to problems.
---
 src/backend/access/transam/twophase.c |   1 -
 src/backend/access/transam/xlog.c     | 127 +++++++++++++---------------------
 src/backend/storage/lmgr/lwlock.c     | 106 +++++++++++-----------------
 src/backend/storage/lmgr/proc.c       |   2 -
 src/include/storage/proc.h            |   3 +-
 5 files changed, 90 insertions(+), 149 deletions(-)

diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e975f8d..7aeb6e8 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -326,7 +326,6 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 	proc->roleId = owner;
 	proc->lwWaiting = false;
 	proc->lwWaitMode = 0;
-	proc->lwWaitLink = NULL;
 	proc->waitLock = NULL;
 	proc->waitProcLock = NULL;
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a95149b..ea2fb93 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -393,8 +393,7 @@ typedef struct
 
 	bool		releaseOK;		/* T if ok to release waiters */
 	char		exclusive;		/* # of exclusive holders (0 or 1) */
-	PGPROC	   *head;			/* head of list of waiting PGPROCs */
-	PGPROC	   *tail;			/* tail of list of waiting PGPROCs */
+	dlist_head	waiters;		/* list of waiting PGPROCs */
 	/* tail is undefined when head is NULL */
 } XLogInsertSlot;
 
@@ -1626,12 +1625,7 @@ WALInsertSlotAcquireOne(int slotno)
 		 */
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_EXCLUSIVE;
-		proc->lwWaitLink = NULL;
-		if (slot->head == NULL)
-			slot->head = proc;
-		else
-			slot->tail->lwWaitLink = proc;
-		slot->tail = proc;
+		dlist_push_tail((dlist_head *) &slot->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&slot->mutex);
@@ -1746,13 +1740,8 @@ WaitOnSlot(volatile XLogInsertSlot *slot, XLogRecPtr waitptr)
 		 */
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-
 		/* waiters are added to the front of the queue */
-		proc->lwWaitLink = slot->head;
-		if (slot->head == NULL)
-			slot->tail = proc;
-		slot->head = proc;
+		dlist_push_head((dlist_head *) &slot->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&slot->mutex);
@@ -1804,9 +1793,8 @@ static void
 WakeupWaiters(XLogRecPtr EndPos)
 {
 	volatile XLogInsertSlot *slot = &XLogCtl->Insert.insertSlots[MySlotNo].slot;
-	PGPROC	   *head;
-	PGPROC	   *proc;
-	PGPROC	   *next;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
 
 	/*
 	 * If we have already reported progress up to the same point, do nothing.
@@ -1818,6 +1806,8 @@ WakeupWaiters(XLogRecPtr EndPos)
 	/* xlogInsertingAt should not go backwards */
 	Assert(slot->xlogInsertingAt < EndPos);
 
+	dlist_init(&wakeup);
+
 	/* Acquire mutex.  Time spent holding mutex should be short! */
 	SpinLockAcquire(&slot->mutex);
 
@@ -1827,25 +1817,19 @@ WakeupWaiters(XLogRecPtr EndPos)
 	slot->xlogInsertingAt = EndPos;
 
 	/*
-	 * See if there are any waiters that need to be woken up.
+	 * See if there are any LW_WAIT_UNTIL_FREE waiters that need to be woken
+	 * up.
 	 */
-	head = slot->head;
-
-	if (head != NULL)
+	dlist_foreach_modify(iter, (dlist_head *) &slot->waiters)
 	{
-		proc = head;
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
 		/* LW_WAIT_UNTIL_FREE waiters are always in the front of the queue */
-		next = proc->lwWaitLink;
-		while (next && next->lwWaitMode == LW_WAIT_UNTIL_FREE)
-		{
-			proc = next;
-			next = next->lwWaitLink;
-		}
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+			break;
 
-		/* proc is now the last PGPROC to be released */
-		slot->head = next;
-		proc->lwWaitLink = NULL;
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 	}
 
 	/* We are done updating shared state of the lock itself. */
@@ -1854,13 +1838,13 @@ WakeupWaiters(XLogRecPtr EndPos)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, (dlist_head *) &wakeup)
 	{
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 }
 
@@ -1886,8 +1870,11 @@ static void
 WALInsertSlotReleaseOne(int slotno)
 {
 	volatile XLogInsertSlot *slot = &XLogCtl->Insert.insertSlots[slotno].slot;
-	PGPROC	   *head;
-	PGPROC	   *proc;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+	bool		releaseOK = true;
+
+	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
 	SpinLockAcquire(&slot->mutex);
@@ -1904,43 +1891,28 @@ WALInsertSlotReleaseOne(int slotno)
 	/*
 	 * See if I need to awaken any waiters..
 	 */
-	head = slot->head;
-	if (head != NULL)
+	dlist_foreach_modify(iter, (dlist_head *) &slot->waiters)
 	{
-		if (slot->releaseOK)
-		{
-			/*
-			 * Remove the to-be-awakened PGPROCs from the queue.
-			 */
-			bool		releaseOK = true;
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-			proc = head;
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 
-			/*
-			 * First wake up any backends that want to be woken up without
-			 * acquiring the lock. These are always in the front of the queue.
-			 */
-			while (proc->lwWaitMode == LW_WAIT_UNTIL_FREE && proc->lwWaitLink)
-				proc = proc->lwWaitLink;
-
-			/*
-			 * Awaken the first exclusive-waiter, if any.
-			 */
-			if (proc->lwWaitLink)
-			{
-				Assert(proc->lwWaitLink->lwWaitMode == LW_EXCLUSIVE);
-				proc = proc->lwWaitLink;
-				releaseOK = false;
-			}
-			/* proc is now the last PGPROC to be released */
-			slot->head = proc->lwWaitLink;
-			proc->lwWaitLink = NULL;
+		/*
+		 * First wake up any backends that want to be woken up without
+		 * acquiring the lock. These are always in the front of the
+		 * queue. Then wakeup the first exclusive-waiter, if any.
+		 */
 
-			slot->releaseOK = releaseOK;
+		/* LW_WAIT_UNTIL_FREE waiters are always in the front of the queue */
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+		{
+			Assert(waiter->lwWaitMode == LW_EXCLUSIVE);
+			releaseOK = false;
+			break;
 		}
-		else
-			head = NULL;
 	}
+	slot->releaseOK = releaseOK;
 
 	/* We are done updating shared state of the slot itself. */
 	SpinLockRelease(&slot->mutex);
@@ -1948,13 +1920,13 @@ WALInsertSlotReleaseOne(int slotno)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, (dlist_head *) &wakeup)
 	{
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 
 	/*
@@ -5104,8 +5076,7 @@ XLOGShmemInit(void)
 
 		slot->releaseOK = true;
 		slot->exclusive = 0;
-		slot->head = NULL;
-		slot->tail = NULL;
+		dlist_init(&slot->waiters);
 	}
 
 	SpinLockInit(&XLogCtl->Insert.insertpos_lck);
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 4f88d3f..732a5d2 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -25,6 +25,7 @@
 #include "access/multixact.h"
 #include "access/subtrans.h"
 #include "commands/async.h"
+#include "lib/ilist.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
 #include "storage/ipc.h"
@@ -43,8 +44,7 @@ typedef struct LWLock
 	bool		releaseOK;		/* T if ok to release waiters */
 	char		exclusive;		/* # of exclusive holders (0 or 1) */
 	int			shared;			/* # of shared holders (0..MaxBackends) */
-	PGPROC	   *head;			/* head of list of waiting PGPROCs */
-	PGPROC	   *tail;			/* tail of list of waiting PGPROCs */
+	dlist_head	waiters;
 	/* tail is undefined when head is NULL */
 } LWLock;
 
@@ -105,9 +105,9 @@ inline static void
 PRINT_LWDEBUG(const char *where, LWLockId lockid, const volatile LWLock *lock)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%d): excl %d shared %d head %p rOK %d",
+		elog(LOG, "%s(%d): excl %d shared %d rOK %d",
 			 where, (int) lockid,
-			 (int) lock->exclusive, lock->shared, lock->head,
+			 (int) lock->exclusive, lock->shared,
 			 (int) lock->releaseOK);
 }
 
@@ -287,8 +287,7 @@ CreateLWLocks(void)
 		lock->lock.releaseOK = true;
 		lock->lock.exclusive = 0;
 		lock->lock.shared = 0;
-		lock->lock.head = NULL;
-		lock->lock.tail = NULL;
+		dlist_init(&lock->lock.waiters);
 	}
 
 	/*
@@ -444,12 +443,7 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = mode;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head((dlist_head *) &lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -657,12 +651,7 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head((dlist_head *) &lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -728,10 +717,12 @@ void
 LWLockRelease(LWLockId lockid)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
-	PGPROC	   *head;
-	PGPROC	   *proc;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
 	int			i;
 
+	dlist_init(&wakeup);
+
 	PRINT_LWDEBUG("LWLockRelease", lockid, lock);
 
 	/*
@@ -767,58 +758,39 @@ LWLockRelease(LWLockId lockid)
 	 * if someone has already awakened waiters that haven't yet acquired the
 	 * lock.
 	 */
-	head = lock->head;
-	if (head != NULL)
+	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
 	{
-		if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
+		/*
+		 * Remove the to-be-awakened PGPROCs from the queue.
+		 */
+		bool		releaseOK = true;
+		bool		wokeup_somebody = false;
+
+		dlist_foreach_modify(iter, (dlist_head *) &lock->waiters)
 		{
-			/*
-			 * Remove the to-be-awakened PGPROCs from the queue.
-			 */
-			bool		releaseOK = true;
+			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-			proc = head;
+			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+				continue;
 
-			/*
-			 * First wake up any backends that want to be woken up without
-			 * acquiring the lock.
-			 */
-			while (proc->lwWaitMode == LW_WAIT_UNTIL_FREE && proc->lwWaitLink)
-				proc = proc->lwWaitLink;
+			dlist_delete(&waiter->lwWaitLink);
+			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 
 			/*
-			 * If the front waiter wants exclusive lock, awaken him only.
-			 * Otherwise awaken as many waiters as want shared access.
+			 * Prevent additional wakeups until retryer gets to
+			 * run. Backends that are just waiting for the lock to become
+			 * free don't retry automatically.
 			 */
-			if (proc->lwWaitMode != LW_EXCLUSIVE)
+			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
 			{
-				while (proc->lwWaitLink != NULL &&
-					   proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
-				{
-					if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
-						releaseOK = false;
-					proc = proc->lwWaitLink;
-				}
-			}
-			/* proc is now the last PGPROC to be released */
-			lock->head = proc->lwWaitLink;
-			proc->lwWaitLink = NULL;
-
-			/*
-			 * Prevent additional wakeups until retryer gets to run. Backends
-			 * that are just waiting for the lock to become free don't retry
-			 * automatically.
-			 */
-			if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
 				releaseOK = false;
+				wokeup_somebody = true;
+			}
 
-			lock->releaseOK = releaseOK;
-		}
-		else
-		{
-			/* lock is still held, can't awaken anything */
-			head = NULL;
+			if(waiter->lwWaitMode == LW_EXCLUSIVE)
+				break;
 		}
+		lock->releaseOK = releaseOK;
 	}
 
 	/* We are done updating shared state of the lock itself. */
@@ -829,14 +801,14 @@ LWLockRelease(LWLockId lockid)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, (dlist_head *) &wakeup)
 	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 		LOG_LWDEBUG("LWLockRelease", lockid, "release waiter");
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 
 	/*
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 222251d..75bddcd 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -370,7 +370,6 @@ InitProcess(void)
 		MyPgXact->vacuumFlags |= PROC_IS_AUTOVACUUM;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
@@ -533,7 +532,6 @@ InitAuxiliaryProcess(void)
 	MyPgXact->vacuumFlags = 0;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 3b04d3c..73cea1c 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -15,6 +15,7 @@
 #define _PROC_H_
 
 #include "access/xlogdefs.h"
+#include "lib/ilist.h"
 #include "storage/latch.h"
 #include "storage/lock.h"
 #include "storage/pg_sema.h"
@@ -101,7 +102,7 @@ struct PGPROC
 	/* Info about LWLock the process is currently waiting for, if any. */
 	bool		lwWaiting;		/* true if waiting for an LW lock */
 	uint8		lwWaitMode;		/* lwlock mode being waited for */
-	struct PGPROC *lwWaitLink;	/* next waiter for same LW lock */
+	dlist_node	lwWaitLink;		/* next waiter for same LW lock */
 
 	/* Info about lock the process is currently waiting for, if any. */
 	/* waitLock and waitProcLock are NULL if not currently waiting. */
-- 
1.8.5.rc1.dirty

0004-Wait-free-LW_SHARED-lwlock-acquiration.patchtext/x-patch; charset=us-asciiDownload
>From 4cc9fd429a073cccf648fdb457ed9999534aa58e Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Tue, 24 Sep 2013 03:04:44 +0200
Subject: [PATCH 4/4] Wait free LW_SHARED lwlock acquiration

---
 src/backend/storage/lmgr/lwlock.c | 754 ++++++++++++++++++++++++++------------
 1 file changed, 523 insertions(+), 231 deletions(-)

diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 732a5d2..fe8858f 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -17,6 +17,82 @@
  * IDENTIFICATION
  *	  src/backend/storage/lmgr/lwlock.c
  *
+ * NOTES:
+ *
+ * This used to be a pretty straight forward reader-writer lock
+ * implementation, in which the internal state was protected by a
+ * spinlock. Unfortunately the overhead of taking the spinlock proved to be
+ * too high for workloads/locks that were locked in shared mode very
+ * frequently.
+ * Thus a new implementation was devised that provides wait-free shared lock
+ * acquiration for locks that aren't exclusively locked.
+ *
+ * The basic idea is to have a single atomic variable 'lockcount' instead of
+ * the formerly separate shared and exclusive counters and to use an atomic
+ * increment to acquire the lock. That's fairly easy to do for rw-spinlocks,
+ * but a lot harder for something like LWLocks that want to wait in the OS.
+ *
+ * For exlusive lock acquisition we use an atomic compare-and-exchange on the
+ * lockcount variable swapping in EXCLUSIVE_LOCK/1<<31/0x80000000 if and only
+ * if the current value of lockcount is 0. If the swap was not successfull, we
+ * have to wait.
+ *
+ * For shared lock acquisition we use an atomic add (lock xadd) to the
+ * lockcount variable to add 1. If the value is bigger than EXCLUSIVE_LOCK we
+ * know that somebody actually has an exclusive lock, and we back out by
+ * atomically decrementing by 1 again. If so, we have to wait for the exlusive
+ * locker to release the lock.
+ *
+ * To release the lock we use an atomic decrement to release the lock. If the
+ * new value is zero (we get that atomically), we know we have to release
+ * waiters.
+ *
+ * The attentive reader probably might have noticed that naively doing the
+ * above has two glaring race conditions:
+ *
+ * 1) too-quick-for-queueing: We try to lock using the atomic operations and
+ * notice that we have to wait. Unfortunately until we have finished queuing,
+ * the former locker very well might have already finished it's work. That's
+ * problematic because we're now stuck waiting inside the OS.
+ *
+ * 2) spurious failed locks: Due to the logic of backing out of shared
+ * locks after we unconditionally added a 1 to lockcount, we might have
+ * prevented another exclusive locker from getting the lock:
+ *   1) Session A: LWLockAcquire(LW_EXCLUSIVE) - success
+ *   2) Session B: LWLockAcquire(LW_SHARED) - lockcount += 1
+ *   3) Session B: LWLockAcquire(LW_SHARED) - oops, bigger than EXCLUSIVE_LOCK
+ *   4) Session A: LWLockRelease()
+ *   5) Session C: LWLockAcquire(LW_EXCLUSIVE) - check if lockcount = 0, no. wait.
+ *   6) Session B: LWLockAcquire(LW_SHARED) - lockcount -= 1
+ *   7) Session B: LWLockAcquire(LW_SHARED) - wait
+ *
+ * So we'd now have both B) and C) waiting on a lock that nobody is holding
+ * anymore. Not good.
+ *
+ * To mitigate those races we use a two phased attempt at locking:
+ *   Phase 1: * Try to do it atomically, if we succeed, nice
+ *   Phase 2: Add us too the waitqueue of the lock
+ *   Phase 3: Try to grab the lock again, if we succeed, remove ourselves from
+ *            the queue
+ *   Phase 4: Sleep till wakeup, goto Phase 1
+ *
+ * This protects us against both problems from above:
+ * 1) Nobody can release too quick, before we're queued, since after Phase 2 since we're
+ *    already queued.
+ * 2) If somebody spuriously got blocked from acquiring the lock, they will
+ *    get queued in Phase 2 and we can wake them up if neccessary or they will
+ *    have gotten the lock in Phase 2.
+ *
+ * There above algorithm only works for LWLockAcquire, not directly for
+ * LWLockAcquireConditional where we don't want to wait. In that case we just
+ * need to retry acquiring the lock until we're sure we didn't disturb anybody
+ * in doing so.
+ *
+ * TODO:
+ * - decide if we need a spinlock fallback
+ * - expand documentation
+ * - make LWLOCK_STATS do something sensible again
+ * - make LOCK_DEBUG output nicer
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -28,6 +104,7 @@
 #include "lib/ilist.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
+#include "storage/atomics.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
@@ -37,15 +114,20 @@
 /* We use the ShmemLock spinlock to protect LWLockAssign */
 extern slock_t *ShmemLock;
 
+#define EXCLUSIVE_LOCK ((uint32) 1 << 31)
+/* must be greater than MAX_BACKENDS */
+#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)
 
 typedef struct LWLock
 {
-	slock_t		mutex;			/* Protects LWLock and queue of PGPROCs */
-	bool		releaseOK;		/* T if ok to release waiters */
-	char		exclusive;		/* # of exclusive holders (0 or 1) */
-	int			shared;			/* # of shared holders (0..MaxBackends) */
-	dlist_head	waiters;
-	/* tail is undefined when head is NULL */
+	slock_t		mutex;		/* Protects LWLock and queue of PGPROCs */
+	bool		releaseOK;	/* T if ok to release waiters */
+	dlist_head	waiters;	/* list of waiters */
+	pg_atomic_uint32 lockcount;	/* state of exlusive/nonexclusive lockers */
+	pg_atomic_uint32 nwaiters; /* number of waiters */
+#ifdef LWLOCK_DEBUG
+	PGPROC	   *owner;
+#endif
 } LWLock;
 
 /*
@@ -60,7 +142,7 @@ typedef struct LWLock
  * LWLock is between 16 and 32 bytes on all known platforms, so these two
  * cases are sufficient.
  */
-#define LWLOCK_PADDED_SIZE	(sizeof(LWLock) <= 16 ? 16 : 32)
+#define LWLOCK_PADDED_SIZE	64
 
 typedef union LWLockPadded
 {
@@ -75,7 +157,6 @@ typedef union LWLockPadded
  */
 NON_EXEC_STATIC LWLockPadded *LWLockArray = NULL;
 
-
 /*
  * We use this structure to keep track of locked LWLocks for release
  * during error recovery.  The maximum size could be determined at runtime
@@ -84,8 +165,14 @@ NON_EXEC_STATIC LWLockPadded *LWLockArray = NULL;
  */
 #define MAX_SIMUL_LWLOCKS	100
 
+typedef struct LWLockHandle
+{
+	LWLockId lock;
+	LWLockMode mode;
+} LWLockHandle;
+
 static int	num_held_lwlocks = 0;
-static LWLockId held_lwlocks[MAX_SIMUL_LWLOCKS];
+static LWLockHandle held_lwlocks[MAX_SIMUL_LWLOCKS];
 
 static int	lock_addin_request = 0;
 static bool lock_addin_request_allowed = true;
@@ -102,24 +189,29 @@ static int *spin_delay_counts;
 bool		Trace_lwlocks = false;
 
 inline static void
-PRINT_LWDEBUG(const char *where, LWLockId lockid, const volatile LWLock *lock)
+PRINT_LWDEBUG(const char *where, LWLockId lockid, volatile LWLock *lock, LWLockMode mode)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%d): excl %d shared %d rOK %d",
-			 where, (int) lockid,
-			 (int) lock->exclusive, lock->shared,
+	{
+		uint32 lockcount = pg_atomic_read_u32(&lock->lockcount);
+		elog(LOG, "%s(%d)%u: excl %u shared %u waiters %u rOK %d",
+			 where, (int) lockid, mode,
+			 (lockcount & EXCLUSIVE_LOCK) >> 31,
+			 (lockcount & SHARED_LOCK_MASK),
+			 pg_atomic_read_u32(&lock->nwaiters),
 			 (int) lock->releaseOK);
+	}
 }
 
 inline static void
-LOG_LWDEBUG(const char *where, LWLockId lockid, const char *msg)
+LOG_LWDEBUG(const char *where, LWLockId lockid, LWLockMode mode, const char *msg)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%d): %s", where, (int) lockid, msg);
+		elog(LOG, "%s(%d)%u: %s", where, (int) lockid, mode, msg);
 }
 #else							/* not LOCK_DEBUG */
-#define PRINT_LWDEBUG(a,b,c)
-#define LOG_LWDEBUG(a,b,c)
+#define PRINT_LWDEBUG(a,b,c,d) ((void)0)
+#define LOG_LWDEBUG(a,b,c,d) ((void)0)
 #endif   /* LOCK_DEBUG */
 
 #ifdef LWLOCK_STATS
@@ -285,8 +377,8 @@ CreateLWLocks(void)
 	{
 		SpinLockInit(&lock->lock.mutex);
 		lock->lock.releaseOK = true;
-		lock->lock.exclusive = 0;
-		lock->lock.shared = 0;
+		pg_atomic_init_u32(&lock->lock.lockcount, 0);
+		pg_atomic_init_u32(&lock->lock.nwaiters, 0);
 		dlist_init(&lock->lock.waiters);
 	}
 
@@ -328,6 +420,231 @@ LWLockAssign(void)
 	return result;
 }
 
+/*
+ * Internal function handling the atomic manipulation of lock->lockcount.
+ *
+ * 'double_check' = true means that we try to check more carefully
+ * against causing somebody else to spuriously believe the lock is
+ * already taken, although we're just about to back out of it.
+ */
+static inline bool
+LWLockAttemptLock(volatile LWLock* lock, LWLockMode mode, bool double_check, bool *potentially_spurious)
+{
+	bool mustwait;
+	uint32 oldstate;
+
+	Assert(mode == LW_EXCLUSIVE || mode == LW_SHARED);
+
+	*potentially_spurious = false;
+
+	if (mode == LW_EXCLUSIVE)
+	{
+		uint32 expected = 0;
+		pg_read_barrier();
+
+		/* check without CAS first; it's way cheaper, frequently locked otherwise */
+		if (pg_atomic_read_u32(&lock->lockcount) != 0)
+			mustwait = true;
+		else if (!pg_atomic_compare_exchange_u32(&lock->lockcount,
+												 &expected, EXCLUSIVE_LOCK))
+		{
+			/*
+			 * ok, no can do. Between the pg_atomic_read() above and the
+			 * CAS somebody else acquired the lock.
+			 */
+			mustwait = true;
+		}
+		else
+		{
+			/* yipeyyahee */
+			mustwait = false;
+#ifdef LWLOCK_DEBUG
+			lock->owner = MyProc;
+#endif
+		}
+	}
+	else
+	{
+		/*
+		 * If requested by caller, do an unlocked check first.  This is useful
+		 * if potentially spurious results have a noticeable cost.
+		 */
+		if (double_check)
+		{
+			pg_read_barrier();
+			if (pg_atomic_read_u32(&lock->lockcount) >= EXCLUSIVE_LOCK)
+			{
+				mustwait = true;
+				goto out;
+			}
+		}
+
+		/*
+		 * Acquire the share lock unconditionally using an atomic addition. We
+		 * might have to back out again if it turns out somebody else has an
+		 * exclusive lock.
+		 */
+		oldstate = pg_atomic_fetch_add_u32(&lock->lockcount, 1);
+
+		if (oldstate >= EXCLUSIVE_LOCK)
+		{
+			/*
+			 * Ok, somebody else holds the lock exclusively. We need to back
+			 * away from the shared lock, since we don't actually hold it right
+			 * now.  Since there's a window between lockcount += 1 and lockcount
+			 * -= 1, the previous exclusive locker could have released and
+			 * another exclusive locker could have seen our +1. We need to
+			 * signal that to the upper layers so they can deal with the race
+			 * condition.
+			 */
+
+			/*
+			 * FIXME: check return value if (double_check), it's not
+			 * spurious if still exclusively locked.
+			 */
+			pg_atomic_fetch_sub_u32(&lock->lockcount, 1);
+
+
+			mustwait = true;
+			*potentially_spurious = true;
+		}
+		else
+		{
+			/* yipeyyahee */
+			mustwait = false;
+		}
+	}
+
+out:
+	return mustwait;
+}
+
+/*
+ * Wakeup all the lockers that currently have a chance to run.
+ */
+static void
+LWLockWakeup(volatile LWLock *lock, LWLockId lockid, LWLockMode mode)
+{
+	bool		releaseOK;
+	bool		wokeup_somebody = false;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+
+	dlist_init(&wakeup);
+
+	/* remove the to-be-awakened PGPROCs from the queue */
+	releaseOK = true;
+
+	/* Acquire mutex.  Time spent holding mutex should be short! */
+	SpinLockAcquire(&lock->mutex);
+
+	dlist_foreach_modify(iter, (dlist_head *) &lock->waiters)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+			continue;
+
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
+
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+		{
+			/*
+			 * Prevent additional wakeups until retryer gets to run. Backends
+			 * that are just waiting for the lock to become free don't retry
+			 * automatically.
+			 */
+			releaseOK = false;
+			/*
+			 * Don't wakeup (further) exclusive locks.
+			 */
+			wokeup_somebody = true;
+		}
+
+		/*
+		 * Once we've woken up an exclusive lock, there's no point in waking
+		 * up anybody else.
+		 */
+		if(waiter->lwWaitMode == LW_EXCLUSIVE)
+			break;
+	}
+	lock->releaseOK = releaseOK;
+
+	/* We are done updating shared state of the lock queue. */
+	SpinLockRelease(&lock->mutex);
+
+	/*
+	 * Awaken any waiters I removed from the queue.
+	 */
+	dlist_foreach_modify(iter, (dlist_head *) &wakeup)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		LOG_LWDEBUG("LWLockRelease", lockid, mode, "release waiter");
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
+	}
+}
+
+/*
+ * Add ourselves to the end of the queue. Mode can be LW_WAIT_UNTIL_FREE here!
+ */
+static inline void
+LWLockQueueSelf(volatile LWLock *lock, LWLockMode mode)
+{
+	/*
+	 * If we don't have a PGPROC structure, there's no way to wait. This
+	 * should never occur, since MyProc should only be null during shared
+	 * memory initialization.
+	 */
+	if (MyProc == NULL)
+		elog(PANIC, "cannot wait without a PGPROC structure");
+
+	pg_atomic_fetch_add_u32(&lock->nwaiters, 1);
+
+	SpinLockAcquire(&lock->mutex);
+	MyProc->lwWaiting = true;
+	MyProc->lwWaitMode = mode;
+	dlist_push_tail((dlist_head *) &lock->waiters, &MyProc->lwWaitLink);
+
+	/* Can release the mutex now */
+	SpinLockRelease(&lock->mutex);
+}
+
+/*
+ * Remove ourselves from the waitlist.  This is used if we queued ourselves
+ * because we thought we needed to sleep but, after further checking, we
+ * discover that we don't actually need to do so. Somebody else might have
+ * already woken us up, in that case return false.
+ */
+static inline bool
+LWLockDequeueSelf(volatile LWLock *lock)
+{
+	bool	found = false;
+	dlist_mutable_iter iter;
+
+	SpinLockAcquire(&lock->mutex);
+
+	/* need to iterate, somebody else could have unqueued us */
+	dlist_foreach_modify(iter, (dlist_head *) &lock->waiters)
+	{
+		PGPROC *proc = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		if (proc == MyProc)
+		{
+			found = true;
+			dlist_delete(&proc->lwWaitLink);
+			break;
+		}
+	}
+
+	SpinLockRelease(&lock->mutex);
+
+	if (found)
+		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+	return found;
+}
 
 /*
  * LWLockAcquire - acquire a lightweight lock in the specified mode
@@ -341,10 +658,13 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
 	PGPROC	   *proc = MyProc;
-	bool		retry = false;
 	int			extraWaits = 0;
+	bool		potentially_spurious;
+	uint32		iterations = 0;
 
-	PRINT_LWDEBUG("LWLockAcquire", lockid, lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockAcquire", lockid, lock, mode);
 
 #ifdef LWLOCK_STATS
 	/* Set up local count state first time through in a given process */
@@ -395,58 +715,71 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 	{
 		bool		mustwait;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
 #ifdef LWLOCK_STATS
+		/* Acquire mutex.  Time spent holding mutex should be short! */
+		/* FIXME this stuff is completely useless now.  Should consider a
+		 * different way to do accounting -- perhaps at LWLockAttemptLock? */
 		spin_delay_counts[lockid] += SpinLockAcquire(&lock->mutex);
-#else
-		SpinLockAcquire(&lock->mutex);
 #endif
 
-		/* If retrying, allow LWLockRelease to release waiters again */
-		if (retry)
-			lock->releaseOK = true;
+		/*
+		 * try to grab the lock the first time, we're not in the waitqueue yet.
+		 */
+		mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious);
+
+		if (!mustwait)
+			break;				/* got the lock */
+
+		/*
+		 * Ok, at this point we couldn't grab the lock on the first try. We
+		 * cannot simply queue ourselves to the end of the list and wait to be
+		 * woken up because by now the lock could long have been released.
+		 * Instead add us to the queue and try to grab the lock again. If we
+		 * suceed we need to revert the queuing and be happy, otherwise we
+		 * recheck the lock. If we still couldn't grab it, we know that the
+		 * other lock will see our queue entries when releasing since they
+		 * existed before we checked for the lock.
+		 */
+
+		/* add to the queue */
+		LWLockQueueSelf(lock, mode);
 
-		/* If I can get the lock, do so quickly. */
-		if (mode == LW_EXCLUSIVE)
+		/* we're now guaranteed to be woken up if necessary */
+		mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious);
+
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
 		{
-			if (lock->exclusive == 0 && lock->shared == 0)
+			if (!LWLockDequeueSelf(lock))
 			{
-				lock->exclusive++;
-				mustwait = false;
+				/*
+				 * Somebody else dequeued us and has or will wake us up. Wait
+				 * for the correct wakeup, otherwise our ->lwWaiting would get
+				 * reset at some inconvenient point later, and releaseOk
+				 * wouldn't be managed correctly.
+				 */
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+				lock->releaseOK = true;
 			}
-			else
-				mustwait = true;
+			PRINT_LWDEBUG("LWLockAcquire undo queue", lockid, lock, mode);
+			break;
 		}
 		else
 		{
-			if (lock->exclusive == 0)
-			{
-				lock->shared++;
-				mustwait = false;
-			}
-			else
-				mustwait = true;
+			PRINT_LWDEBUG("LWLockAcquire waiting 4", lockid, lock, mode);
 		}
 
-		if (!mustwait)
-			break;				/* got the lock */
-
 		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait. This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
+		 * NB: There's no need to deal with spurious lock attempts
+		 * here. Anyone we prevented from acquiring the lock will
+		 * enqueue themselves using the same protocol we used here.
 		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
-
-		proc->lwWaiting = true;
-		proc->lwWaitMode = mode;
-		dlist_push_head((dlist_head *) &lock->waiters, &proc->lwWaitLink);
-
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
 
 		/*
 		 * Wait until awakened.
@@ -460,7 +793,7 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockAcquire", lockid, "waiting");
+		LOG_LWDEBUG("LWLockAcquire", lockid, mode, "waiting");
 
 #ifdef LWLOCK_STATS
 		block_counts[lockid]++;
@@ -476,22 +809,22 @@ LWLockAcquire(LWLockId lockid, LWLockMode mode)
 				break;
 			extraWaits++;
 		}
+		lock->releaseOK = true;
 
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(lockid, mode);
 
-		LOG_LWDEBUG("LWLockAcquire", lockid, "awakened");
+		LOG_LWDEBUG("LWLockAcquire", lockid, mode, "awakened");
 
 		/* Now loop back and try to acquire lock again. */
-		retry = true;
+		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+		iterations++;
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(lockid, mode);
 
 	/* Add lock to list of locks held by this backend */
-	held_lwlocks[num_held_lwlocks++] = lockid;
+	held_lwlocks[num_held_lwlocks].lock = lockid;
+	held_lwlocks[num_held_lwlocks++].mode = mode;
 
 	/*
 	 * Fix the process wait semaphore's count for any absorbed wakeups.
@@ -512,8 +845,11 @@ LWLockConditionalAcquire(LWLockId lockid, LWLockMode mode)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
 	bool		mustwait;
+	bool		potentially_spurious;
+
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
 
-	PRINT_LWDEBUG("LWLockConditionalAcquire", lockid, lock);
+	PRINT_LWDEBUG("LWLockConditionalAcquire", lockid, lock, mode);
 
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
@@ -526,48 +862,42 @@ LWLockConditionalAcquire(LWLockId lockid, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
+retry:
+	/*
+	 * passing 'true' to check more carefully to avoid potential
+	 * spurious acquisitions
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, true, &potentially_spurious);
 
 	if (mustwait)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockConditionalAcquire", lockid, "failed");
+		LOG_LWDEBUG("LWLockConditionalAcquire", lockid, mode, "failed");
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(lockid, mode);
+
+		/*
+		 * We ran into an exclusive lock and might have blocked another
+		 * exclusive lock from taking a shot because it took a time to back
+		 * off. Retry till we are either sure we didn't block somebody (because
+		 * somebody else certainly has the lock) or till we got it.
+		 *
+		 * We cannot rely on the two-step lock-acquisition protocol as in
+		 * LWLockAcquire because we're not using it.
+		 */
+		if (potentially_spurious)
+		{
+			SPIN_DELAY();
+			goto retry;
+		}
 	}
 	else
 	{
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lockid;
+		held_lwlocks[num_held_lwlocks].lock = lockid;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(lockid, mode);
 	}
-
 	return !mustwait;
 }
 
@@ -592,8 +922,11 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 	PGPROC	   *proc = MyProc;
 	bool		mustwait;
 	int			extraWaits = 0;
+	bool		potentially_spurious_first;
+	bool		potentially_spurious_second;
 
-	PRINT_LWDEBUG("LWLockAcquireOrWait", lockid, lock);
+	Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+	PRINT_LWDEBUG("LWLockAcquireOrWait", lockid, lock, mode);
 
 #ifdef LWLOCK_STATS
 	/* Set up local count state first time through in a given process */
@@ -612,79 +945,64 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
+	/*
+	 * NB: We're using nearly the same twice-in-a-row lock acquisition
+	 * protocol as LWLockAcquire(). Check its comments for details.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious_first);
 
 	if (mustwait)
 	{
-		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait.  This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
-		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		dlist_push_head((dlist_head *) &lock->waiters, &proc->lwWaitLink);
+		mustwait = LWLockAttemptLock(lock, mode, false, &potentially_spurious_second);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
-
-		/*
-		 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
-		 * wakups, because we share the semaphore with ProcWaitForSignal.
-		 */
-		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, "waiting");
+		if (mustwait)
+		{
+			/*
+			 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
+			 * wakups, because we share the semaphore with ProcWaitForSignal.
+			 */
+			LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "waiting");
 
 #ifdef LWLOCK_STATS
-		block_counts[lockid]++;
+			block_counts[lockid]++;
 #endif
+			TRACE_POSTGRESQL_LWLOCK_WAIT_START(lockid, mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_START(lockid, mode);
+			for (;;)
+			{
+				/* "false" means cannot accept cancel/die interrupt here. */
+				PGSemaphoreLock(&proc->sem, false);
+				if (!proc->lwWaiting)
+					break;
+				extraWaits++;
+			}
+			pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
 
-		for (;;)
-		{
-			/* "false" means cannot accept cancel/die interrupt here. */
-			PGSemaphoreLock(&proc->sem, false);
-			if (!proc->lwWaiting)
-				break;
-			extraWaits++;
-		}
+			TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(lockid, mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(lockid, mode);
+			LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "awakened");
+		}
+		else
+		{
+			/* got lock in the second attempt, undo queueing */
+			if (!LWLockDequeueSelf(lock))
+			{
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+			}
 
-		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, "awakened");
-	}
-	else
-	{
-		/* We are done updating shared state of the lock itself. */
-		SpinLockRelease(&lock->mutex);
+			/* FIXME: don't need that anymore? */
+#if 0
+			LWLockWakeup(lock, lockid, mode);
+#endif
+		}
 	}
 
 	/*
@@ -697,13 +1015,15 @@ LWLockAcquireOrWait(LWLockId lockid, LWLockMode mode)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, "failed");
+		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "failed");
 		TRACE_POSTGRESQL_LWLOCK_WAIT_UNTIL_FREE_FAIL(lockid, mode);
 	}
 	else
 	{
+		LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "suceeded");
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lockid;
+		held_lwlocks[num_held_lwlocks].lock = lockid;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_WAIT_UNTIL_FREE(lockid, mode);
 	}
 
@@ -717,13 +1037,11 @@ void
 LWLockRelease(LWLockId lockid)
 {
 	volatile LWLock *lock = &(LWLockArray[lockid].lock);
-	dlist_head	wakeup;
-	dlist_mutable_iter iter;
 	int			i;
-
-	dlist_init(&wakeup);
-
-	PRINT_LWDEBUG("LWLockRelease", lockid, lock);
+	LWLockMode	mode;
+	uint32		lockcount;
+	bool		check_waiters;
+	bool		have_waiters = false;
 
 	/*
 	 * Remove lock from list of locks held.  Usually, but not always, it will
@@ -731,8 +1049,11 @@ LWLockRelease(LWLockId lockid)
 	 */
 	for (i = num_held_lwlocks; --i >= 0;)
 	{
-		if (lockid == held_lwlocks[i])
+		if (lockid == held_lwlocks[i].lock)
+		{
+			mode = held_lwlocks[i].mode;
 			break;
+		}
 	}
 	if (i < 0)
 		elog(ERROR, "lock %d is not held", (int) lockid);
@@ -740,78 +1061,50 @@ LWLockRelease(LWLockId lockid)
 	for (; i < num_held_lwlocks; i++)
 		held_lwlocks[i] = held_lwlocks[i + 1];
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
+	PRINT_LWDEBUG("LWLockRelease", lockid, lock, mode);
 
-	/* Release my hold on lock */
-	if (lock->exclusive > 0)
-		lock->exclusive--;
+	pg_read_barrier();
+
+	/* Release my hold on lock, both are a full barrier */
+	if (mode == LW_EXCLUSIVE)
+		lockcount = pg_atomic_sub_fetch_u32(&lock->lockcount, EXCLUSIVE_LOCK);
 	else
-	{
-		Assert(lock->shared > 0);
-		lock->shared--;
-	}
+		lockcount = pg_atomic_sub_fetch_u32(&lock->lockcount, 1);
+
+	/* nobody else can have that kind of lock */
+	Assert(lockcount < EXCLUSIVE_LOCK);
 
 	/*
-	 * See if I need to awaken any waiters.  If I released a non-last shared
-	 * hold, there cannot be anything to do.  Also, do not awaken any waiters
-	 * if someone has already awakened waiters that haven't yet acquired the
-	 * lock.
+	 * Anybody we need to wakeup needs to have started queueing before
+	 * we removed ourselves from the queue and the __sync_ operations
+	 * above are full barriers.
 	 */
-	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
-	{
-		/*
-		 * Remove the to-be-awakened PGPROCs from the queue.
-		 */
-		bool		releaseOK = true;
-		bool		wokeup_somebody = false;
-
-		dlist_foreach_modify(iter, (dlist_head *) &lock->waiters)
-		{
-			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-
-			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
-				continue;
-
-			dlist_delete(&waiter->lwWaitLink);
-			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 
-			/*
-			 * Prevent additional wakeups until retryer gets to
-			 * run. Backends that are just waiting for the lock to become
-			 * free don't retry automatically.
-			 */
-			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
-			{
-				releaseOK = false;
-				wokeup_somebody = true;
-			}
+	if (pg_atomic_read_u32(&lock->nwaiters) > 0)
+		have_waiters = true;
+
+	/* we're still waiting for backends to get scheduled, don't release again */
+	if (!lock->releaseOK)
+		check_waiters = false;
+	/* grant permission to run, even if a spurious share lock increases lockcount */
+	else if (mode == LW_EXCLUSIVE && have_waiters)
+		check_waiters = true;
+	/* nobody has this locked anymore, potential exclusive lockers get a chance */
+	else if (lockcount == 0 && have_waiters)
+		check_waiters = true;
+	/* nobody queued or not free */
+	else
+		check_waiters = false;
 
-			if(waiter->lwWaitMode == LW_EXCLUSIVE)
-				break;
-		}
-		lock->releaseOK = releaseOK;
+	if (check_waiters)
+	{
+		PRINT_LWDEBUG("LWLockRelease releasing", lockid, lock, mode);
+		LWLockWakeup(lock, lockid, mode);
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_RELEASE(lockid);
 
 	/*
-	 * Awaken any waiters I removed from the queue.
-	 */
-	dlist_foreach_modify(iter, (dlist_head *) &wakeup)
-	{
-		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-		LOG_LWDEBUG("LWLockRelease", lockid, "release waiter");
-		dlist_delete(&waiter->lwWaitLink);
-		pg_write_barrier();
-		waiter->lwWaiting = false;
-		PGSemaphoreUnlock(&waiter->sem);
-	}
-
-	/*
 	 * Now okay to allow cancel/die interrupts.
 	 */
 	RESUME_INTERRUPTS();
@@ -834,7 +1127,7 @@ LWLockReleaseAll(void)
 	{
 		HOLD_INTERRUPTS();		/* match the upcoming RESUME_INTERRUPTS */
 
-		LWLockRelease(held_lwlocks[num_held_lwlocks - 1]);
+		LWLockRelease(held_lwlocks[num_held_lwlocks - 1].lock);
 	}
 }
 
@@ -842,8 +1135,7 @@ LWLockReleaseAll(void)
 /*
  * LWLockHeldByMe - test whether my process currently holds a lock
  *
- * This is meant as debug support only.  We do not distinguish whether the
- * lock is held shared or exclusive.
+ * This is meant as debug support only.
  */
 bool
 LWLockHeldByMe(LWLockId lockid)
@@ -852,7 +1144,7 @@ LWLockHeldByMe(LWLockId lockid)
 
 	for (i = 0; i < num_held_lwlocks; i++)
 	{
-		if (held_lwlocks[i] == lockid)
+		if (held_lwlocks[i].lock == lockid)
 			return true;
 	}
 	return false;
-- 
1.8.5.rc1.dirty

#18Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#17)
Re: Wait free LW_SHARED acquisition - v0.2

On 2013-11-15 20:47:26 +0100, Andres Freund wrote:

Hi,

On 2013-09-27 00:55:45 +0200, Andres Freund wrote:

So what's todo? The file header tells us:
* - revive pure-spinlock implementation
* - abstract away atomic ops, we really only need a few.
* - CAS
* - LOCK XADD
* - convert PGPROC->lwWaitLink to ilist.h slist or even dlist.
* - remove LWLockWakeup dealing with MyProc
* - overhaul the mask offsets, make SHARED/EXCLUSIVE_LOCK_MASK wider, MAX_BACKENDS

So, here's the next version of this patchset:
1) I've added an abstracted atomic ops implementation. Needs a fair
amount of work, also submitted as a separate CF entry. (Patch 1 & 2)
2) I've converted PGPROC->lwWaiting into a dlist. That makes a fair bit
of code easier to read and reduces the size of the patchset. Also
fixes a bug in the xlog-scalability code. (Patch 3)
3) Alvaro and I updated the comments in lwlock.c. (Patch 4)

I think 2) should be committable pretty soon. It's imo a pretty clear
win in readability. 1) will need a good bit of more work.

With regard to the scalable lwlock work, what's most needed now is a
good amount of testing.

Please note that you need to 'autoreconf' after applying the patchset. I
don't have a compatible autoconf version on this computer causing the
diff to be humongous if I include those changes.

Please also note that due to the current state of the atomics
implementation this likely will only work on a somewhat recent gcc and
that the performance might be slightly worse than in the previous
version because the atomic add is implemented using the CAS fallback.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#19Peter Eisentraut
peter_e@gmx.net
In reply to: Andres Freund (#17)
Re: Wait free LW_SHARED acquisition - v0.2

On Fri, 2013-11-15 at 20:47 +0100, Andres Freund wrote:

So, here's the next version of this patchset:

The 0002 patch contains non-ASCII, non-UTF8 characters:

0002-Very-basic-atomic-ops-implementation.patch: line 609, char 1, byte offset 43: invalid UTF-8 code

Please change that to ASCII, or UTF-8 if necessary.

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#20Peter Eisentraut
peter_e@gmx.net
In reply to: Andres Freund (#17)
Re: Wait free LW_SHARED acquisition - v0.2

This patch didn't make it out of the 2013-11 commit fest. You should
move it to the next commit fest (probably with an updated patch)
before January 15th.

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#21Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#17)
Re: Wait free LW_SHARED acquisition - v0.2

On Fri, Nov 15, 2013 at 11:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

1) I've added an abstracted atomic ops implementation. Needs a fair
amount of work, also submitted as a separate CF entry. (Patch 1 & 2)

Commit 220b34331f77effdb46798ddd7cca0cffc1b2858 caused bitrot when
applying 0002-Very-basic-atomic-ops-implementation.patch. Please
rebase.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#22Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#21)
Re: Wait free LW_SHARED acquisition - v0.2

Hi,

On 2014-01-28 21:27:29 -0800, Peter Geoghegan wrote:

On Fri, Nov 15, 2013 at 11:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

1) I've added an abstracted atomic ops implementation. Needs a fair
amount of work, also submitted as a separate CF entry. (Patch 1 & 2)

Commit 220b34331f77effdb46798ddd7cca0cffc1b2858 caused bitrot when
applying 0002-Very-basic-atomic-ops-implementation.patch. Please
rebase.

I've pushed a rebased version of the patchset to
http://git.postgresql.org/gitweb/?p=users/andresfreund/postgres.git
branch rwlock contention.
220b34331f77effdb46798ddd7cca0cffc1b2858 actually was the small problem,
ea9df812d8502fff74e7bc37d61bdc7d66d77a7f was the major PITA.

I plan to split the atomics patch into smaller chunks before
reposting. Imo the "Convert the PGPROC->lwWaitLink list into a dlist
instead of open coding it." is worth being applied independently from
the rest of the series, it simplies code and it fixes a bug...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#23Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#22)
Re: Wait free LW_SHARED acquisition - v0.2

On Fri, Jan 31, 2014 at 1:54 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I plan to split the atomics patch into smaller chunks before
reposting. Imo the "Convert the PGPROC->lwWaitLink list into a dlist
instead of open coding it." is worth being applied independently from
the rest of the series, it simplies code and it fixes a bug...

For things that require a format-patch series, I personally find it
easier to work off a feature branch on a remote under the control of
the patch author. The only reason that I don't do so myself is that I
know that that isn't everyone's preference.

I have access to a large server for the purposes of benchmarking this.
On the plus side, this is a box very much capable of exercising these
bottlenecks: a 48 core AMD system, with 256GB of RAM. However, I had
to instruct someone else on how to conduct the benchmark, since I
didn't have SSH access, and the OS and toolchain were antiquated,
particularly for this kind of thing. This is Linux kernel
2.6.18-274.3.1.el5 (RHEL5.7). The GCC version that Postgres was built
with was 4.1.2. This is not what I'd hoped for; obviously I would have
preferred to be able to act on your warning: "Please also note that
due to the current state of the atomics implementation this likely
will only work on a somewhat recent gcc and that the performance might
be slightly worse than in the previous version because the atomic add
is implemented using the CAS fallback". Even still, it might be
marginally useful to get a sense of that cost.

I'm looking at alternative options, because this is not terribly
helpful. With those big caveats in mind, consider the results of the
benchmark, which show the patch performing somewhat worse than the
master baseline at higher client counts:

http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/rwlock-contention/

This is exactly what you said would happen, but at least now we have a
sense of that cost.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#24Peter Geoghegan
pg@heroku.com
In reply to: Peter Geoghegan (#23)
Re: Wait free LW_SHARED acquisition - v0.2

I thought I'd try out what I was in an immediate position to do
without having access to dedicated multi-socket hardware: A benchmark
on AWS. This was a "c3.8xlarge" instance, which are reportedly backed
by Intel Xeon E5-2680 processors. Since the Intel ARK website reports
that these CPUs have 16 "threads" (8 cores + hyperthreading), and
AWS's marketing material indicates that this instance type has 32
"vCPUs", I inferred that the underlying hardware had 2 sockets.
However, reportedly that wasn't the case when procfs was consulted, no
doubt due to Xen Hypervisor voodoo:

ubuntu@ip-10-67-128-2:~$ lscpu
Architecture: x86_64
CPU op-mode(s): 32-bit, 64-bit
Byte Order: Little Endian
CPU(s): 32
On-line CPU(s) list: 0-31
Thread(s) per core: 32
Core(s) per socket: 1
Socket(s): 1
NUMA node(s): 1
Vendor ID: GenuineIntel
CPU family: 6
Model: 62
Stepping: 4
CPU MHz: 2800.074
BogoMIPS: 5600.14
Hypervisor vendor: Xen
Virtualization type: full
L1d cache: 32K
L1i cache: 32K
L2 cache: 256K
L3 cache: 25600K
NUMA node0 CPU(s): 0-31

I ran the benchmark on Ubuntu 13.10 server, because that seemed to be
the only prominent "enterprise" x86_64 AMI (operating system image)
that came with GCC 4.8 as part its standard toolchain. This exact
setup is benchmarked here:

http://www.phoronix.com/scan.php?page=article&amp;item=amazon_ec2_c3&amp;num=1

(Incidentally, some of the other benchmarks on that site use pgbench
to benchmark the Linux kernel, filesystems, disks and so on. e.g.:
http://www.phoronix.com/scan.php?page=news_item&amp;px=NzI0NQ).

I was hesitant to benchmark using a virtualized system. There is a lot
of contradictory information about the overhead and/or noise added,
which may vary from one workload or hypervisor to the next. But, needs
must when the devil drives, and all that. Besides, this kind of setup
is very commercially relevant these days, so it doesn't seem
unreasonable to see how things work out on an AWS instance that
generally performs well for this workload. Of course, I still want to
replicate the big improvement you reported for multi-socket systems,
but you might have to wait a while for that, unless some kindly
benefactor that has a 4 socket server lying around would like to help
me out.

Results:

http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/c38xlarge-rwlocks/

You can drill down and find the postgresql.conf settings from the
report. There appears to be a modest improvement in transaction
throughput. It's not as large as the improvement you reported for your
2 socket workstation, but it's there, just about.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#25Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#23)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-01-31 17:52:58 -0800, Peter Geoghegan wrote:

On Fri, Jan 31, 2014 at 1:54 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I plan to split the atomics patch into smaller chunks before
reposting. Imo the "Convert the PGPROC->lwWaitLink list into a dlist
instead of open coding it." is worth being applied independently from
the rest of the series, it simplies code and it fixes a bug...

For things that require a format-patch series, I personally find it
easier to work off a feature branch on a remote under the control of
the patch author. The only reason that I don't do so myself is that I
know that that isn't everyone's preference.

I do to, that's why I have a git branch for all but the most trivial
branches.

I have access to a large server for the purposes of benchmarking this.
On the plus side, this is a box very much capable of exercising these
bottlenecks: a 48 core AMD system, with 256GB of RAM. However, I had
to instruct someone else on how to conduct the benchmark, since I
didn't have SSH access, and the OS and toolchain were antiquated,
particularly for this kind of thing. This is Linux kernel
2.6.18-274.3.1.el5 (RHEL5.7). The GCC version that Postgres was built
with was 4.1.2. This is not what I'd hoped for; obviously I would have
preferred to be able to act on your warning: "Please also note that
due to the current state of the atomics implementation this likely
will only work on a somewhat recent gcc and that the performance might
be slightly worse than in the previous version because the atomic add
is implemented using the CAS fallback". Even still, it might be
marginally useful to get a sense of that cost.

I *think* it should actually work on gcc 4.1, I've since added the
fallbacks I hadn't back when I wrote the above. I've added exactly those
atomics that are needed for the scalable lwlock things (xadd, xsub (yes,
that's essentially the same) and cmpxchg).

I'm looking at alternative options, because this is not terribly
helpful. With those big caveats in mind, consider the results of the
benchmark, which show the patch performing somewhat worse than the
master baseline at higher client counts:

I think that's actually something else. I'd tried to make some
definitions simpler, and that has, at least for the machine I have
occasional access to, pessimized things. I can't always run the tests
there, so I hadn't noticed before the repost.
I've pushed a preliminary relief to the git repository, any chance you
could retry?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#26Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#25)
Re: Wait free LW_SHARED acquisition - v0.2

On Sat, Feb 1, 2014 at 4:57 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I'm looking at alternative options, because this is not terribly
helpful. With those big caveats in mind, consider the results of the
benchmark, which show the patch performing somewhat worse than the
master baseline at higher client counts:

I think that's actually something else. I'd tried to make some
definitions simpler, and that has, at least for the machine I have
occasional access to, pessimized things. I can't always run the tests
there, so I hadn't noticed before the repost.

I should have been clearer on one point: The pre-rebased patch (actual
patch series) [1]/messages/by-id/20131115194725.GG5489@awork2.anarazel.de -- Peter Geoghegan was applied on top of a commit from around the same
period, in order to work around the bit rot. However, I tested the
most recent revision from your git remote on the AWS instance.

[1]: /messages/by-id/20131115194725.GG5489@awork2.anarazel.de -- Peter Geoghegan
--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#27Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#26)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-02-01 13:40:20 -0800, Peter Geoghegan wrote:

On Sat, Feb 1, 2014 at 4:57 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I'm looking at alternative options, because this is not terribly
helpful. With those big caveats in mind, consider the results of the
benchmark, which show the patch performing somewhat worse than the
master baseline at higher client counts:

I think that's actually something else. I'd tried to make some
definitions simpler, and that has, at least for the machine I have
occasional access to, pessimized things. I can't always run the tests
there, so I hadn't noticed before the repost.

I should have been clearer on one point: The pre-rebased patch (actual
patch series) [1] was applied on top of a commit from around the same
period, in order to work around the bit rot.

Ah. Then I indeed wouldn't expect improvements.

However, I tested the
most recent revision from your git remote on the AWS instance.

[1] /messages/by-id/20131115194725.GG5489@awork2.anarazel.de

But that was before my fix, right. Except you managed to timetravel :)

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#28Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#27)
Re: Wait free LW_SHARED acquisition - v0.2

On Sat, Feb 1, 2014 at 1:41 PM, Andres Freund <andres@2ndquadrant.com> wrote:

However, I tested the
most recent revision from your git remote on the AWS instance.

But that was before my fix, right. Except you managed to timetravel :)

Heh, okay. So Nathan Boley has generously made available a machine
with 4 AMD Opteron 6272s. I've performed the same benchmark on that
server. However, I thought it might be interesting to circle back and
get some additional numbers for the AWS instance already tested - I'd
like to see what it looks like after your recent tweaks to fix the
regression. The single client performance of that instance seems to be
markedly better than that of Nathan's server.

Tip: AWS command line tools + S3 are a great way to easily publish
bulky pgbench-tools results, once you figure out how to correctly set
your S3 bucket's security manifest to allow public http. It has
similar advantages to rsync, and just works with the minimum of fuss.

Anyway, I don't think that the new, third c3.8xlarge-rwlocks testset
tells us much of anything:
http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/c38xlarge-rwlocks/

Here are the results of a benchmark on Nathan Boley's 64-core, 4
socket server: http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/amd-4-socket-rwlocks/

Perhaps I should have gone past 64 clients, because in the document
"Lock Scaling Analysis on Intel Xeon Processors" [1]http://www.intel.com/content/dam/www/public/us/en/documents/white-papers/xeon-lock-scaling-analysis-paper.pdf -- Peter Geoghegan, Intel write:

"This implies that with oversubscription (more threads running than
available logical CPUs), the performance of spinlocks can depend
heavily on the exact OS scheduler behavior, and may change drastically
with operating system or VM updates."

I haven't bothered with a higher client counts though, because Andres
noted it's the same with 90 clients on this AMD system. Andres: Do you
see big problems when # clients < # logical cores on the affected
Intel systems?

There is only a marginal improvement in performance on this big 4
socket system. Andres informs me privately that he has reproduced the
problem on multiple new 4-socket Intel servers, so it seems reasonable
to suppose that more or less an Intel thing.

The Intel document [1]http://www.intel.com/content/dam/www/public/us/en/documents/white-papers/xeon-lock-scaling-analysis-paper.pdf -- Peter Geoghegan further notes:

"As the number of threads polling the status of a lock address
increases, the time it takes to process those polling requests will
increase. Initially, the latency to transfer data across socket
boundaries will always be an order of magnitude longer than the
on-chip cache-to-cache transfer latencies. Such cross-socket
transfers, if they are not effectively minimized by software, will
negatively impact the performance of any lock algorithm that depends
on them."

So, I think it's fair to say, given what we now know from Andres'
numbers and the numbers I got from Nathan's server, that this patch is
closer to being something that addresses a particularly unfortunate
pathology on many-socket Intel system than it is to being a general
performance optimization. Based on the above quoted passage, it isn't
unreasonable to suppose other vendors or architectures could be
affected, but that isn't in evidence. While I welcome the use of
atomic operations in the context of LW_SHARED acquisition as general
progress, I think that to the outside world my proposed messaging is
more useful. It's not quite a bug-fix, but if you're using a
many-socket Intel server, you're *definitely* going to want to use a
PostgreSQL version that is unaffected. You may well not want to take
on the burden of waiting for 9.4, or waiting for it to fully
stabilize.

I note that Andres has a feature branch of this backported to Postgres
9.2, no doubt because of a request from a 2ndQuadrant customer. I have
to wonder if we should think about making this available with a
configure switch in one or more back branches. I think that the
complete back-porting of the fsync request queue issue's fix in commit
758728 could be considered a precedent - that too was a fix for a
really bad worst-case that was encountered fairly infrequently in the
wild. It's sort of horrifying to have red-hot spinlocks in production,
so that seems like the kind of thing we should make an effort to
address for those running multi-socket systems. Of those running
Postgres on new multi-socket systems, the reality is that the majority
are running on Intel hardware. Unfortunately, everyone knows that
Intel will soon be the only game in town when it comes to high-end
x86_64 servers, which contributes to my feeling that we need to target
back branches. We should do something about the possible regression
with older compilers using the fallback first, though.

It would be useful to know more about the nature of the problem that
made such an appreciable difference in Andres' original post. Again,
through private e-mail, I saw perf profiles from affected servers and
an unaffected though roughly comparable server (i.e. Nathan's 64 core
AMD server). Andres observed that "stalled-cycles-frontend" and
"stalled-cycles-backend" Linux perf events were at huge variance
depending on whether these Intel systems were patched or unpatched.
They were about the same on the AMD system to begin with.

[1]: http://www.intel.com/content/dam/www/public/us/en/documents/white-papers/xeon-lock-scaling-analysis-paper.pdf -- Peter Geoghegan
--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#29Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#28)
Re: Wait free LW_SHARED acquisition - v0.2

Hi,

On 2014-02-01 19:47:29 -0800, Peter Geoghegan wrote:

Here are the results of a benchmark on Nathan Boley's 64-core, 4
socket server: http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/amd-4-socket-rwlocks/

That's interesting. The maximum number of what you see here (~293125)
is markedly lower than what I can get.

... poke around ...

Hm, that's partially because you're using pgbench without -M prepared if
I see that correctly. The bottleneck in that case is primarily memory
allocation. But even after that I am getting higher
numbers: ~342497.

Trying to nail down the differnce it oddly seems to be your
max_connections=80 vs my 100. The profile in both cases is markedly
different, way much more spinlock contention with 80. All in
Pin/UnpinBuffer().

I think =80 has to lead to some data being badly aligned. I can
reproduce that =91 has *much* better performance than =90. 170841.844938
vs 368490.268577 in a 10s test. Reproducable both with an without the test.
That's certainly worth some investigation.
This is *not* reproducable on the intel machine, so it might the
associativity of the L1/L2 cache on the AMD.

Perhaps I should have gone past 64 clients, because in the document
"Lock Scaling Analysis on Intel Xeon Processors" [1], Intel write:

"This implies that with oversubscription (more threads running than
available logical CPUs), the performance of spinlocks can depend
heavily on the exact OS scheduler behavior, and may change drastically
with operating system or VM updates."

I haven't bothered with a higher client counts though, because Andres
noted it's the same with 90 clients on this AMD system. Andres: Do you
see big problems when # clients < # logical cores on the affected
Intel systems?

There's some slowdown with the patch applied, but it's not big. Without
it, the slowdown is much earlier.

There is only a marginal improvement in performance on this big 4
socket system. Andres informs me privately that he has reproduced the
problem on multiple new 4-socket Intel servers, so it seems reasonable
to suppose that more or less an Intel thing.

I've just poked around, it's not just 4 socket, but also 2 socket
systems.

Some background:
The setups that triggered me into working on the patchset didn't really
have a pgbench like workload, the individual queries were/are more
complicated even though it's still an high throughput OLTP workload. And
the contention was *much* higher than what I can reproduce with pgbench
-S, there was often nearly all time spent in the lwlock's spinlock, and
it was primarily the buffer mapping lwlocks, being locked in shared
mode. The difference is that instead of locking very few buffers per
query like pgbench does, they touched much more.
If you look at a profile of a pgbench -S workload -cj64 it's pretty much all
bottlenecked by GetSnapshotData():
unpatched:
-  40.91%  postgres_plainl  postgres_plainlw    [.] s_lock
   - s_lock
      - 51.34% LWLockAcquire
           GetSnapshotData
         - GetTransactionSnapshot
            + 55.23% PortalStart
            + 44.77% PostgresMain
      - 48.66% LWLockRelease
           GetSnapshotData
         - GetTransactionSnapshot
            + 53.64% PostgresMain
            + 46.36% PortalStart
+   2.65%          pgbench  [kernel.kallsyms]   [k] try_to_wake_up
+   2.61%  postgres_plainl  postgres_plainlw    [.] LWLockRelease
+   1.36%  postgres_plainl  postgres_plainlw    [.] LWLockAcquire
+   1.25%  postgres_plainl  postgres_plainlw    [.] GetSnapshotData
patched:
-   2.94%       postgres  postgres              [.] LWLockAcquire
   - LWLockAcquire
      + 26.61% ReadBuffer_common
      + 17.08% GetSnapshotData
      + 10.71% _bt_relandgetbuf
      + 10.24% LockAcquireExtended
      + 10.11% VirtualXactLockTableInsert
      + 9.17% VirtualXactLockTableCleanup
      + 7.55% _bt_getbuf
      + 3.40% index_fetch_heap
      + 1.51% LockReleaseAll
      + 0.89% StartTransactionCommand
      + 0.83% _bt_next
      + 0.75% LockRelationOid
      + 0.52% ReadBufferExtended
-   2.75%       postgres  postgres              [.] _bt_compare
   - _bt_compare
      + 96.72% _bt_binsrch
      + 1.71% _bt_moveright
      + 1.29% _bt_search
-   2.67%       postgres  postgres              [.] GetSnapshotData
   - GetSnapshotData
      + 97.03% GetTransactionSnapshot
      + 2.16% PostgresMain
      + 0.81% PortalStart

So now the profile looks much saner/less contended which immediately is
visible in transaction rates: 192584.218530 vs 552834.002573.

But if you try to attack the contention from the other end, by setting
default_transaction_isolation='repeatable read' to reduce the number of
snapshots taken, its suddenly 536789.807391 vs 566839.328922. A *much*
smaller benefit.
The reason the patch doesn't help that much with that setting is that there
simply isn't as much actual contention there:

+   2.77%       postgres  postgres            [.] _bt_compare
-   2.72%       postgres  postgres            [.] LWLockAcquire
   - LWLockAcquire
      + 30.51% ReadBuffer_common
      + 12.45% VirtualXactLockTableInsert
      + 12.07% _bt_relandgetbuf
      + 10.61% VirtualXactLockTableCleanup
      + 9.95% LockAcquireExtended
      + 8.59% GetSnapshotData
      + 7.51% _bt_getbuf
      + 2.54% index_fetch_heap
      + 1.47% LockReleaseAll
      + 1.16% StartTransactionCommand
      + 0.95% LockRelationOid
      + 0.89% _bt_next
      + 0.77% ReadBufferExtended
-   2.19%       postgres  postgres            [.] s_lock
   - s_lock
      + 52.41% PinBuffer
      + 47.04% UnpinBuffer

While locking is still visible, it's not so extraordinarily dominant. If
you then profile using -e stalled-cycles-backend, you can see that the
actual problematic pipeline stalls are caused by Pin/UnpinBuffer. So
that's now the bottleneck. Since that doesn't use lwlocks, it's not
surprising to see an lwlocks patch doesn't bring as much benefit.

So, I think it's fair to say, given what we now know from Andres'
numbers and the numbers I got from Nathan's server, that this patch is
closer to being something that addresses a particularly unfortunate
pathology on many-socket Intel system than it is to being a general
performance optimization.

I think one would see bigger improvements if we'd ha testcase that also
caused real lock contention in the AMDs. If you look at a profile there:

unpatched, -c default_transaction_isolation=read committed
-   3.64%  postgres_plainl  postgres_plainlw    [.] s_lock
   - s_lock
      + 51.49% LWLockAcquire
      + 45.19% LWLockRelease
      + 1.73% PinBuffer
      + 1.59% UnpinBuffer
-   3.46%  postgres_plainl  postgres_plainlw    [.] LWLockAcquire
   - LWLockAcquire
      + 36.67% GetSnapshotData
      + 24.61% ReadBuffer_common
      + 11.45% _bt_relandgetbuf
      + 6.78% _bt_getbuf
      + 5.99% LockAcquireExtended
      + 5.27% VirtualXactLockTableInsert
      + 5.00% VirtualXactLockTableCleanup
      + 1.52% _bt_next
      + 1.31% index_fetch_heap
      + 0.76% LockReleaseAll
-   2.90%  postgres_plainl  postgres_plainlw    [.] GetSnapshotData
   - GetSnapshotData
      + 97.40% GetTransactionSnapshot
      + 2.31% PostgresMain

unpatched, -c default_transaction_isolation=repeatable read

-   2.78%  postgres_plainl  postgres_plainlw    [.] LWLockAcquire
   - LWLockAcquire
      + 35.41% ReadBuffer_common
      + 15.12% _bt_relandgetbuf
      + 11.78% GetSnapshotData
      + 9.29% _bt_getbuf
      + 7.36% VirtualXactLockTableInsert
      + 7.25% LockAcquireExtended
      + 6.67% VirtualXactLockTableCleanup
      + 1.98% index_fetch_heap
      + 1.88% _bt_next
      + 1.47% LockReleaseAll
      + 0.50% StartTransactionCommand
-   2.21%  postgres_plainl  postgres_plainlw    [.] hash_search_with_hash_value
   - hash_search_with_hash_value
      + 53.23% BufTableLookup
      + 13.92% LockAcquireExtended
      + 8.99% GetPortalByName
      + 6.79% RelationIdGetRelation
      + 4.98% FetchPreparedStatement
      + 4.43% CreatePortal
      + 3.61% PortalDrop
      + 2.46% RemoveLocalLock

There's obviously not even close as much contention as in the intel
case. Which shows in the benchmark results:
440910.048200 vs. 473789.980486

So, the lwlock patch cannot improve concurrency by the degree it did for
intel, since there simply isn't much of a bottleneck. But I'd be
surprised if there aren't cases with much more prominent bottlenecks,
this just isn't one of them.

The changed algorithm for lwlock imo is an *algorithmic* improvement,
not one for a particular architecture. The advantage being that locking
a lwlock which is primarily taken in shared mode will never need need to
wait or loop.

Based on the above quoted passage, it isn't
unreasonable to suppose other vendors or architectures could be
affected, but that isn't in evidence. While I welcome the use of
atomic operations in the context of LW_SHARED acquisition as general
progress, I think that to the outside world my proposed messaging is
more useful. It's not quite a bug-fix, but if you're using a
many-socket Intel server, you're *definitely* going to want to use a
PostgreSQL version that is unaffected. You may well not want to take
on the burden of waiting for 9.4, or waiting for it to fully
stabilize.

I note that Andres has a feature branch of this backported to Postgres
9.2, no doubt because of a request from a 2ndQuadrant customer. I have
to wonder if we should think about making this available with a
configure switch in one or more back branches.

Yes, that branch is used by some of them. But to make that clear to all
that are still reading, I have *first* presented the patch & findings to
-hackers and *then* backported it, and I have referenced the existance
of the patch for 9.2 on list before. This isn't some kind of "secret
sauce" deal...

But note that there is one significant difference between the 9.2 and
HEAD version, the former directly uses gcc intrinsics, without having
any form of checks or abstraction across compilers...

I think that the
complete back-porting of the fsync request queue issue's fix in commit
758728 could be considered a precedent - that too was a fix for a
really bad worst-case that was encountered fairly infrequently in the
wild. It's sort of horrifying to have red-hot spinlocks in production,
so that seems like the kind of thing we should make an effort to
address for those running multi-socket systems. Of those running
Postgres on new multi-socket systems, the reality is that the majority
are running on Intel hardware. Unfortunately, everyone knows that
Intel will soon be the only game in town when it comes to high-end
x86_64 servers, which contributes to my feeling that we need to target
back branches. We should do something about the possible regression
with older compilers using the fallback first, though.

That might be something to do later, as it *really* can hurt in
practice. We had one server go from load 240 to 11...

But I think we should first focus on getting the patch ready for
master, then we can see where it's going. At the very least I'd like to
split of the part modifying the current spinlocks to use the atomics,
that seems far to invasive.

It would be useful to know more about the nature of the problem that
made such an appreciable difference in Andres' original post.

I unfortunately can't tell you that much more, not because it's private,
but because it mostly was diagnosed by remote hand debugging, limiting
insights considerably.
What I *do* know is that the bottleneck entirely was caused by the
buffer mapping lwlocks, all taken in shared mode according to call graph
profiles. The call graph and the queries I have seen indicated that lots
of the frequent queries involved nested loops over not inconsiderable
number of pages.
We've also tried to increase the number of buffer mapping locks, but
that didn't prove to be very helpful.

Hm, that went on a bit...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#30Jeff Janes
jeff.janes@gmail.com
In reply to: Andres Freund (#29)
Re: Wait free LW_SHARED acquisition - v0.2

On Sun, Feb 2, 2014 at 6:00 AM, Andres Freund <andres@2ndquadrant.com>wrote:

Some background:
The setups that triggered me into working on the patchset didn't really
have a pgbench like workload, the individual queries were/are more
complicated even though it's still an high throughput OLTP workload. And
the contention was *much* higher than what I can reproduce with pgbench
-S, there was often nearly all time spent in the lwlock's spinlock, and
it was primarily the buffer mapping lwlocks, being locked in shared
mode. The difference is that instead of locking very few buffers per
query like pgbench does, they touched much more.

Perhaps I should try to argue for this extension to pgbench again:

/messages/by-id/CAMkU=1w0K3RNhtPuLF8WQoVi6gxgG6mcnpC=-iVjwKJKyDPysw@mail.gmail.com

I think it would go a good job of exercising what you want, provided you
set the scale so that all data fit in RAM but not in shared_buffers.

Or maybe you want it to fit in shared_buffers, since the buffer mapping
lock was contended in shared mode--that suggests the problem is finding the
buffer that already has the page, not making a buffer to have the page.

Cheers,

Jeff

#31Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#29)
Re: Wait free LW_SHARED acquisition - v0.2

On Sun, Feb 2, 2014 at 6:00 AM, Andres Freund <andres@2ndquadrant.com> wrote:

The changed algorithm for lwlock imo is an *algorithmic* improvement,
not one for a particular architecture. The advantage being that locking
a lwlock which is primarily taken in shared mode will never need need to
wait or loop.

I agree. My point was only that the messaging ought to be that this is
something that those with multi-socket Intel systems should take note
of.

Yes, that branch is used by some of them. But to make that clear to all
that are still reading, I have *first* presented the patch & findings to
-hackers and *then* backported it, and I have referenced the existance
of the patch for 9.2 on list before. This isn't some kind of "secret
sauce" deal...

No, of course not. I certainly didn't mean to imply that. My point was
only that anyone that is affected to the same degree as the party with
the 4 socket server might be left with a very poor impression of
Postgres if we failed to fix the problem. It clearly rises to the
level of a bugfix.

That might be something to do later, as it *really* can hurt in
practice. We had one server go from load 240 to 11...

Well, we have to commit something on master first. But it should be a
priority to avoid having this hurt users further, since the problems
are highly predictable for certain types of servers.

But I think we should first focus on getting the patch ready for
master, then we can see where it's going. At the very least I'd like to
split of the part modifying the current spinlocks to use the atomics,
that seems far to invasive.

Agreed.

I unfortunately can't tell you that much more, not because it's private,
but because it mostly was diagnosed by remote hand debugging, limiting
insights considerably.

Of course.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#32Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#29)
Re: Wait free LW_SHARED acquisition - v0.2

On Sun, Feb 2, 2014 at 6:00 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-02-01 19:47:29 -0800, Peter Geoghegan wrote:

Here are the results of a benchmark on Nathan Boley's 64-core, 4
socket server: http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/amd-4-socket-rwlocks/

That's interesting. The maximum number of what you see here (~293125)
is markedly lower than what I can get.

... poke around ...

Hm, that's partially because you're using pgbench without -M prepared if
I see that correctly. The bottleneck in that case is primarily memory
allocation. But even after that I am getting higher
numbers: ~342497.

Trying to nail down the differnce it oddly seems to be your
max_connections=80 vs my 100. The profile in both cases is markedly
different, way much more spinlock contention with 80. All in
Pin/UnpinBuffer().

I updated this benchmark, with your BufferDescriptors alignment patch
[1]: "Misaligned BufferDescriptors causing major performance problems on AMD": /messages/by-id/20140202151319.GD32123@awork2.anarazel.de -- Peter Geoghegan
order to keep the numbers comparable). So once again, that's:

http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/amd-4-socket-rwlocks/

It made a bigger, fairly noticeable difference, but not so big a
difference as you describe here. Are you sure that you saw this kind
of difference with only 64 clients, as you mentioned elsewhere [1]"Misaligned BufferDescriptors causing major performance problems on AMD": /messages/by-id/20140202151319.GD32123@awork2.anarazel.de -- Peter Geoghegan
(perhaps you fat-fingered [1]"Misaligned BufferDescriptors causing major performance problems on AMD": /messages/by-id/20140202151319.GD32123@awork2.anarazel.de -- Peter Geoghegan -- "-cj" is ambiguous)? Obviously
max_connections is still 80 in the above. Should I have gone past 64
clients to see the problem? The best numbers I see with the [1]"Misaligned BufferDescriptors causing major performance problems on AMD": /messages/by-id/20140202151319.GD32123@awork2.anarazel.de -- Peter Geoghegan patch
applied on master is only ~327809 for -S 10 64 clients. Perhaps I've
misunderstood.

[1]: "Misaligned BufferDescriptors causing major performance problems on AMD": /messages/by-id/20140202151319.GD32123@awork2.anarazel.de -- Peter Geoghegan
on AMD": /messages/by-id/20140202151319.GD32123@awork2.anarazel.de
--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#33Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#32)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-02-03 17:51:20 -0800, Peter Geoghegan wrote:

On Sun, Feb 2, 2014 at 6:00 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-02-01 19:47:29 -0800, Peter Geoghegan wrote:

Here are the results of a benchmark on Nathan Boley's 64-core, 4
socket server: http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/amd-4-socket-rwlocks/

That's interesting. The maximum number of what you see here (~293125)
is markedly lower than what I can get.

... poke around ...

Hm, that's partially because you're using pgbench without -M prepared if
I see that correctly. The bottleneck in that case is primarily memory
allocation. But even after that I am getting higher
numbers: ~342497.

Trying to nail down the differnce it oddly seems to be your
max_connections=80 vs my 100. The profile in both cases is markedly
different, way much more spinlock contention with 80. All in
Pin/UnpinBuffer().

I updated this benchmark, with your BufferDescriptors alignment patch
[1] applied on top of master (while still not using "-M prepared" in
order to keep the numbers comparable). So once again, that's:

http://postgres-benchmarks.s3-website-us-east-1.amazonaws.com/amd-4-socket-rwlocks/

It made a bigger, fairly noticeable difference, but not so big a
difference as you describe here. Are you sure that you saw this kind
of difference with only 64 clients, as you mentioned elsewhere [1]
(perhaps you fat-fingered [1] -- "-cj" is ambiguous)? Obviously
max_connections is still 80 in the above. Should I have gone past 64
clients to see the problem? The best numbers I see with the [1] patch
applied on master is only ~327809 for -S 10 64 clients. Perhaps I've
misunderstood.

That's likely -M prepared. It was with -c 64 -j 64...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#34Christian Kruse
christian@2ndQuadrant.com
In reply to: Andres Freund (#1)
Re: Wait free LW_SHARED acquisition

Hi,

I'm doing some benchmarks regarding this problem: one set with
baseline and one set with your patch. Machine was a 32 core machine (4
CPUs with 8 cores), 252 gib RAM. Both versions have the type align
patch applied. pgbench-tools config:

SCALES="100"
SETCLIENTS="1 4 8 16 32 48 64 96 128"
SETTIMES=2

I added -M prepared to the pgbench call in the benchwarmer script.

The read-only tests are finished, I come to similiar results as yours:

<http://wwwtech.de/pg/benchmarks-lwlock-read-only/&gt;

I think the small differences are caused by the fact that I use TCP
connections and not Unix domain sockets.

The results are pretty impressive… I will post the read-write results
as soon as they are finished.

Best regards,

--
Christian Kruse http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#35Peter Geoghegan
pg@heroku.com
In reply to: Christian Kruse (#34)
Re: Wait free LW_SHARED acquisition

On Tue, Feb 4, 2014 at 11:39 AM, Christian Kruse
<christian@2ndquadrant.com> wrote:

I'm doing some benchmarks regarding this problem: one set with
baseline and one set with your patch. Machine was a 32 core machine (4
CPUs with 8 cores), 252 gib RAM. Both versions have the type align
patch applied.

It certainly seems as if the interesting cases are where clients > cores.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#36Peter Geoghegan
pg@heroku.com
In reply to: Christian Kruse (#34)
Re: Wait free LW_SHARED acquisition

On Tue, Feb 4, 2014 at 11:39 AM, Christian Kruse
<christian@2ndquadrant.com> wrote:

I added -M prepared to the pgbench call in the benchwarmer script.

The read-only tests are finished, I come to similiar results as yours:

<http://wwwtech.de/pg/benchmarks-lwlock-read-only/&gt;

Note that Christian ran this test with max_connections=201, presumably
to exercise the alignment problem.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#37Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#36)
Re: Wait free LW_SHARED acquisition

On 2014-02-04 11:48:14 -0800, Peter Geoghegan wrote:

On Tue, Feb 4, 2014 at 11:39 AM, Christian Kruse
<christian@2ndquadrant.com> wrote:

I added -M prepared to the pgbench call in the benchwarmer script.

The read-only tests are finished, I come to similiar results as yours:

<http://wwwtech.de/pg/benchmarks-lwlock-read-only/&gt;

Note that Christian ran this test with max_connections=201, presumably
to exercise the alignment problem.

I think he has applied the patch to hack around the alignment issue I
pushed to git for both branches. It's not nice enough to be applied yet,
but it should fix the issue.
I think the 201 is just a remembrance of debugging the issue.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#38Peter Geoghegan
pg@heroku.com
In reply to: Andres Freund (#37)
Re: Wait free LW_SHARED acquisition

On Tue, Feb 4, 2014 at 11:50 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I think he has applied the patch to hack around the alignment issue I
pushed to git for both branches. It's not nice enough to be applied yet,
but it should fix the issue.
I think the 201 is just a remembrance of debugging the issue.

I guess that given that *both* cases tested had the patch applied,
that makes sense. However, I would have liked to see a real master
baseline.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#39Peter Geoghegan
pg@heroku.com
In reply to: Christian Kruse (#34)
Re: Wait free LW_SHARED acquisition

On Tue, Feb 4, 2014 at 11:39 AM, Christian Kruse
<christian@2ndquadrant.com> wrote:

I'm doing some benchmarks regarding this problem: one set with
baseline and one set with your patch. Machine was a 32 core machine (4
CPUs with 8 cores), 252 gib RAM.

What CPU model? Can you post /proc/cpuinfo? The distinction between
logical and physical cores matters here.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#40Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#38)
Re: Wait free LW_SHARED acquisition

On February 4, 2014 8:53:36 PM CET, Peter Geoghegan <pg@heroku.com> wrote:

On Tue, Feb 4, 2014 at 11:50 AM, Andres Freund <andres@2ndquadrant.com>
wrote:

I think he has applied the patch to hack around the alignment issue I
pushed to git for both branches. It's not nice enough to be applied

yet,

but it should fix the issue.
I think the 201 is just a remembrance of debugging the issue.

I guess that given that *both* cases tested had the patch applied,
that makes sense. However, I would have liked to see a real master
baseline.

Christian, could you rerun with master (the commit on which the branch is based on), the alignment patch, and then the lwlock patch? Best with max_connections 200.
That's probably more important than the write tests as a first step..

Thanks,
Andres

--
Please excuse brevity and formatting - I am writing this on my mobile phone.

Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#41Christian Kruse
christian@2ndquadrant.com
In reply to: Peter Geoghegan (#39)
1 attachment(s)
Re: Wait free LW_SHARED acquisition

Hi,

On 04/02/14 12:02, Peter Geoghegan wrote:

On Tue, Feb 4, 2014 at 11:39 AM, Christian Kruse
<christian@2ndquadrant.com> wrote:

I'm doing some benchmarks regarding this problem: one set with
baseline and one set with your patch. Machine was a 32 core machine (4
CPUs with 8 cores), 252 gib RAM.

What CPU model? Can you post /proc/cpuinfo? The distinction between
logical and physical cores matters here.

model name : Intel(R) Xeon(R) CPU E5-4620 0 @ 2.20GHz

32 physical cores, 64 logical cores. /proc/cpuinfo is applied.

Best regards,

--
Christian Kruse http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

cpuinfotext/plain; charset=us-asciiDownload
#42Christian Kruse
christian@2ndquadrant.com
In reply to: Christian Kruse (#41)
1 attachment(s)
Re: Wait free LW_SHARED acquisition

Hi,

On 04/02/14 21:05, Christian Kruse wrote:

[…] /proc/cpuinfo is applied.

I meant „attached“ and it seems that scp on /proc/cpuinfo doesn't
work…

This time the cpuinfo is attached with content ;-)

Best regards,

--
Christian Kruse http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

cpuinfotext/plain; charset=us-asciiDownload
#43Christian Kruse
christian@2ndquadrant.com
In reply to: Andres Freund (#40)
Re: Wait free LW_SHARED acquisition

Hi,

On 04/02/14 21:03, Andres Freund wrote:

Christian, could you rerun with master (the commit on which the
branch is based on), the alignment patch, and then the lwlock patch?
Best with max_connections 200. That's probably more important than
the write tests as a first step..

Ok, benchmark for baseline+alignment patch is running. This will take
a couple of hours and since I have to get up at about 05:00 I won't be
able to post it before tomorrow.

Best regards,

--
Christian Kruse http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#44Peter Geoghegan
pg@heroku.com
In reply to: Christian Kruse (#43)
Re: Wait free LW_SHARED acquisition

On Tue, Feb 4, 2014 at 12:30 PM, Christian Kruse
<christian@2ndquadrant.com> wrote:

Ok, benchmark for baseline+alignment patch is running.

I see that you have enabled latency information. For this kind of
thing I prefer to hack pgbench-tools to not collect this (i.e. to not
pass the "-l" flag, "Per-Transaction Logging"). Just remove it and
pgbench-tools rolls with it. It may well be that the overhead added is
completely insignificant, but for something like this, where the
latency information is unlikely to add any value, I prefer to not take
the chance. This is a fairly minor point, however, especially since
these are only 60 second runs where you're unlikely to accumulate
enough transaction latency information to notice any effect.

--
Peter Geoghegan

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#45Andres Freund
andres@2ndquadrant.com
In reply to: Peter Geoghegan (#44)
Re: Wait free LW_SHARED acquisition

On 2014-02-04 13:42:51 -0800, Peter Geoghegan wrote:

On Tue, Feb 4, 2014 at 12:30 PM, Christian Kruse
<christian@2ndquadrant.com> wrote:

Ok, benchmark for baseline+alignment patch is running.

I see that you have enabled latency information. For this kind of
thing I prefer to hack pgbench-tools to not collect this (i.e. to not
pass the "-l" flag, "Per-Transaction Logging"). Just remove it and
pgbench-tools rolls with it. It may well be that the overhead added is
completely insignificant, but for something like this, where the
latency information is unlikely to add any value, I prefer to not take
the chance. This is a fairly minor point, however, especially since
these are only 60 second runs where you're unlikely to accumulate
enough transaction latency information to notice any effect.

Hm, I don't find that convincing. If you look at the results from the
last run the latency information is actually quite interesting.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#46Heikki Linnakangas
hlinnakangas@vmware.com
In reply to: Andres Freund (#22)
Re: Wait free LW_SHARED acquisition - v0.2

On 01/31/2014 11:54 AM, Andres Freund wrote:

Hi,

On 2014-01-28 21:27:29 -0800, Peter Geoghegan wrote:

On Fri, Nov 15, 2013 at 11:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

1) I've added an abstracted atomic ops implementation. Needs a fair
amount of work, also submitted as a separate CF entry. (Patch 1 & 2)

Commit 220b34331f77effdb46798ddd7cca0cffc1b2858 caused bitrot when
applying 0002-Very-basic-atomic-ops-implementation.patch. Please
rebase.

I've pushed a rebased version of the patchset to
http://git.postgresql.org/gitweb/?p=users/andresfreund/postgres.git
branch rwlock contention.
220b34331f77effdb46798ddd7cca0cffc1b2858 actually was the small problem,
ea9df812d8502fff74e7bc37d61bdc7d66d77a7f was the major PITA.

I plan to split the atomics patch into smaller chunks before
reposting. Imo the "Convert the PGPROC->lwWaitLink list into a dlist
instead of open coding it." is worth being applied independently from
the rest of the series, it simplies code and it fixes a bug...

I committed a fix for the WakeupWaiters() bug now, without the rest of
the "open coding" patch. Converting lwWaitLInk into a dlist is probably
a good idea, but seems better to fix the bug separately, for the sake of
git history if nothing else.

- Heikki

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#47Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#22)
Re: Wait free LW_SHARED acquisition - v0.2

On Fri, Jan 31, 2014 at 3:24 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

I've pushed a rebased version of the patchset to
http://git.postgresql.org/gitweb/?p=users/andresfreund/postgres.git
branch rwlock contention.
220b34331f77effdb46798ddd7cca0cffc1b2858 actually was the small problem,
ea9df812d8502fff74e7bc37d61bdc7d66d77a7f was the major PITA.

As per discussion in developer meeting, I wanted to test shared
buffer scaling patch with this branch. I am getting merge
conflicts as per HEAD. Could you please get it resolved, so that
I can get the data.

From git://git.postgresql.org/git/users/andresfreund/postgres
* branch rwlock-contention -> FETCH_HEAD
Auto-merging src/test/regress/regress.c
CONFLICT (content): Merge conflict in src/test/regress/regress.c
Auto-merging src/include/storage/proc.h
Auto-merging src/include/storage/lwlock.h
CONFLICT (content): Merge conflict in src/include/storage/lwlock.h
Auto-merging src/include/storage/ipc.h
CONFLICT (content): Merge conflict in src/include/storage/ipc.h
Auto-merging src/include/storage/barrier.h
CONFLICT (content): Merge conflict in src/include/storage/barrier.h
Auto-merging src/include/pg_config_manual.h
Auto-merging src/include/c.h
Auto-merging src/backend/storage/lmgr/spin.c
Auto-merging src/backend/storage/lmgr/proc.c
Auto-merging src/backend/storage/lmgr/lwlock.c
CONFLICT (content): Merge conflict in src/backend/storage/lmgr/lwlock.c
Auto-merging src/backend/storage/ipc/shmem.c
Auto-merging src/backend/storage/ipc/ipci.c
Auto-merging src/backend/access/transam/xlog.c
CONFLICT (content): Merge conflict in src/backend/access/transam/xlog.c
Auto-merging src/backend/access/transam/twophase.c
Auto-merging configure.in
Auto-merging configure
Auto-merging config/c-compiler.m4
Automatic merge failed; fix conflicts and then commit the result.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#48Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#47)
Re: Wait free LW_SHARED acquisition - v0.2

On Fri, May 23, 2014 at 10:01 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Fri, Jan 31, 2014 at 3:24 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

I've pushed a rebased version of the patchset to
http://git.postgresql.org/gitweb/?p=users/andresfreund/postgres.git
branch rwlock contention.
220b34331f77effdb46798ddd7cca0cffc1b2858 actually was the small problem,
ea9df812d8502fff74e7bc37d61bdc7d66d77a7f was the major PITA.

As per discussion in developer meeting, I wanted to test shared
buffer scaling patch with this branch. I am getting merge
conflicts as per HEAD. Could you please get it resolved, so that
I can get the data.

I have started looking into this patch and have few questions/
findings which are shared below:

1. I think stats for lwstats->ex_acquire_count will be counted twice,
first it is incremented in LWLockAcquireCommon() and then in
LWLockAttemptLock()

2.
Handling of potentialy_spurious case seems to be pending
in LWLock functions like LWLockAcquireCommon().

LWLockAcquireCommon()
{
..
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);

}

I think it can lead to some problems, example:

Session -1
1. Acquire Exclusive LWlock

Session -2
1. Acquire Shared LWlock

1a. Unconditionally incrementing shared count by session-2

Session -1
2. Release Exclusive lock

Session -3
1. Acquire Exclusive LWlock
It will put itself to wait queue by seeing the lock count incremented
by Session-2

Session-2
1b. Decrement the shared count and add it to wait queue.

Session-4
1. Acquire Exclusive lock
This session will get the exclusive lock, because even
though other lockers are waiting, lockcount is zero.

Session-2
2. Try second time to take shared lock, it won't get
as session-4 already has an exclusive lock, so it will
start waiting

Session-4
2. Release Exclusive lock
it will not wake the waiters because waiters have been added
before acquiring this lock.

So in above scenario, Session-3 and Session-2 are waiting in queue
with nobody to awake them.

I have not reproduced the exact scenario above,
so I might be missing some thing which will not
lead to above situation.

3.
LWLockAcquireCommon()
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
..
}
proc->lwWaiting is checked, updated without spinklock where
as previously it was done under spinlock, won't it be unsafe?

4.
LWLockAcquireCommon()
{
..
for (;;)
{
/* "false" means cannot accept cancel/die interrupt here. */
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
}

lock->releaseOK is updated/checked without spinklock where
as previously it was done under spinlock, won't it be unsafe?

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#49Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#48)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-06-17 12:41:26 +0530, Amit Kapila wrote:

On Fri, May 23, 2014 at 10:01 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Fri, Jan 31, 2014 at 3:24 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

I've pushed a rebased version of the patchset to
http://git.postgresql.org/gitweb/?p=users/andresfreund/postgres.git
branch rwlock contention.
220b34331f77effdb46798ddd7cca0cffc1b2858 actually was the small problem,
ea9df812d8502fff74e7bc37d61bdc7d66d77a7f was the major PITA.

As per discussion in developer meeting, I wanted to test shared
buffer scaling patch with this branch. I am getting merge
conflicts as per HEAD. Could you please get it resolved, so that
I can get the data.

I have started looking into this patch and have few questions/
findings which are shared below:

1. I think stats for lwstats->ex_acquire_count will be counted twice,
first it is incremented in LWLockAcquireCommon() and then in
LWLockAttemptLock()

Hrmpf. Will fix.

2.
Handling of potentialy_spurious case seems to be pending
in LWLock functions like LWLockAcquireCommon().

LWLockAcquireCommon()
{
..
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);

}

I think it can lead to some problems, example:

Session -1
1. Acquire Exclusive LWlock

Session -2
1. Acquire Shared LWlock

1a. Unconditionally incrementing shared count by session-2

Session -1
2. Release Exclusive lock

Session -3
1. Acquire Exclusive LWlock
It will put itself to wait queue by seeing the lock count incremented
by Session-2

Session-2
1b. Decrement the shared count and add it to wait queue.

Session-4
1. Acquire Exclusive lock
This session will get the exclusive lock, because even
though other lockers are waiting, lockcount is zero.

Session-2
2. Try second time to take shared lock, it won't get
as session-4 already has an exclusive lock, so it will
start waiting

Session-4
2. Release Exclusive lock
it will not wake the waiters because waiters have been added
before acquiring this lock.

I don't understand this step here? When releasing the lock it'll notice
that the waiters is <> 0 and acquire the spinlock which should protect
against badness here?

3.
LWLockAcquireCommon()
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
..
}
proc->lwWaiting is checked, updated without spinklock where
as previously it was done under spinlock, won't it be unsafe?

It was previously checked/unset without a spinlock as well:
/*
* Awaken any waiters I removed from the queue.
*/
while (head != NULL)
{
LOG_LWDEBUG("LWLockRelease", T_NAME(l), T_ID(l), "release waiter");
proc = head;
head = proc->lwWaitLink;
proc->lwWaitLink = NULL;
proc->lwWaiting = false;
PGSemaphoreUnlock(&proc->sem);
}
I don't think there's dangers here, lwWaiting will only ever be
manipulated by the the PGPROC's owner. As discussed elsewhere there
needs to be a write barrier before the proc->lwWaiting = false, even in
upstream code.

4.
LWLockAcquireCommon()
{
..
for (;;)
{
/* "false" means cannot accept cancel/die interrupt here. */
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
}

lock->releaseOK is updated/checked without spinklock where
as previously it was done under spinlock, won't it be unsafe?

Hm. That's probably buggy. Good catch. Especially if you have a compiler
that does byte manipulation by reading e.g. 4 bytes from a struct and
then write the wider variable back... So the releaseOk bit needs to move
into LWLockDequeueSelf().

Thanks for looking!

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#50Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#49)
Re: Wait free LW_SHARED acquisition - v0.2

On Tue, Jun 17, 2014 at 3:56 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-17 12:41:26 +0530, Amit Kapila wrote:

On Fri, May 23, 2014 at 10:01 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Fri, Jan 31, 2014 at 3:24 PM, Andres Freund <andres@2ndquadrant.com

wrote:

I've pushed a rebased version of the patchset to
http://git.postgresql.org/gitweb/?p=users/andresfreund/postgres.git
branch rwlock contention.
220b34331f77effdb46798ddd7cca0cffc1b2858 actually was the small

problem,

ea9df812d8502fff74e7bc37d61bdc7d66d77a7f was the major PITA.

As per discussion in developer meeting, I wanted to test shared
buffer scaling patch with this branch. I am getting merge
conflicts as per HEAD. Could you please get it resolved, so that
I can get the data.

I have started looking into this patch and have few questions/
findings which are shared below:

1. I think stats for lwstats->ex_acquire_count will be counted twice,
first it is incremented in LWLockAcquireCommon() and then in
LWLockAttemptLock()

Hrmpf. Will fix.

2.
Handling of potentialy_spurious case seems to be pending
in LWLock functions like LWLockAcquireCommon().

LWLockAcquireCommon()
{
..
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);

}

I think it can lead to some problems, example:

Session -1
1. Acquire Exclusive LWlock

Session -2
1. Acquire Shared LWlock

1a. Unconditionally incrementing shared count by session-2

Session -1
2. Release Exclusive lock

Session -3
1. Acquire Exclusive LWlock
It will put itself to wait queue by seeing the lock count incremented
by Session-2

Session-2
1b. Decrement the shared count and add it to wait queue.

Session-4
1. Acquire Exclusive lock
This session will get the exclusive lock, because even
though other lockers are waiting, lockcount is zero.

Session-2
2. Try second time to take shared lock, it won't get
as session-4 already has an exclusive lock, so it will
start waiting

Session-4
2. Release Exclusive lock
it will not wake the waiters because waiters have been added
before acquiring this lock.

I don't understand this step here? When releasing the lock it'll notice
that the waiters is <> 0 and acquire the spinlock which should protect
against badness here?

While Releasing lock, I think it will not go to Wakeup waiters
(LWLockWakeup), because releaseOK will be false. releaseOK
can be set as false when Session-1 has Released Exclusive lock
and wakedup some previous waiter. Once it is set to false, it can
be reset to true only for retry logic(after getting semaphore).

3.

I don't think there's dangers here, lwWaiting will only ever be
manipulated by the the PGPROC's owner. As discussed elsewhere there
needs to be a write barrier before the proc->lwWaiting = false, even in
upstream code.

Agreed.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#51Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#50)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-06-17 18:01:58 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 3:56 PM, Andres Freund <andres@2ndquadrant.com>

On 2014-06-17 12:41:26 +0530, Amit Kapila wrote:

2.
Handling of potentialy_spurious case seems to be pending
in LWLock functions like LWLockAcquireCommon().

LWLockAcquireCommon()
{
..
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);

}

I think it can lead to some problems, example:

Session -1
1. Acquire Exclusive LWlock

Session -2
1. Acquire Shared LWlock

1a. Unconditionally incrementing shared count by session-2

Session -1
2. Release Exclusive lock

Session -3
1. Acquire Exclusive LWlock
It will put itself to wait queue by seeing the lock count incremented
by Session-2

Session-2
1b. Decrement the shared count and add it to wait queue.

Session-4
1. Acquire Exclusive lock
This session will get the exclusive lock, because even
though other lockers are waiting, lockcount is zero.

Session-2
2. Try second time to take shared lock, it won't get
as session-4 already has an exclusive lock, so it will
start waiting

Session-4
2. Release Exclusive lock
it will not wake the waiters because waiters have been added
before acquiring this lock.

I don't understand this step here? When releasing the lock it'll notice
that the waiters is <> 0 and acquire the spinlock which should protect
against badness here?

While Releasing lock, I think it will not go to Wakeup waiters
(LWLockWakeup), because releaseOK will be false. releaseOK
can be set as false when Session-1 has Released Exclusive lock
and wakedup some previous waiter. Once it is set to false, it can
be reset to true only for retry logic(after getting semaphore).

I unfortunately still can't follow. If Session-1 woke up some previous
waiter the woken up process will set releaseOK to true again when it
loops to acquire the lock?

Somewhat unrelated:

I have a fair amount of doubt about the effectiveness of the releaseOK
logic (which imo also is pretty poorly documented).
Essentially its intent is to avoid unneccessary scheduling when other
processes have already been woken up (i.e. releaseOK has been set to
false). I believe the theory is that if any process has already been
woken up it's pointless to wake up additional processes
(i.e. PGSemaphoreUnlock()) because the originally woken up process will
wake up at some point. But if the to-be-woken up process is scheduled
out because it used all his last timeslices fully that means we'll not
wakeup other waiters for a relatively long time.

It's been introduced in the course of
5b9a058384e714b89e050fc0b6381f97037c665a whose logic generally is rather
sound - I just doubt that the releaseOK part is necessary.

It'd certainly interesting to rip releaseOK out and benchmark the
result... My theory is that the average latency will go down on busy
systems that aren't IO bound.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#52Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#51)
Re: Wait free LW_SHARED acquisition - v0.2

On Tue, Jun 17, 2014 at 6:35 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-17 18:01:58 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 3:56 PM, Andres Freund <andres@2ndquadrant.com>

On 2014-06-17 12:41:26 +0530, Amit Kapila wrote:

2.
Handling of potentialy_spurious case seems to be pending
in LWLock functions like LWLockAcquireCommon().

LWLockAcquireCommon()
{
..
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);

}

I think it can lead to some problems, example:

Session -1
1. Acquire Exclusive LWlock

Session -2
1. Acquire Shared LWlock

1a. Unconditionally incrementing shared count by session-2

Session -1
2. Release Exclusive lock

Session -3
1. Acquire Exclusive LWlock
It will put itself to wait queue by seeing the lock count

incremented

by Session-2

Session-2
1b. Decrement the shared count and add it to wait queue.

Session-4
1. Acquire Exclusive lock
This session will get the exclusive lock, because even
though other lockers are waiting, lockcount is zero.

Session-2
2. Try second time to take shared lock, it won't get
as session-4 already has an exclusive lock, so it will
start waiting

Session-4
2. Release Exclusive lock
it will not wake the waiters because waiters have been added
before acquiring this lock.

I don't understand this step here? When releasing the lock it'll

notice

that the waiters is <> 0 and acquire the spinlock which should protect
against badness here?

While Releasing lock, I think it will not go to Wakeup waiters
(LWLockWakeup), because releaseOK will be false. releaseOK
can be set as false when Session-1 has Released Exclusive lock
and wakedup some previous waiter. Once it is set to false, it can
be reset to true only for retry logic(after getting semaphore).

I unfortunately still can't follow.

You have followed it pretty well as far as I can understand from your
replies, as there is no reproducible test (which I think is bit tricky to
prepare), so it becomes difficult to explain by theory.

If Session-1 woke up some previous
waiter the woken up process will set releaseOK to true again when it
loops to acquire the lock?

You are right, it will wakeup the existing waiters, but I think the
new logic has one difference which is that it can allow the backend to
take Exclusive lock when there are already waiters in queue. As per
above example even though Session-2 and Session-3 are in wait
queue, Session-4 will be able to acquire Exclusive lock which I think
was previously not possible.

Somewhat unrelated:

I have a fair amount of doubt about the effectiveness of the releaseOK
logic (which imo also is pretty poorly documented).
Essentially its intent is to avoid unneccessary scheduling when other
processes have already been woken up (i.e. releaseOK has been set to
false). I believe the theory is that if any process has already been
woken up it's pointless to wake up additional processes
(i.e. PGSemaphoreUnlock()) because the originally woken up process will
wake up at some point. But if the to-be-woken up process is scheduled
out because it used all his last timeslices fully that means we'll not
wakeup other waiters for a relatively long time.

I think it will also maintain that the wokedup process won't stall for
very long time, because if we wake new waiters, then previously woked
process can again enter into wait queue and similar thing can repeat
for long time.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#53Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#52)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-06-17 20:47:51 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 6:35 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-17 18:01:58 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 3:56 PM, Andres Freund <andres@2ndquadrant.com>

On 2014-06-17 12:41:26 +0530, Amit Kapila wrote:

I unfortunately still can't follow.

You have followed it pretty well as far as I can understand from your
replies, as there is no reproducible test (which I think is bit tricky to
prepare), so it becomes difficult to explain by theory.

I'm working an updated patch that moves the releaseOK into the
spinlocks. Maybe that's the problem already - it's certainly not correct
as is.

If Session-1 woke up some previous
waiter the woken up process will set releaseOK to true again when it
loops to acquire the lock?

You are right, it will wakeup the existing waiters, but I think the
new logic has one difference which is that it can allow the backend to
take Exclusive lock when there are already waiters in queue. As per
above example even though Session-2 and Session-3 are in wait
queue, Session-4 will be able to acquire Exclusive lock which I think
was previously not possible.

I think that was previously possible as well - in a slightly different
set of circumstances though. After a process releases a lock and wakes
up some of several waiters another process can come in and acquire the
lock. Before the woken up process gets scheduled again. lwlocks aren't
fair locks...

Somewhat unrelated:

I have a fair amount of doubt about the effectiveness of the releaseOK
logic (which imo also is pretty poorly documented).
Essentially its intent is to avoid unneccessary scheduling when other
processes have already been woken up (i.e. releaseOK has been set to
false). I believe the theory is that if any process has already been
woken up it's pointless to wake up additional processes
(i.e. PGSemaphoreUnlock()) because the originally woken up process will
wake up at some point. But if the to-be-woken up process is scheduled
out because it used all his last timeslices fully that means we'll not
wakeup other waiters for a relatively long time.

I think it will also maintain that the wokedup process won't stall for
very long time, because if we wake new waiters, then previously woked
process can again enter into wait queue and similar thing can repeat
for long time.

I don't think it effectively does that - newly incoming lockers ignore
the queue and just acquire the lock. Even if there's some other backend
scheduled to wake up. And shared locks can be acquired when there's
exclusive locks waiting.

I think both are actually critical for performance... Otherwise even a
only lightly contended lock would require scheduler activity when a
processes tries to lock something twice. Given the frequency we acquire
some locks with that'd be disastrous...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#54Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#53)
Re: Wait free LW_SHARED acquisition - v0.2

On Tue, Jun 17, 2014 at 8:56 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-17 20:47:51 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 6:35 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

You have followed it pretty well as far as I can understand from your
replies, as there is no reproducible test (which I think is bit tricky

to

prepare), so it becomes difficult to explain by theory.

I'm working an updated patch that moves the releaseOK into the
spinlocks. Maybe that's the problem already - it's certainly not correct
as is.

Sure, I will do the test/performance test with updated patch; you
might want to include some more changes based on comments
in mail below:

You are right, it will wakeup the existing waiters, but I think the
new logic has one difference which is that it can allow the backend to
take Exclusive lock when there are already waiters in queue. As per
above example even though Session-2 and Session-3 are in wait
queue, Session-4 will be able to acquire Exclusive lock which I think
was previously not possible.

I think that was previously possible as well - in a slightly different
set of circumstances though. After a process releases a lock and wakes
up some of several waiters another process can come in and acquire the
lock. Before the woken up process gets scheduled again. lwlocks aren't
fair locks...

Okay, but I think changing behaviour for lwlocks might impact some
tests/applications. As they are not fair, I think defining exact
behaviour is not easy and we don't have any concrete scenario which
can be effected, so there should not be problem in accepting
slightly different behaviour.

Few more comments:

1.
LWLockAcquireCommon()
{
..
iterations++;
}

In current logic, I could not see any use of these *iterations* variable.

2.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
/*
* Somebody else dequeued us and has or will..
..
*/
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
}

Do we want to set result = false; after waking in above code?
The idea behind setting false is to indicate whether we get the lock
immediately or not which previously was decided based on if it needs
to queue itself?

3.
LWLockAcquireCommon()
{
..
/*
* Ok, at this point we couldn't grab the lock on the first try. We
* cannot simply queue ourselves to the end of the list and wait to be
* woken up because by now the lock could long have been released.
* Instead add us to the queue and try to grab the lock again. If we
* suceed we need to revert the queuing and be happy, otherwise we
* recheck the lock. If we still couldn't grab it, we know that the
* other lock will see our queue entries when releasing since they
* existed before we checked for the lock.
*/
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);
..
}

a. By reading above code and comments, it is not quite clear why
second attempt is important unless somebody thinks on it or refer
your comments in *Notes* section at top of file. I think it's better to
indicate in some way so that code reader can refer to Notes section or
whereever you are planing to keep those comments.

b. There is typo in above comment suceed/succeed.

4.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
..
}

Setting releaseOK in above context might not be required because if the
control comes in this part of code, it will not retry to acquire another
time.

5.
LWLockWaitForVar()
{
..
else
mustwait = false;

if (!mustwait)
break;
..
}

I think we can directly break in else part in above code.

6.
LWLockWaitForVar()
{
..
/*
* Quick test first to see if it the slot is free right now.
*
* XXX: the caller uses a spinlock before this,...
*/
if (pg_atomic_read_u32(&lock->lockcount) == 0)
return true;
}

Does the part of comment that refers to spinlock is still relevant
after using atomic ops?

7.
LWLockWaitForVar()
{
..
/*
* Add myself to wait queue. Note that this is racy, somebody else
* could wakeup before we're finished queuing.
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, LW_EXCLUSIVE, false,
&potentially_spurious);
}

Why is it important to Attempt lock after queuing in this case, can't
we just re-check exclusive lock as done before queuing?

8.
LWLockWaitForVar()
{
..
PRINT_LWDEBUG("LWLockAcquire undo queue", lock, mode);
break;
}
else
{
PRINT_LWDEBUG("LWLockAcquire waiting 4", lock, mode);
}
..
}

a. I think instead of LWLockAcquire, here we should use
LWLockWaitForVar
b. Isn't it better to use LOG_LWDEBUG instead of PRINT_LWDEBUG(),
as PRINT_LWDEBUG() is generally used in file at entry of functions to
log info about locks?

9.
LWLockUpdateVar()
{
/* Acquire mutex. Time spent holding mutex should be short! */
SpinLockAcquire(&lock->mutex);
..
}

Current code has an Assert for exclusive lock which is missing in
patch, is there any reason for removing Assert?

10.
LWLockRelease(LWLock *l)
{
..
if (l == held_lwlocks[i].lock)
{
mode = held_lwlocks[i].mode;
..
}

It seems that mode is added to held_lwlocks to use it in LWLockRelease().
If yes, then can we deduce the same from lockcount?

11.
LWLockRelease()
{
..
PRINT_LWDEBUG("LWLockRelease", lock, mode);
}

Shouldn't this be in begining of LWLockRelease function rather than
after processing held_lwlocks array?

12.
#ifdef LWLOCK_DEBUG
lock->owner = MyProc;
#endif

Shouldn't it be reset in LWLockRelease?

13.
* This protects us against both problems from above:
* 1) Nobody can release too quick, before we're queued, since after Phase
2 since we're
* already queued.

Second *since* seems to be typo.

* 2) If somebody spuriously got blocked from acquiring the lock, they will
* get queued in Phase 2 and we can wake them up if neccessary or they
will
* have gotten the lock in Phase 2.

In the above line, I think the second mention of *Phase 2* should be *Phase
3*.

Somewhat unrelated:

I think it will also maintain that the wokedup process won't stall for
very long time, because if we wake new waiters, then previously woked
process can again enter into wait queue and similar thing can repeat
for long time.

I don't think it effectively does that - newly incoming lockers ignore
the queue and just acquire the lock. Even if there's some other backend
scheduled to wake up. And shared locks can be acquired when there's
exclusive locks waiting.

They ignore the queue but I think they won't wakeup new waiters unless
some previous wokedup waiter again tries to acquire lock (as that will
set releaseOK). I am not sure how much such a restriction helps, but
still I think it reduces the chance of getting it stalled.

I think both are actually critical for performance... Otherwise even a
only lightly contended lock would require scheduler activity when a
processes tries to lock something twice. Given the frequency we acquire
some locks with that'd be disastrous...

Do you have any suggestion how both behaviours can be retained?

Note - Still there is more to review in this patch, however I feel it is
good idea to start some test/performance test of this patch, if you
agree, then I will start the same with the updated patch (result
of conclusion of current review comments).

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#55Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#54)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-06-23 19:59:10 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 8:56 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-17 20:47:51 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 6:35 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

You have followed it pretty well as far as I can understand from your
replies, as there is no reproducible test (which I think is bit tricky

to

prepare), so it becomes difficult to explain by theory.

I'm working an updated patch that moves the releaseOK into the
spinlocks. Maybe that's the problem already - it's certainly not correct
as is.

Sure, I will do the test/performance test with updated patch; you
might want to include some more changes based on comments
in mail below:

I'm nearly finished in cleaning up the atomics part of the patch which
also includes a bit of cleanup of the lwlocks code.

Few more comments:

1.
LWLockAcquireCommon()
{
..
iterations++;
}

In current logic, I could not see any use of these *iterations* variable.

It's useful for debugging. Should be gone in the final code.

2.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
/*
* Somebody else dequeued us and has or will..
..
*/
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
}

Do we want to set result = false; after waking in above code?
The idea behind setting false is to indicate whether we get the lock
immediately or not which previously was decided based on if it needs
to queue itself?

Hm. I don't think it's clear which version is better.

3.
LWLockAcquireCommon()
{
..
/*
* Ok, at this point we couldn't grab the lock on the first try. We
* cannot simply queue ourselves to the end of the list and wait to be
* woken up because by now the lock could long have been released.
* Instead add us to the queue and try to grab the lock again. If we
* suceed we need to revert the queuing and be happy, otherwise we
* recheck the lock. If we still couldn't grab it, we know that the
* other lock will see our queue entries when releasing since they
* existed before we checked for the lock.
*/
/* add to the queue */
LWLockQueueSelf(l, mode);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);
..
}

a. By reading above code and comments, it is not quite clear why
second attempt is important unless somebody thinks on it or refer
your comments in *Notes* section at top of file. I think it's better to
indicate in some way so that code reader can refer to Notes section or
whereever you are planing to keep those comments.

Ok.

b. There is typo in above comment suceed/succeed.

Thanks, fixed.

4.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
..
}

Setting releaseOK in above context might not be required because if the
control comes in this part of code, it will not retry to acquire another
time.

Hm. You're probably right.

5.
LWLockWaitForVar()
{
..
else
mustwait = false;

if (!mustwait)
break;
..
}

I think we can directly break in else part in above code.

Well, there's another case of mustwait=false above which is triggered
while the spinlock is held. Don't think it'd get simpler.

6.
LWLockWaitForVar()
{
..
/*
* Quick test first to see if it the slot is free right now.
*
* XXX: the caller uses a spinlock before this,...
*/
if (pg_atomic_read_u32(&lock->lockcount) == 0)
return true;
}

Does the part of comment that refers to spinlock is still relevant
after using atomic ops?

Yes. pg_atomic_read_u32() isn't a memory barrier (and explicitly
documented not to be).

7.
LWLockWaitForVar()
{
..
/*
* Add myself to wait queue. Note that this is racy, somebody else
* could wakeup before we're finished queuing.
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, LW_EXCLUSIVE, false,
&potentially_spurious);
}

Why is it important to Attempt lock after queuing in this case, can't
we just re-check exclusive lock as done before queuing?

Well, that's how Heikki designed LWLockWaitForVar().

8.
LWLockWaitForVar()
{
..
PRINT_LWDEBUG("LWLockAcquire undo queue", lock, mode);
break;
}
else
{
PRINT_LWDEBUG("LWLockAcquire waiting 4", lock, mode);
}
..
}

a. I think instead of LWLockAcquire, here we should use
LWLockWaitForVar

right.

b. Isn't it better to use LOG_LWDEBUG instead of PRINT_LWDEBUG(),
as PRINT_LWDEBUG() is generally used in file at entry of functions to
log info about locks?

Fine with me.

9.
LWLockUpdateVar()
{
/* Acquire mutex. Time spent holding mutex should be short! */
SpinLockAcquire(&lock->mutex);
..
}

Current code has an Assert for exclusive lock which is missing in
patch, is there any reason for removing Assert?

That assert didn't use to be there on master... I'll add it again.

10.
LWLockRelease(LWLock *l)
{
..
if (l == held_lwlocks[i].lock)
{
mode = held_lwlocks[i].mode;
..
}

It seems that mode is added to held_lwlocks to use it in LWLockRelease().
If yes, then can we deduce the same from lockcount?

No. It can be temporarily too high (the whole backout stuff). It's also
much cheaper to test a process local variable.

11.
LWLockRelease()
{
..
PRINT_LWDEBUG("LWLockRelease", lock, mode);
}

Shouldn't this be in begining of LWLockRelease function rather than
after processing held_lwlocks array?

Ok.

12.
#ifdef LWLOCK_DEBUG
lock->owner = MyProc;
#endif

Shouldn't it be reset in LWLockRelease?

That's actually intentional. It's quite useful to know the last owner
when debugging lwlock code.

* 2) If somebody spuriously got blocked from acquiring the lock, they will
* get queued in Phase 2 and we can wake them up if neccessary or they
will
* have gotten the lock in Phase 2.

In the above line, I think the second mention of *Phase 2* should be *Phase
3*.

Right, good catch.

I think both are actually critical for performance... Otherwise even a
only lightly contended lock would require scheduler activity when a
processes tries to lock something twice. Given the frequency we acquire
some locks with that'd be disastrous...

Do you have any suggestion how both behaviours can be retained?

Not sure what you mean. They currently *are* retained? Or do you mean
whether they could be retained while making lwlocks fai?

Note - Still there is more to review in this patch, however I feel it is
good idea to start some test/performance test of this patch, if you
agree, then I will start the same with the updated patch (result
of conclusion of current review comments).

I'll post a new version of this + the atomics patch tomorrow. Since the
whole atomics stuff has changed noticeably it probably makes sense to
wait till then.

Thanks for the look!

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#56Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#55)
Re: Wait free LW_SHARED acquisition - v0.2

On Mon, Jun 23, 2014 at 9:12 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-23 19:59:10 +0530, Amit Kapila wrote:

On Tue, Jun 17, 2014 at 8:56 PM, Andres Freund <andres@2ndquadrant.com>
wrote:
2.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
/*
* Somebody else dequeued us and has or will..
..
*/
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
}

Do we want to set result = false; after waking in above code?
The idea behind setting false is to indicate whether we get the lock
immediately or not which previously was decided based on if it needs
to queue itself?

Hm. I don't think it's clear which version is better.

I thought if we get the lock at first attempt, then result should be
true which seems to be clear, but for the case of second attempt you
are right that it's not clear. In such a case, I think we can go either
way and then later during tests or otherwise if any problem is discovered,
we can revert it.

7.
LWLockWaitForVar()
{
..
/*
* Add myself to wait queue. Note that this is racy, somebody else
* could wakeup before we're finished queuing.
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, LW_EXCLUSIVE, false,
&potentially_spurious);
}

Why is it important to Attempt lock after queuing in this case, can't
we just re-check exclusive lock as done before queuing?

Well, that's how Heikki designed LWLockWaitForVar().

In that case I might be missing some point here, un-patched code of
LWLockWaitForVar() never tries to acquire the lock, but the new code
does so. Basically I am not able to think what is the problem if we just
do below after queuing:
mustwait = pg_atomic_read_u32(&lock->lockcount) != 0;

Could you please explain what is the problem in just rechecking?

I think both are actually critical for performance... Otherwise even a
only lightly contended lock would require scheduler activity when a
processes tries to lock something twice. Given the frequency we

acquire

some locks with that'd be disastrous...

Do you have any suggestion how both behaviours can be retained?

Not sure what you mean.

I just wanted to say that current behaviour of releaseOK seems to
be of use for some cases and if you want to change it, then would it
retain the current behaviour we get by releaseOK?

I understand that till now your patch has not changed anything specific
to releaseOK, but by above discussion I got the impression that you are
planing to change it, that's why I had asked above question.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#57Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#55)
Re: Wait free LW_SHARED acquisition - v0.2

On Mon, Jun 23, 2014 at 9:12 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-23 19:59:10 +0530, Amit Kapila wrote:

12.
#ifdef LWLOCK_DEBUG
lock->owner = MyProc;
#endif

Shouldn't it be reset in LWLockRelease?

That's actually intentional. It's quite useful to know the last owner
when debugging lwlock code.

Won't it cause any problem if the last owner process exits?

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#58Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#56)
Re: Wait free LW_SHARED acquisition - v0.2

On Tue, Jun 24, 2014 at 9:33 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Mon, Jun 23, 2014 at 9:12 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

On 2014-06-23 19:59:10 +0530, Amit Kapila wrote:

7.
LWLockWaitForVar()
{
..
/*
* Add myself to wait queue. Note that this is racy, somebody else
* could wakeup before we're finished queuing.
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

/* we're now guaranteed to be woken up if necessary */
mustwait = LWLockAttemptLock(l, LW_EXCLUSIVE, false,
&potentially_spurious);
}

Why is it important to Attempt lock after queuing in this case, can't
we just re-check exclusive lock as done before queuing?

Well, that's how Heikki designed LWLockWaitForVar().

In that case I might be missing some point here, un-patched code of
LWLockWaitForVar() never tries to acquire the lock, but the new code
does so. Basically I am not able to think what is the problem if we just
do below after queuing:
mustwait = pg_atomic_read_u32(&lock->lockcount) != 0;

Could you please explain what is the problem in just rechecking?

I have further reviewed the lwlock related changes and thought
its good to share my findings with you. This completes my initial
review for lwlock related changes and below are my findings:

1.
LWLockRelease()
{
..
TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(l), T_ID(l));
}

Dynamic tracing macro seems to be omitted from LWLockRelease()
call.

2.
LWLockWakeup()
{
..
#ifdef LWLOCK_STATS
lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
#else
SpinLockAcquire(&lock->mutex);
#endif
..
}

Earlier while releasing lock, we don't count it towards LWLock stats
spin_delay_count. I think if we see other places in lwlock.c, it only gets
counted when we try to acquire it in a loop.

3.
LWLockRelease()
{
..
/* grant permission to run, even if a spurious share lock increases
lockcount */
else if (mode == LW_EXCLUSIVE && have_waiters)
check_waiters = true;
/* nobody has this locked anymore, potential exclusive lockers get a chance
*/
else if (lockcount == 0 && have_waiters)
check_waiters = true;
..
}

It seems comments have been reversed in above code.

4.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
..
}

Shouldn't we need to use volatile variable in above loop (lock instead of
l)?

5.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &wakeup)
{
PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
LOG_LWDEBUG("LWLockRelease", l, mode, "release waiter");
dlist_delete(&waiter->lwWaitLink);
pg_write_barrier();
waiter->lwWaiting = false;
PGSemaphoreUnlock(&waiter->sem);
}
..
}

Why can't we decrement the nwaiters after waking up? I don't think
there is any major problem even if callers do that themselves, but
in some rare cases LWLockRelease() might spuriously assume that
there are some waiters and tries to call LWLockWakeup(). Although
this doesn't create any problem, keeping the value sane is good unless
there is some problem in doing so.

6.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
continue;
..
if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
{
..
wokeup_somebody = true;
}
..
}
..
}

a.
IIUC above logic, if the waiter queue is as follows:
(S-Shared; X-Exclusive) S X S S S X S S

it can skip the exclusive waiters and release shared waiter.

If my understanding is right, then I think instead of continue, there
should be *break* in above logic.

b.
Consider below sequence of waiters:
(S-Shared; X-Exclusive) S S X S S

I think as per un-patched code, it will wakeup waiters uptill (including)
first Exclusive, but patch will wake up uptill (*excluding*) first
Exclusive.

7.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
dlist_delete(&waiter->lwWaitLink);
dlist_push_tail(&wakeup, &waiter->lwWaitLink);
..
}
..
}

Use of dlist has simplified the code, but I think there might be a slight
overhead of maintaining wakeup queue as compare to un-patched
mechanism especially when there is a long waiter queue.

8.
LWLockConditionalAcquire()
{
..
/*
* We ran into an exclusive lock and might have blocked another
* exclusive lock from taking a shot because it took a time to back
* off. Retry till we are either sure we didn't block somebody (because
* somebody else certainly has the lock) or till we got it.
*
* We cannot rely on the two-step lock-acquisition protocol as in
* LWLockAcquire because we're not using it.
*/
if (potentially_spurious)
{
SPIN_DELAY();
goto retry;
}
..
}

Due to above logic, I think it can keep on retrying for long time before
it actually concludes whether it got lock or not incase other backend/'s
takes Exclusive lock after *double_check* and release before
unconditional increment of shared lock in function LWLockAttemptLock.
I understand that it might be difficult to have such a practical scenario,
however still there is a theoratical possibility of same.

Is there any advantage of retrying in LWLockConditionalAcquire()?

I think its improtant to have 2-phase LockAttempt in case of
LWLockAcquireCommon() as we have splitted the work of trying to
acquire a lock and queuing it for wait incase didn't got the lock,
but here there is no such thing, so I am wondering is there any
problem, if we just return false after failure of first attempt?

9.
LWLockAcquireOrWait()
{
..
/*
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious_first);

if (mustwait)
{
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious_second);

}

In this function, it doesn't seem to be required to use the return value
*mustwait* of second LWLockAttemptLock() call as a return value of
function, as per usage of this function if we don't get the lock at first
attempt, then it needs to check if the corresponding WAL record is flushed.
I think here we need the logic what you have used in LWLockWaitForVar()
(release the lock if we get it in second attempt).

10.
LWLockAcquireOrWait()
{
..
bool potentially_spurious_first;
bool potentially_spurious_second;
..
}

Why to use *_first and *_second in this function, can't we just have
one variable as in other LWLock.. variant functions?

11.
LWLockAcquireOrWait()
{
..
Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
}

Isn't it better to use AssertArg() rather than Assert in above usage?

12.
LWLockAcquireOrWait()
{
..
if (mustwait)
{
/*
* Wait until awakened. Like in LWLockAcquire, be prepared for bogus
* wakups, because we share the semaphore with ProcWaitForSignal.
*/
LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(l), T_ID(l), "waiting");
..
}

log for awakened is missing, it's there is current code.

13.
LWLockAcquireOrWait()
{
..
LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "suceeded");
..
}

a. spelling of "suceeded" is wrong.
b. such a log is not there in any other LWLock.. variants, if we want
to introduce it, then shouldn't it be done at other places as well.

14.
typedef struct lwlock_stats
{
+ int ex_race;
..
}

Currently I don't see the usage of this variable, is there a plan to use
it in future?

15.
/* must be greater than MAX_BACKENDS */
#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)

a. how can we guarantee it to be greater than MaxBackends,
as MaxBackends is int (the max value for which will be
equal to SHARED_LOCK_MASK)?
b. This is used only for LWLOCK_STATS, so shouldn't we
define this under LWLOCK_STATS.

16.
LWLockAcquireCommon()
{
volatile LWLock *lock = l;
..
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious);
..
lock->releaseOK = true;
..
pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
}

Shouldn't we need to use *lock* variable in LWLockAttemptLock()
function call?

Note - I have completed the review of LWLock related changes of your
overall patch in 3 different parts, as the changes are more and it makes
me understand your views behind implementation. I am maintaining
all the findings, so when you send the updated patch, I will verify by
using the same. I hope that it's not inconvenient for you.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#59Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#1)
Re: Wait free LW_SHARED acquisition

Hi,

Over at -performance Mark Kirkwood tested a recent version of this
(http://archives.postgresql.org/message-id/53B283F3.7020005%40catalyst.net.nz)
. I thought it's interesting to add the numbers to this thread:

Test: pgbench
Options: scale 500
read only
Os: Ubuntu 14.04
Pg: 9.3.4
Pg Options:
max_connections = 200
shared_buffers = 10GB
maintenance_work_mem = 1GB
effective_io_concurrency = 10
wal_buffers = 32MB
checkpoint_segments = 192
checkpoint_completion_target = 0.8

Results

Clients | 9.3 tps 32 cores | 9.3 tps 60 cores
--------+------------------+-----------------
6 | 70400 | 71028
12 | 98918 | 129140
24 | 230345 | 240631
48 | 324042 | 409510
96 | 346929 | 120464
192 | 312621 | 92663

So we have anti scaling with 60 cores as we increase the client connections.
Ouch! A level of urgency led to trying out Andres's 'rwlock' 9.4 branch [1]
- cherry picking the last 5 commits into 9.4 branch and building a package
from that and retesting:

Clients | 9.4 tps 60 cores (rwlock)
--------+--------------------------
6 | 70189
12 | 128894
24 | 233542
48 | 422754
96 | 590796
192 | 630672

Now, this is a bit of a skewed comparison due to 9.4 vs. 9.3 but still
interesting.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#60Heikki Linnakangas
hlinnakangas@vmware.com
In reply to: Andres Freund (#59)
Re: Wait free LW_SHARED acquisition

On 07/01/2014 01:08 PM, Andres Freund wrote:

Hi,

Over at -performance Mark Kirkwood tested a recent version of this
(http://archives.postgresql.org/message-id/53B283F3.7020005%40catalyst.net.nz)
. I thought it's interesting to add the numbers to this thread:

Test: pgbench
Options: scale 500
read only
Os: Ubuntu 14.04
Pg: 9.3.4
Pg Options:
max_connections = 200
shared_buffers = 10GB
maintenance_work_mem = 1GB
effective_io_concurrency = 10
wal_buffers = 32MB
checkpoint_segments = 192
checkpoint_completion_target = 0.8

Results

Clients | 9.3 tps 32 cores | 9.3 tps 60 cores
--------+------------------+-----------------
6 | 70400 | 71028
12 | 98918 | 129140
24 | 230345 | 240631
48 | 324042 | 409510
96 | 346929 | 120464
192 | 312621 | 92663

So we have anti scaling with 60 cores as we increase the client connections.
Ouch! A level of urgency led to trying out Andres's 'rwlock' 9.4 branch [1]
- cherry picking the last 5 commits into 9.4 branch and building a package
from that and retesting:

Clients | 9.4 tps 60 cores (rwlock)
--------+--------------------------
6 | 70189
12 | 128894
24 | 233542
48 | 422754
96 | 590796
192 | 630672

Now, this is a bit of a skewed comparison due to 9.4 vs. 9.3 but still
interesting.

It looks like the issue I reported here:

/messages/by-id/5190E17B.9060804@vmware.com

fixed by this commit:

http://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=b03d196be055450c7260749f17347c2d066b4254.

So, definitely need to compare plain 9.4 vs patched 9.4, not 9.3.

- Heikki

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#61Mark Kirkwood
mark.kirkwood@catalyst.net.nz
In reply to: Heikki Linnakangas (#60)
Re: Wait free LW_SHARED acquisition

On 01/07/14 23:25, Heikki Linnakangas wrote:

On 07/01/2014 01:08 PM, Andres Freund wrote:

Hi,

Over at -performance Mark Kirkwood tested a recent version of this
(http://archives.postgresql.org/message-id/53B283F3.7020005%40catalyst.net.nz)

. I thought it's interesting to add the numbers to this thread:

Test: pgbench
Options: scale 500
read only
Os: Ubuntu 14.04
Pg: 9.3.4
Pg Options:
max_connections = 200
shared_buffers = 10GB
maintenance_work_mem = 1GB
effective_io_concurrency = 10
wal_buffers = 32MB
checkpoint_segments = 192
checkpoint_completion_target = 0.8

Results

Clients | 9.3 tps 32 cores | 9.3 tps 60 cores
--------+------------------+-----------------
6 | 70400 | 71028
12 | 98918 | 129140
24 | 230345 | 240631
48 | 324042 | 409510
96 | 346929 | 120464
192 | 312621 | 92663

So we have anti scaling with 60 cores as we increase the client
connections.
Ouch! A level of urgency led to trying out Andres's 'rwlock' 9.4
branch [1]
- cherry picking the last 5 commits into 9.4 branch and building a
package
from that and retesting:

Clients | 9.4 tps 60 cores (rwlock)
--------+--------------------------
6 | 70189
12 | 128894
24 | 233542
48 | 422754
96 | 590796
192 | 630672

Now, this is a bit of a skewed comparison due to 9.4 vs. 9.3 but still
interesting.

It looks like the issue I reported here:

/messages/by-id/5190E17B.9060804@vmware.com

fixed by this commit:

http://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=b03d196be055450c7260749f17347c2d066b4254.

So, definitely need to compare plain 9.4 vs patched 9.4, not 9.3.

Here's plain 9.4 vs patched 9.4:

Clients | 9.4 tps 60 cores | 9.4 tps 60 cores (rwlock)
--------+------------------+--------------------------
6 | 69490 | 70189
12 | 128200 | 128894
24 | 232243 | 233542
48 | 417689 | 422754
96 | 464037 | 590796
192 | 418252 | 630672

It appears that plain 9.4 does not exhibit the dramatic anti scaling
that 9.3 showed, but there is still evidence of some contention in the
higher client numbers, and we peak at the 96 client mark. The patched
variant looks pretty much free from this, still scaling at 192
connections (might have been interesting to try more, but had
max_connections set to 200)!

Cheers

Mark

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#62Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#58)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

2.
LWLockWakeup()
{
..
#ifdef LWLOCK_STATS
lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
#else
SpinLockAcquire(&lock->mutex);
#endif
..
}

Earlier while releasing lock, we don't count it towards LWLock stats
spin_delay_count. I think if we see other places in lwlock.c, it only gets
counted when we try to acquire it in a loop.

I think the previous situation was clearly suboptimal. I've now modified
things so all spinlock acquirations are counted.

3.
LWLockRelease()
{
..
/* grant permission to run, even if a spurious share lock increases
lockcount */
else if (mode == LW_EXCLUSIVE && have_waiters)
check_waiters = true;
/* nobody has this locked anymore, potential exclusive lockers get a chance
*/
else if (lockcount == 0 && have_waiters)
check_waiters = true;
..
}

It seems comments have been reversed in above code.

No, they look right. But I've expanded them in the version I'm going to
post in a couple minutes.

5.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &wakeup)
{
PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
LOG_LWDEBUG("LWLockRelease", l, mode, "release waiter");
dlist_delete(&waiter->lwWaitLink);
pg_write_barrier();
waiter->lwWaiting = false;
PGSemaphoreUnlock(&waiter->sem);
}
..
}

Why can't we decrement the nwaiters after waking up? I don't think
there is any major problem even if callers do that themselves, but
in some rare cases LWLockRelease() might spuriously assume that
there are some waiters and tries to call LWLockWakeup(). Although
this doesn't create any problem, keeping the value sane is good unless
there is some problem in doing so.

That won't work because then LWLockWakeup() wouldn't be called when
necessary - precisely because nwaiters is 0.

6.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
continue;
..
if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
{
..
wokeup_somebody = true;
}
..
}
..
}

a.
IIUC above logic, if the waiter queue is as follows:
(S-Shared; X-Exclusive) S X S S S X S S

it can skip the exclusive waiters and release shared waiter.

If my understanding is right, then I think instead of continue, there
should be *break* in above logic.

No, it looks correct to me. What happened is that the first S was woken
up. So there's no point in waking up an exclusive locker, but further
non-exclusive lockers can be woken up.

b.
Consider below sequence of waiters:
(S-Shared; X-Exclusive) S S X S S

I think as per un-patched code, it will wakeup waiters uptill (including)
first Exclusive, but patch will wake up uptill (*excluding*) first
Exclusive.

I don't think the current code does that. And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time it'd
make sense for X to be woken up is when it gets run faster than the S
processes.

7.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
dlist_delete(&waiter->lwWaitLink);
dlist_push_tail(&wakeup, &waiter->lwWaitLink);
..
}
..
}

Use of dlist has simplified the code, but I think there might be a slight
overhead of maintaining wakeup queue as compare to un-patched
mechanism especially when there is a long waiter queue.

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.
And the code is *so* much more readable.

8.
LWLockConditionalAcquire()
{
..
/*
* We ran into an exclusive lock and might have blocked another
* exclusive lock from taking a shot because it took a time to back
* off. Retry till we are either sure we didn't block somebody (because
* somebody else certainly has the lock) or till we got it.
*
* We cannot rely on the two-step lock-acquisition protocol as in
* LWLockAcquire because we're not using it.
*/
if (potentially_spurious)
{
SPIN_DELAY();
goto retry;
}
..
}

Due to above logic, I think it can keep on retrying for long time before
it actually concludes whether it got lock or not incase other backend/'s
takes Exclusive lock after *double_check* and release before
unconditional increment of shared lock in function LWLockAttemptLock.
I understand that it might be difficult to have such a practical scenario,
however still there is a theoratical possibility of same.

I'm not particularly concerned. We could optimize it a bit, but I really
don't think it's necessary.

Is there any advantage of retrying in LWLockConditionalAcquire()?

It's required for correctness. We only retry if we potentially blocked
an exclusive acquirer (by spuriously incrementing/decrementing lockcount
with 1). We need to be sure to either get the lock (in which case we can
wake up the waiter on release), or be sure that we didn't disturb
anyone.

9.
LWLockAcquireOrWait()
{
..
/*
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious_first);

if (mustwait)
{
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

mustwait = LWLockAttemptLock(l, mode, false, &potentially_spurious_second);

}

In this function, it doesn't seem to be required to use the return value
*mustwait* of second LWLockAttemptLock() call as a return value of
function, as per usage of this function if we don't get the lock at first
attempt, then it needs to check if the corresponding WAL record is
flushed.

I don't think that's the appropriate comparison. Acquiring the lock in
the lock in the new implementation essentially consists out of these two
steps. We *DID* get the lock here. Without sleeping. So returning the
appropriate return code is correct.
In fact, returning false would break things, because the caller would
hold the lock without freeing it again?

11.
LWLockAcquireOrWait()
{
..
Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
}

Isn't it better to use AssertArg() rather than Assert in above usage?

I've changed that. But I have to admit, I don't really see the point of
AssertArg(). If it'd output the value, then it'd be beneficial, but it
doesn't.

15.
/* must be greater than MAX_BACKENDS */
#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)

a. how can we guarantee it to be greater than MaxBackends,
as MaxBackends is int (the max value for which will be
equal to SHARED_LOCK_MASK)?

MaxBackends luckily is limited to something lower. I've added a comment
to that regard.

b. This is used only for LWLOCK_STATS, so shouldn't we
define this under LWLOCK_STATS.

It's a general value, so I don't think that's appropriate.

Thanks for the review!

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#63Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#62)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-08 14:47:44 +0200, Andres Freund wrote:

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

5.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &wakeup)
{
PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
LOG_LWDEBUG("LWLockRelease", l, mode, "release waiter");
dlist_delete(&waiter->lwWaitLink);
pg_write_barrier();
waiter->lwWaiting = false;
PGSemaphoreUnlock(&waiter->sem);
}
..
}

Why can't we decrement the nwaiters after waking up? I don't think
there is any major problem even if callers do that themselves, but
in some rare cases LWLockRelease() might spuriously assume that
there are some waiters and tries to call LWLockWakeup(). Although
this doesn't create any problem, keeping the value sane is good unless
there is some problem in doing so.

That won't work because then LWLockWakeup() wouldn't be called when
necessary - precisely because nwaiters is 0.

Err, this is bogus. Memory fail.

The reason I've done so is that it's otherwise much harder to debug
issues where there are backend that have been woken up already, but
haven't rerun yet. Without this there's simply no evidence of that
state. I can't see this being relevant for performance, so I'd rather
have it stay that way.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#64Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#1)
2 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.9

Hi,

Attached you can find the next version of my LW_SHARED patchset. Now
that atomics are committed, it seems like a good idea to also add their
raison d'�tre.

Since the last public version I have:
* Addressed lots of Amit's comments. Thanks!
* Peformed a fair amount of testing.
* Rebased the code. The volatile removal made that not entirely
trivial...
* Significantly cleaned up and simplified the code.
* Updated comments and such
* Fixed a minor bug (unpaired HOLD/RESUME_INTERRUPTS in a corner case)

The feature currently consists out of two patches:
1) Convert PGPROC->lwWaitLink into a dlist. The old code was frail and
verbose. This also does:
* changes the logic in LWLockRelease() to release all shared lockers
when waking up any. This can yield some significant performance
improvements - and the fairness isn't really much worse than
before,
as we always allowed new shared lockers to jump the queue.

* adds a memory pg_write_barrier() in the wakeup paths between
dequeuing and unsetting ->lwWaiting. That was always required on
weakly ordered machines, but f4077cda2 made it more urgent. I can
reproduce crashes without it.
2) Implement the wait free LW_SHARED algorithm.

Personally I'm quite happy with the new state. I think it needs more
review, but I personally don't know of anything that needs
changing. There's lots of further improvements that could be done, but
let's get this in first.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-Convert-the-PGPROC-lwWaitLink-list-into-a-dlist-inst.patchtext/x-patch; charset=us-asciiDownload
>From 6885a15cc6f2e193ff575a4463d90ad252d74f5e Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Tue, 7 Oct 2014 15:32:34 +0200
Subject: [PATCH 1/2] Convert the PGPROC->lwWaitLink list into a dlist instead
 of open coding it.

Besides being shorter and much easier to read it:

* changes the logic in LWLockRelease() to release all shared lockers
  when waking up any. This can yield some significant performance
  improvements - and the fairness isn't really much worse than before,
  as we always allowed new shared lockers to jump the queue.

* adds a memory pg_write_barrier() in the wakeup paths between
  dequeuing and unsetting ->lwWaiting. That was always required on
  weakly ordered machines, but f4077cda2 made it more urgent.

Author: Andres Freund
---
 src/backend/access/transam/twophase.c |   1 -
 src/backend/storage/lmgr/lwlock.c     | 151 +++++++++++++---------------------
 src/backend/storage/lmgr/proc.c       |   2 -
 src/include/storage/lwlock.h          |   5 +-
 src/include/storage/proc.h            |   3 +-
 5 files changed, 60 insertions(+), 102 deletions(-)

diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index d5409a6..6401943 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -389,7 +389,6 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 	proc->roleId = owner;
 	proc->lwWaiting = false;
 	proc->lwWaitMode = 0;
-	proc->lwWaitLink = NULL;
 	proc->waitLock = NULL;
 	proc->waitProcLock = NULL;
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 9fe6855..e6f9158 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -35,6 +35,7 @@
 #include "miscadmin.h"
 #include "pg_trace.h"
 #include "replication/slot.h"
+#include "storage/barrier.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
@@ -115,9 +116,9 @@ inline static void
 PRINT_LWDEBUG(const char *where, const LWLock *lock)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): excl %d shared %d head %p rOK %d",
+		elog(LOG, "%s(%s %d): excl %d shared %d rOK %d",
 			 where, T_NAME(lock), T_ID(lock),
-			 (int) lock->exclusive, lock->shared, lock->head,
+			 (int) lock->exclusive, lock->shared,
 			 (int) lock->releaseOK);
 }
 
@@ -475,8 +476,7 @@ LWLockInitialize(LWLock *lock, int tranche_id)
 	lock->exclusive = 0;
 	lock->shared = 0;
 	lock->tranche = tranche_id;
-	lock->head = NULL;
-	lock->tail = NULL;
+	dlist_init(&lock->waiters);
 }
 
 
@@ -615,12 +615,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = mode;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -836,12 +831,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -997,13 +987,8 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		 */
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-
 		/* waiters are added to the front of the queue */
-		proc->lwWaitLink = lock->head;
-		if (lock->head == NULL)
-			lock->tail = proc;
-		lock->head = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -1079,9 +1064,10 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 void
 LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 {
-	PGPROC	   *head;
-	PGPROC	   *proc;
-	PGPROC	   *next;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+
+	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
 	SpinLockAcquire(&lock->mutex);
@@ -1096,24 +1082,16 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	 * See if there are any LW_WAIT_UNTIL_FREE waiters that need to be woken
 	 * up. They are always in the front of the queue.
 	 */
-	head = lock->head;
-
-	if (head != NULL && head->lwWaitMode == LW_WAIT_UNTIL_FREE)
+	dlist_foreach_modify(iter, &lock->waiters)
 	{
-		proc = head;
-		next = proc->lwWaitLink;
-		while (next && next->lwWaitMode == LW_WAIT_UNTIL_FREE)
-		{
-			proc = next;
-			next = next->lwWaitLink;
-		}
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-		/* proc is now the last PGPROC to be released */
-		lock->head = next;
-		proc->lwWaitLink = NULL;
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+			break;
+
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 	}
-	else
-		head = NULL;
 
 	/* We are done updating shared state of the lock itself. */
 	SpinLockRelease(&lock->mutex);
@@ -1121,13 +1099,13 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, &wakeup)
 	{
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 }
 
@@ -1138,10 +1116,12 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 void
 LWLockRelease(LWLock *lock)
 {
-	PGPROC	   *head;
-	PGPROC	   *proc;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
 	int			i;
 
+	dlist_init(&wakeup);
+
 	PRINT_LWDEBUG("LWLockRelease", lock);
 
 	/*
@@ -1177,58 +1157,39 @@ LWLockRelease(LWLock *lock)
 	 * if someone has already awakened waiters that haven't yet acquired the
 	 * lock.
 	 */
-	head = lock->head;
-	if (head != NULL)
+	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
 	{
-		if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
+		/*
+		 * Remove the to-be-awakened PGPROCs from the queue.
+		 */
+		bool		releaseOK = true;
+		bool		wokeup_somebody = false;
+
+		dlist_foreach_modify(iter, &lock->waiters)
 		{
-			/*
-			 * Remove the to-be-awakened PGPROCs from the queue.
-			 */
-			bool		releaseOK = true;
+			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-			proc = head;
+			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+				continue;
 
-			/*
-			 * First wake up any backends that want to be woken up without
-			 * acquiring the lock.
-			 */
-			while (proc->lwWaitMode == LW_WAIT_UNTIL_FREE && proc->lwWaitLink)
-				proc = proc->lwWaitLink;
+			dlist_delete(&waiter->lwWaitLink);
+			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 
 			/*
-			 * If the front waiter wants exclusive lock, awaken him only.
-			 * Otherwise awaken as many waiters as want shared access.
+			 * Prevent additional wakeups until retryer gets to
+			 * run. Backends that are just waiting for the lock to become
+			 * free don't retry automatically.
 			 */
-			if (proc->lwWaitMode != LW_EXCLUSIVE)
+			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
 			{
-				while (proc->lwWaitLink != NULL &&
-					   proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
-				{
-					if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
-						releaseOK = false;
-					proc = proc->lwWaitLink;
-				}
-			}
-			/* proc is now the last PGPROC to be released */
-			lock->head = proc->lwWaitLink;
-			proc->lwWaitLink = NULL;
-
-			/*
-			 * Prevent additional wakeups until retryer gets to run. Backends
-			 * that are just waiting for the lock to become free don't retry
-			 * automatically.
-			 */
-			if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
 				releaseOK = false;
+				wokeup_somebody = true;
+			}
 
-			lock->releaseOK = releaseOK;
-		}
-		else
-		{
-			/* lock is still held, can't awaken anything */
-			head = NULL;
+			if(waiter->lwWaitMode == LW_EXCLUSIVE)
+				break;
 		}
+		lock->releaseOK = releaseOK;
 	}
 
 	/* We are done updating shared state of the lock itself. */
@@ -1239,15 +1200,15 @@ LWLockRelease(LWLock *lock)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, &wakeup)
 	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 		LOG_LWDEBUG("LWLockRelease", T_NAME(lock), T_ID(lock),
 					"release waiter");
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 
 	/*
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index ea88a24..a4789fc 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -372,7 +372,6 @@ InitProcess(void)
 		MyPgXact->vacuumFlags |= PROC_IS_AUTOVACUUM;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
@@ -535,7 +534,6 @@ InitAuxiliaryProcess(void)
 	MyPgXact->vacuumFlags = 0;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 02c8f1a..fea5d33 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -14,6 +14,7 @@
 #ifndef LWLOCK_H
 #define LWLOCK_H
 
+#include "lib/ilist.h"
 #include "storage/s_lock.h"
 
 struct PGPROC;
@@ -50,9 +51,7 @@ typedef struct LWLock
 	char		exclusive;		/* # of exclusive holders (0 or 1) */
 	int			shared;			/* # of shared holders (0..MaxBackends) */
 	int			tranche;		/* tranche ID */
-	struct PGPROC *head;		/* head of list of waiting PGPROCs */
-	struct PGPROC *tail;		/* tail of list of waiting PGPROCs */
-	/* tail is undefined when head is NULL */
+	dlist_head	waiters;		/* list of waiting PGPROCs */
 } LWLock;
 
 /*
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index c23f4da..38758d3 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -15,6 +15,7 @@
 #define _PROC_H_
 
 #include "access/xlogdefs.h"
+#include "lib/ilist.h"
 #include "storage/latch.h"
 #include "storage/lock.h"
 #include "storage/pg_sema.h"
@@ -104,7 +105,7 @@ struct PGPROC
 	/* Info about LWLock the process is currently waiting for, if any. */
 	bool		lwWaiting;		/* true if waiting for an LW lock */
 	uint8		lwWaitMode;		/* lwlock mode being waited for */
-	struct PGPROC *lwWaitLink;	/* next waiter for same LW lock */
+	dlist_node	lwWaitLink;		/* position in LW lock wait list */
 
 	/* Info about lock the process is currently waiting for, if any. */
 	/* waitLock and waitProcLock are NULL if not currently waiting. */
-- 
1.8.3.251.g1462b67

0002-Wait-free-LW_SHARED-LWLock-acquisition.patchtext/x-patch; charset=us-asciiDownload
>From cbd80574f5dfe631f492dd497700dd05b211b3e3 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Thu, 18 Sep 2014 16:14:16 +0200
Subject: [PATCH 2/2] Wait free LW_SHARED LWLock acquisition.

The old LWLock implementation had the problem that concurrent shared
lock acquisitions required exclusively acquiring a spinlock. Often
that could lead to acquirers waiting behind the spinlock, even if the
actual LWLock was free.

The new implementation doesn't acquire the spinlock when acquiring the
lock itself. Instead the new atomic operations are used to atomically
manipulate the state. Only the waitqueue, only used in the slow path,
is still protected by the spinlock. Check lwlock.c's header for an
explanation about the used algorithm.

For some common workloads on larger machines this can yield
significant performance improvements. Particularly in read mostly
workloads.

Reviewed-By: Amit Kapila
Author: Andres Freund

Discussion: 20130926225545.GB26663@awork2.anarazel.de
---
 src/backend/storage/lmgr/lwlock.c | 968 +++++++++++++++++++++++++++-----------
 src/include/storage/lwlock.h      |   9 +-
 2 files changed, 704 insertions(+), 273 deletions(-)

diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index e6f9158..b10c121 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -24,6 +24,78 @@
  * IDENTIFICATION
  *	  src/backend/storage/lmgr/lwlock.c
  *
+ * NOTES:
+ *
+ * This used to be a pretty straight forward reader-writer lock
+ * implementation, in which the internal state was protected by a
+ * spinlock. Unfortunately the overhead of taking the spinlock proved to be
+ * too high for workloads/locks that were locked in shared mode very
+ * frequently. Often we were spinning in the (obviously exlusive) spinlock,
+ * while trying to acquire a shared lock that was actually free.
+ *
+ * Thus a new implementation was devised that provides wait-free shared lock
+ * acquiration for locks that aren't exclusively locked.
+ *
+ * The basic idea is to have a single atomic variable 'lockcount' instead of
+ * the formerly separate shared and exclusive counters and to use an atomic
+ * increment to acquire the lock. That's fairly easy to do for rw-spinlocks,
+ * but a lot harder for something like LWLocks that want to wait in the OS.
+ *
+ * For exlusive lock acquisition we use an atomic compare-and-exchange on the
+ * lockcount variable swapping in EXCLUSIVE_LOCK/1<<31-1/0x7FFFFFFF if and only
+ * if the current value of lockcount is 0. If the swap was not successfull, we
+ * have to wait.
+ *
+ * For shared lock acquisition we use an atomic add to the lockcount variable
+ * to add 1. If the value is bigger than EXCLUSIVE_LOCK we know that somebody
+ * actually has an exclusive lock, and we back out by atomically decrementing
+ * by 1 again. If so, we have to wait for the exlusive locker to release the
+ * lock.
+ *
+ * To release the lock we use an atomic decrement to release the lock. If the
+ * new value is zero (we get that atomically), we know we have to release
+ * waiters.
+ *
+ * The attentive reader probably might have noticed that naively doing the
+ * above has two glaring race conditions:
+ *
+ * 1) too-quick-for-queueing: We try to lock using the atomic operations and
+ * notice that we have to wait. Unfortunately until we have finished queuing,
+ * the former locker very well might have already finished it's work. That's
+ * problematic because we're now stuck waiting inside the OS.
+ *
+ * 2) spurious failed locks: Due to the logic of backing out of shared
+ * locks after we unconditionally added a 1 to lockcount, we might have
+ * prevented another exclusive locker from getting the lock:
+ *   1) Session A: LWLockAcquire(LW_EXCLUSIVE) - success
+ *   2) Session B: LWLockAcquire(LW_SHARED) - lockcount += 1
+ *   3) Session B: LWLockAcquire(LW_SHARED) - oops, bigger than EXCLUSIVE_LOCK
+ *   4) Session A: LWLockRelease()
+ *   5) Session C: LWLockAcquire(LW_EXCLUSIVE) - check if lockcount = 0, no. wait.
+ *   6) Session B: LWLockAcquire(LW_SHARED) - lockcount -= 1
+ *   7) Session B: LWLockAcquire(LW_SHARED) - wait
+ *
+ * So we'd now have both B) and C) waiting on a lock that nobody is holding
+ * anymore. Not good.
+ *
+ * To mitigate those races we use a two phased attempt at locking:
+ *   Phase 1: Try to do it atomically, if we succeed, nice
+ *   Phase 2: Add us too the waitqueue of the lock
+ *   Phase 3: Try to grab the lock again, if we succeed, remove ourselves from
+ *            the queue
+ *   Phase 4: Sleep till wakeup, goto Phase 1
+ *
+ * This protects us against both problems from above:
+ * 1) Nobody can release too quick, before we're queued, since after Phase 2
+ *    we're already queued.
+ * 2) If somebody spuriously got blocked from acquiring the lock, they will
+ *    get queued in Phase 2 and we can wake them up if neccessary or they will
+ *    have gotten the lock in Phase 3.
+ *
+ * There above algorithm only works for LWLockAcquire, not directly for
+ * LWLockAcquireConditional where we don't want to wait. In that case we just
+ * need to retry acquiring the lock until we're sure we didn't disturb anybody
+ * in doing so.
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -35,7 +107,6 @@
 #include "miscadmin.h"
 #include "pg_trace.h"
 #include "replication/slot.h"
-#include "storage/barrier.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
@@ -50,6 +121,11 @@
 /* We use the ShmemLock spinlock to protect LWLockAssign */
 extern slock_t *ShmemLock;
 
+#define EXCLUSIVE_LOCK (((uint32) 1) << (31 - 1))
+
+/* Must be greater than MAX_BACKENDS - which is 2^23-1, so we're fine. */
+#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)
+
 /*
  * This is indexed by tranche ID and stores metadata for all tranches known
  * to the current backend.
@@ -80,8 +156,14 @@ static LWLockTranche MainLWLockTranche;
  */
 #define MAX_SIMUL_LWLOCKS	200
 
+typedef struct LWLockHandle
+{
+	LWLock *lock;
+	LWLockMode	mode;
+} LWLockHandle;
+
 static int	num_held_lwlocks = 0;
-static LWLock *held_lwlocks[MAX_SIMUL_LWLOCKS];
+static LWLockHandle held_lwlocks[MAX_SIMUL_LWLOCKS];
 
 static int	lock_addin_request = 0;
 static bool lock_addin_request_allowed = true;
@@ -100,8 +182,10 @@ typedef struct lwlock_stats
 {
 	lwlock_stats_key key;
 	int			sh_acquire_count;
+	int			sh_attempt_backout;
 	int			ex_acquire_count;
 	int			block_count;
+	int			dequeue_self_count;
 	int			spin_delay_count;
 }	lwlock_stats;
 
@@ -113,24 +197,32 @@ static lwlock_stats lwlock_stats_dummy;
 bool		Trace_lwlocks = false;
 
 inline static void
-PRINT_LWDEBUG(const char *where, const LWLock *lock)
+PRINT_LWDEBUG(const char *where, LWLock *lock, LWLockMode mode)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): excl %d shared %d rOK %d",
+	{
+		uint32 lockcount = pg_atomic_read_u32(&lock->lockcount);
+
+		elog(LOG, "%d: %s(%s %d): excl %u shared %u waiters %u rOK %d\n",
+			 MyProcPid,
 			 where, T_NAME(lock), T_ID(lock),
-			 (int) lock->exclusive, lock->shared,
+			 lockcount >= EXCLUSIVE_LOCK,
+			 lockcount & SHARED_LOCK_MASK,
+			 pg_atomic_read_u32(&lock->nwaiters),
 			 (int) lock->releaseOK);
+	}
 }
 
 inline static void
-LOG_LWDEBUG(const char *where, const char *name, int index, const char *msg)
+LOG_LWDEBUG(const char *where, LWLock *lock, const char *msg)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): %s", where, name, index, msg);
+		elog(LOG, "%s(%s %d): %s", where, T_NAME(lock), T_ID(lock), msg);
 }
+
 #else							/* not LOCK_DEBUG */
-#define PRINT_LWDEBUG(a,b)
-#define LOG_LWDEBUG(a,b,c,d)
+#define PRINT_LWDEBUG(a,b,c)
+#define LOG_LWDEBUG(a,b,c)
 #endif   /* LOCK_DEBUG */
 
 #ifdef LWLOCK_STATS
@@ -192,11 +284,12 @@ print_lwlock_stats(int code, Datum arg)
 	while ((lwstats = (lwlock_stats *) hash_seq_search(&scan)) != NULL)
 	{
 		fprintf(stderr,
-			  "PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u\n",
+				"PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u backout %u dequeue self %u\n",
 				MyProcPid, LWLockTrancheArray[lwstats->key.tranche]->name,
 				lwstats->key.instance, lwstats->sh_acquire_count,
 				lwstats->ex_acquire_count, lwstats->block_count,
-				lwstats->spin_delay_count);
+				lwstats->spin_delay_count, lwstats->sh_attempt_backout,
+				lwstats->dequeue_self_count);
 	}
 
 	LWLockRelease(&MainLWLockArray[0].lock);
@@ -224,8 +317,10 @@ get_lwlock_stats_entry(LWLock *lock)
 	if (!found)
 	{
 		lwstats->sh_acquire_count = 0;
+		lwstats->sh_attempt_backout = 0;
 		lwstats->ex_acquire_count = 0;
 		lwstats->block_count = 0;
+		lwstats->dequeue_self_count = 0;
 		lwstats->spin_delay_count = 0;
 	}
 	return lwstats;
@@ -473,12 +568,309 @@ LWLockInitialize(LWLock *lock, int tranche_id)
 {
 	SpinLockInit(&lock->mutex);
 	lock->releaseOK = true;
-	lock->exclusive = 0;
-	lock->shared = 0;
+	pg_atomic_init_u32(&lock->lockcount, 0);
+	pg_atomic_init_u32(&lock->nwaiters, 0);
 	lock->tranche = tranche_id;
 	dlist_init(&lock->waiters);
 }
 
+/*
+ * Internal function that tries to atomically acquire the lwlock in the passed
+ * in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * callers job.
+ *
+ * Returns true if the lock isn't free and we need to wait.
+ *
+ * When acquiring shared locks it's possible that we disturb an exclusive
+ * waiter. If that's a problem for the specific user, pass in a valid pointer
+ * for 'potentially_spurious'. Its value will be set to true if we possibly
+ * did so. The caller then has to handle that scenario.
+ */
+static bool
+LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)
+{
+	uint32		oldstate;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	AssertArg(mode == LW_EXCLUSIVE || mode == LW_SHARED);
+
+	if (potentially_spurious != NULL)
+		*potentially_spurious = false;
+
+	if (mode == LW_EXCLUSIVE)
+	{
+		uint32 expected;
+
+		/*
+		 * First check whether the variable is free without a atomic
+		 * operation; it's often quite a bit cheaper for contended
+		 * locks. Doing so can cause a superflous shared-exclusive cacheline
+		 * transition, but benchmarks show that it's still worth doing so.
+		 */
+		expected = pg_atomic_read_u32(&lock->lockcount);
+
+		if (expected != 0)
+			return true;
+		else if (!pg_atomic_compare_exchange_u32(&lock->lockcount,
+												 &expected, EXCLUSIVE_LOCK))
+		{
+			/*
+			 * No can do. Between the pg_atomic_read() above and the CAS
+			 * somebody else acquired the lock.
+			 */
+			return true;
+		}
+		else
+		{
+			/* yipeyyahee */
+#ifdef LOCK_DEBUG
+			lock->owner = MyProc;
+#endif
+			Assert(expected == 0);
+			return false;
+		}
+	}
+	else
+	{
+		/*
+		 * If the caller is interested in spurious locks, do an unlocked check
+		 * first.  This is useful if potentially spurious results have a
+		 * noticeable cost.
+		 */
+		if (potentially_spurious != NULL &&
+			pg_atomic_read_u32(&lock->lockcount) >= EXCLUSIVE_LOCK)
+			return true;
+
+		/*
+		 * Acquire the share lock unconditionally using an atomic addition. We
+		 * might have to back out again if it turns out somebody else has an
+		 * exclusive lock.
+		 */
+		oldstate = pg_atomic_fetch_add_u32(&lock->lockcount, 1);
+
+		if (oldstate >= EXCLUSIVE_LOCK)
+		{
+			/*
+			 * Ok, somebody else holds the lock exclusively. We need to back
+			 * away from the shared lock, since we don't actually hold it right
+			 * now.  Since there's a window between lockcount += 1 and lockcount
+			 * -= 1, the previous exclusive locker could have released and
+			 * another exclusive locker could have seen our +1. We need to
+			 * signal that to the upper layers so they can deal with the race
+			 * condition.
+			 */
+
+			/*
+			 * XXX: We could check the return value if (double_check), it's
+			 * not spurious if still exclusively locked. Should work for the
+			 * current callers. There might be some cases where ABA like
+			 * problems exist.
+			 */
+			pg_atomic_fetch_sub_u32(&lock->lockcount, 1);
+
+
+			if (potentially_spurious != NULL)
+				*potentially_spurious = true;
+#ifdef LWLOCK_STATS
+			lwstats->sh_attempt_backout++;
+#endif
+			return true;
+		}
+		else
+		{
+			/* yipeyyahee */
+			return false;
+		}
+	}
+
+	pg_unreachable();
+}
+
+/*
+ * Wakeup all the lockers that currently have a chance to run.
+ */
+static void
+LWLockWakeup(LWLock *lock, LWLockMode released_mode)
+{
+	bool		releaseOK;
+	bool		wokeup_somebody = false;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	dlist_init(&wakeup);
+
+	/* remove the to-be-awakened PGPROCs from the queue */
+	releaseOK = true;
+
+	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/*
+	 * We're still waiting for backends to get scheduled, don't wake them up
+	 * again.
+	 */
+	if (!lock->releaseOK)
+	{
+		SpinLockRelease(&lock->mutex);
+		LOG_LWDEBUG("LWLockRelease", lock, "skipping due to releaseOK");
+		return;
+	}
+
+	dlist_foreach_modify(iter, &lock->waiters)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+			continue;
+
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
+
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+		{
+			/*
+			 * Prevent additional wakeups until retryer gets to run. Backends
+			 * that are just waiting for the lock to become free don't retry
+			 * automatically.
+			 */
+			releaseOK = false;
+			/*
+			 * Don't wakeup (further) exclusive locks.
+			 */
+			wokeup_somebody = true;
+		}
+
+		/*
+		 * Once we've woken up an exclusive lock, there's no point in waking
+		 * up anybody else.
+		 */
+		if(waiter->lwWaitMode == LW_EXCLUSIVE)
+			break;
+	}
+	lock->releaseOK = releaseOK;
+
+
+	/* We are done updating shared state of the lock queue. */
+	SpinLockRelease(&lock->mutex);
+
+	/*
+	 * Awaken any waiters I removed from the queue.
+	 */
+	dlist_foreach_modify(iter, &wakeup)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		LOG_LWDEBUG("LWLockRelease", lock, "release waiter");
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
+	}
+}
+
+/*
+ * Add ourselves to the end of the queue. Mode can be LW_WAIT_UNTIL_FREE here!
+ */
+static void
+LWLockQueueSelf(LWLock *lock, LWLockMode mode)
+{
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	/*
+	 * If we don't have a PGPROC structure, there's no way to wait. This
+	 * should never occur, since MyProc should only be null during shared
+	 * memory initialization.
+	 */
+	if (MyProc == NULL)
+		elog(PANIC, "cannot wait without a PGPROC structure");
+
+	pg_atomic_fetch_add_u32(&lock->nwaiters, 1);
+
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	if (MyProc->lwWaiting)
+		elog(PANIC, "queueing for lock while waiting on another one");
+
+	MyProc->lwWaiting = true;
+	MyProc->lwWaitMode = mode;
+
+	/* LW_WAIT_UNTIL_FREE waiters are always at the front of the queue */
+	if (mode == LW_WAIT_UNTIL_FREE)
+		dlist_push_head(&lock->waiters, &MyProc->lwWaitLink);
+	else
+		dlist_push_tail(&lock->waiters, &MyProc->lwWaitLink);
+
+	/* Can release the mutex now */
+	SpinLockRelease(&lock->mutex);
+}
+
+/*
+ * Remove ourselves from the waitlist.  This is used if we queued ourselves
+ * because we thought we needed to sleep but, after further checking, we
+ * discovered that we don't actually need to do so. Somebody else might have
+ * already woken us up though, in that case return false.
+ */
+static bool
+LWLockDequeueSelf(LWLock *lock)
+{
+	bool	found = false;
+	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/* need to iterate, somebody else could have unqueued us */
+	dlist_foreach_modify(iter, &lock->waiters)
+	{
+		PGPROC *proc = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		if (proc == MyProc)
+		{
+			found = true;
+			dlist_delete(&proc->lwWaitLink);
+			break;
+		}
+	}
+
+	/* clear waiting state again, nice for debugging */
+	if (found)
+		MyProc->lwWaiting = false;
+
+	SpinLockRelease(&lock->mutex);
+
+	pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+	return found;
+}
 
 /*
  * LWLockAcquire - acquire a lightweight lock in the specified mode
@@ -510,18 +902,19 @@ static inline bool
 LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 {
 	PGPROC	   *proc = MyProc;
-	bool		retry = false;
 	bool		result = true;
 	int			extraWaits = 0;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
 #endif
 
-	PRINT_LWDEBUG("LWLockAcquire", lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
 
-#ifdef LWLOCK_STATS
-	lwstats = get_lwlock_stats_entry(lock);
+	PRINT_LWDEBUG("LWLockAcquire", lock, mode);
 
+#ifdef LWLOCK_STATS
 	/* Count lock acquisition attempts */
 	if (mode == LW_EXCLUSIVE)
 		lwstats->ex_acquire_count++;
@@ -567,58 +960,78 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 	{
 		bool		mustwait;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
-#ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
-#else
-		SpinLockAcquire(&lock->mutex);
-#endif
-
-		/* If retrying, allow LWLockRelease to release waiters again */
-		if (retry)
-			lock->releaseOK = true;
+		/*
+		 * try to grab the lock the first time, we're not in the waitqueue yet.
+		 */
+		mustwait = LWLockAttemptLock(lock, mode, NULL);
 
-		/* If I can get the lock, do so quickly. */
-		if (mode == LW_EXCLUSIVE)
+		if (!mustwait)
 		{
-			if (lock->exclusive == 0 && lock->shared == 0)
-			{
-				lock->exclusive++;
-				mustwait = false;
-			}
-			else
-				mustwait = true;
+			/* XXX: remove before commit? */
+			LOG_LWDEBUG("LWLockAcquire", lock, "immediately acquired lock");
+			break;				/* got the lock */
 		}
-		else
+
+		/*
+		 * Ok, at this point we couldn't grab the lock on the first try. We
+		 * cannot simply queue ourselves to the end of the list and wait to be
+		 * woken up because by now the lock could long have been released.
+		 * Instead add us to the queue and try to grab the lock again. If we
+		 * succeed we need to revert the queuing and be happy, otherwise we
+		 * recheck the lock. If we still couldn't grab it, we know that the
+		 * other lock will see our queue entries when releasing since they
+		 * existed before we checked for the lock.
+		 */
+
+		/* add to the queue */
+		LWLockQueueSelf(lock, mode);
+
+		/* we're now guaranteed to be woken up if necessary */
+		mustwait = LWLockAttemptLock(lock, mode, NULL);
+
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
 		{
-			if (lock->exclusive == 0)
+			LOG_LWDEBUG("LWLockAcquire", lock, "acquired, undoing queue");
+
+#ifdef LWLOCK_STATS
+			lwstats->dequeue_self_count++;
+#endif
+			if (!LWLockDequeueSelf(lock))
 			{
-				lock->shared++;
-				mustwait = false;
+				/*
+				 * Somebody else dequeued us and has or will wake us up. Wait
+				 * for the correct wakeup, otherwise our ->lwWaiting would get
+				 * reset at some inconvenient point later, and releaseOk
+				 * wouldn't be managed correctly.
+				 */
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+				/*
+				 * Reset releaseOk - if somebody woke us they'll have set it
+				 * to false.
+				 */
+#ifdef LWLOCK_STATS
+				lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+				SpinLockAcquire(&lock->mutex);
+#endif
+				lock->releaseOK = true;
+				SpinLockRelease(&lock->mutex);
 			}
-			else
-				mustwait = true;
+			break;
 		}
 
-		if (!mustwait)
-			break;				/* got the lock */
-
 		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait. This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
+		 * NB: There's no need to deal with spurious lock attempts
+		 * here. Anyone we prevented from acquiring the lock will
+		 * enqueue themselves using the same protocol we used here.
 		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
-
-		proc->lwWaiting = true;
-		proc->lwWaitMode = mode;
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
-
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
 
 		/*
 		 * Wait until awakened.
@@ -632,7 +1045,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockAcquire", T_NAME(lock), T_ID(lock), "waiting");
+		LOG_LWDEBUG("LWLockAcquire", lock, "waiting");
 
 #ifdef LWLOCK_STATS
 		lwstats->block_count++;
@@ -649,12 +1062,22 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 			extraWaits++;
 		}
 
+		/* not waiting anymore */
+		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
-		LOG_LWDEBUG("LWLockAcquire", T_NAME(lock), T_ID(lock), "awakened");
+		LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
 
 		/* Now loop back and try to acquire lock again. */
-		retry = true;
+#ifdef LWLOCK_STATS
+		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+		SpinLockAcquire(&lock->mutex);
+#endif
+		lock->releaseOK = true;
+		SpinLockRelease(&lock->mutex);
+
 		result = false;
 	}
 
@@ -662,13 +1085,11 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 	if (valptr)
 		*valptr = val;
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), mode);
 
 	/* Add lock to list of locks held by this backend */
-	held_lwlocks[num_held_lwlocks++] = lock;
+	held_lwlocks[num_held_lwlocks].lock = lock;
+	held_lwlocks[num_held_lwlocks++].mode = mode;
 
 	/*
 	 * Fix the process wait semaphore's count for any absorbed wakeups.
@@ -690,8 +1111,11 @@ bool
 LWLockConditionalAcquire(LWLock *lock, LWLockMode mode)
 {
 	bool		mustwait;
+	bool		potentially_spurious;
 
-	PRINT_LWDEBUG("LWLockConditionalAcquire", lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockConditionalAcquire", lock, mode);
 
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
@@ -704,50 +1128,44 @@ LWLockConditionalAcquire(LWLock *lock, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
+retry:
+	/*
+	 * We need to check whether it's possible that we prevented somebody else
+	 * from acquiring the lock. If so, potentially_spurious will be set, and
+	 * we'll retry.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, &potentially_spurious);
 
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
+	if (mustwait)
 	{
-		if (lock->exclusive == 0)
+		/*
+		 * We ran into an exclusive lock and might have blocked another
+		 * exclusive lock from taking a shot because it took a time to back
+		 * off. Retry till we are either sure we didn't block somebody (because
+		 * somebody else certainly has the lock) or till we got it.
+		 *
+		 * We cannot rely on the two-step lock-acquisition protocol as in
+		 * LWLockAcquire because we're not using it.
+		 */
+		if (potentially_spurious)
 		{
-			lock->shared++;
-			mustwait = false;
+			SPIN_DELAY();
+			goto retry;
 		}
-		else
-			mustwait = true;
-	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
-	if (mustwait)
-	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockConditionalAcquire",
-					T_NAME(lock), T_ID(lock), "failed");
-		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock),
-												 T_ID(lock), mode);
+
+		LOG_LWDEBUG("LWLockConditionalAcquire", lock, "failed");
+		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock), T_ID(lock), mode);
 	}
 	else
 	{
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lock;
+		held_lwlocks[num_held_lwlocks].lock = lock;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(T_NAME(lock), T_ID(lock), mode);
 	}
-
 	return !mustwait;
 }
 
@@ -773,14 +1191,14 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	int			extraWaits = 0;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
-#endif
-
-	PRINT_LWDEBUG("LWLockAcquireOrWait", lock);
 
-#ifdef LWLOCK_STATS
 	lwstats = get_lwlock_stats_entry(lock);
 #endif
 
+	Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockAcquireOrWait", lock, mode);
+
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
 		elog(ERROR, "too many LWLocks taken");
@@ -792,81 +1210,61 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
+	/*
+	 * NB: We're using nearly the same twice-in-a-row lock acquisition
+	 * protocol as LWLockAcquire(). Check its comments for details.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, NULL);
 
 	if (mustwait)
 	{
-		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait.  This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
-		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		mustwait = LWLockAttemptLock(lock, mode, NULL);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
-
-		/*
-		 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
-		 * wakups, because we share the semaphore with ProcWaitForSignal.
-		 */
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock),
-					"waiting");
+		if (mustwait)
+		{
+			/*
+			 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
+			 * wakups, because we share the semaphore with ProcWaitForSignal.
+			 */
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "waiting");
 
 #ifdef LWLOCK_STATS
-		lwstats->block_count++;
+			lwstats->block_count++;
 #endif
+			TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(l), T_ID(l), mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+			for (;;)
+			{
+				/* "false" means cannot accept cancel/die interrupt here. */
+				PGSemaphoreLock(&proc->sem, false);
+				if (!proc->lwWaiting)
+					break;
+				extraWaits++;
+			}
+			pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
 
-		for (;;)
-		{
-			/* "false" means cannot accept cancel/die interrupt here. */
-			PGSemaphoreLock(&proc->sem, false);
-			if (!proc->lwWaiting)
-				break;
-			extraWaits++;
-		}
+			TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
+		}
+		else
+		{
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "acquired, undoing queue");
 
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock),
-					"awakened");
-	}
-	else
-	{
-		/* We are done updating shared state of the lock itself. */
-		SpinLockRelease(&lock->mutex);
+			/* got lock in the second attempt, undo queueing */
+			if (!LWLockDequeueSelf(lock))
+			{
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+			}
+		}
 	}
 
 	/*
@@ -879,16 +1277,17 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock), "failed");
+		LOG_LWDEBUG("LWLockAcquireOrWait", lock, "failed");
 		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT_FAIL(T_NAME(lock), T_ID(lock),
 													 mode);
 	}
 	else
 	{
+		LOG_LWDEBUG("LWLockAcquireOrWait", lock, "succeeded");
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lock;
-		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock),
-												mode);
+		held_lwlocks[num_held_lwlocks].lock = lock;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
+		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock), mode);
 	}
 
 	return !mustwait;
@@ -920,13 +1319,11 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 	bool		result = false;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
-#endif
 
-	PRINT_LWDEBUG("LWLockWaitForVar", lock);
-
-#ifdef LWLOCK_STATS
 	lwstats = get_lwlock_stats_entry(lock);
-#endif   /* LWLOCK_STATS */
+#endif
+
+	PRINT_LWDEBUG("LWLockWaitForVar", lock, LW_WAIT_UNTIL_FREE);
 
 	/*
 	 * Quick test first to see if it the slot is free right now.
@@ -935,7 +1332,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 	 * barrier here as far as the current usage is concerned.  But that might
 	 * not be safe in general.
 	 */
-	if (lock->exclusive == 0)
+	if (pg_atomic_read_u32(&lock->lockcount) == 0)
 		return true;
 
 	/*
@@ -953,21 +1350,24 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		bool		mustwait;
 		uint64		value;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
+		mustwait = pg_atomic_read_u32(&lock->lockcount) != 0;
+
+		if (mustwait)
+		{
+			/*
+			 * Perform comparison using spinlock as we can't rely on atomic 64
+			 * bit reads/stores.
+			 */
 #ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+			lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
 #else
-		SpinLockAcquire(&lock->mutex);
+			SpinLockAcquire(&lock->mutex);
 #endif
 
-		/* Is the lock now free, and if not, does the value match? */
-		if (lock->exclusive == 0)
-		{
-			result = true;
-			mustwait = false;
-		}
-		else
-		{
+			/*
+			 * XXX: We can significantly optimize this on platforms with 64bit
+			 * atomics.
+			 */
 			value = *valptr;
 			if (value != oldval)
 			{
@@ -977,21 +1377,58 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 			}
 			else
 				mustwait = true;
+			SpinLockRelease(&lock->mutex);
 		}
+		else
+			mustwait = false;
 
 		if (!mustwait)
 			break;				/* the lock was free or value didn't match */
 
 		/*
-		 * Add myself to wait queue.
+		 * Add myself to wait queue. Note that this is racy, somebody else
+		 * could wakeup before we're finished queuing.
+		 * NB: We're using nearly the same twice-in-a-row lock acquisition
+		 * protocol as LWLockAcquire(). Check its comments for details.
 		 */
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		/* waiters are added to the front of the queue */
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
+		/*
+		 * We're now guaranteed to be woken up if necessary. Recheck the
+		 * lock's state.
+		 */
+		mustwait = pg_atomic_read_u32(&lock->lockcount) != 0;
+
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
+		{
+#ifdef LWLOCK_STATS
+			lwstats->dequeue_self_count++;
+#endif
+			LOG_LWDEBUG("LWLockWaitForVar", lock, "free, undoing queue");
+
+			if (!LWLockDequeueSelf(lock))
+			{
+				/*
+				 * Somebody else dequeued us and has or will wake us up. Wait
+				 * for the correct wakeup, otherwise our ->lwWaiting would get
+				 * reset at some inconvenient point later.
+				 */
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+			}
+			break;
+		}
+
+		/*
+		 * NB: Just as in LWLockAcquireCommon() there's no need to deal with
+		 * spurious lock attempts here.
+		 */
 
 		/*
 		 * Wait until awakened.
@@ -1005,7 +1442,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockWaitForVar", T_NAME(lock), T_ID(lock), "waiting");
+		LOG_LWDEBUG("LWLockWaitForVar", lock, "waiting");
 
 #ifdef LWLOCK_STATS
 		lwstats->block_count++;
@@ -1023,17 +1460,16 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 			extraWaits++;
 		}
 
+		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock),
 										  LW_EXCLUSIVE);
 
-		LOG_LWDEBUG("LWLockWaitForVar", T_NAME(lock), T_ID(lock), "awakened");
+		LOG_LWDEBUG("LWLockWaitForVar", lock, "awakened");
 
 		/* Now loop back and check the status of the lock again. */
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), LW_EXCLUSIVE);
 
 	/*
@@ -1066,14 +1502,24 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 {
 	dlist_head	wakeup;
 	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	PRINT_LWDEBUG("LWLockUpdateVar", lock, LW_EXCLUSIVE);
 
 	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
 	SpinLockAcquire(&lock->mutex);
+#endif
 
-	/* we should hold the lock */
-	Assert(lock->exclusive == 1);
+	Assert(pg_atomic_read_u32(&lock->lockcount) >= EXCLUSIVE_LOCK);
 
 	/* Update the lock's value */
 	*valptr = val;
@@ -1116,22 +1562,23 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 void
 LWLockRelease(LWLock *lock)
 {
-	dlist_head	wakeup;
-	dlist_mutable_iter iter;
+	LWLockMode	mode;
+	uint32		lockcount;
+	bool		check_waiters;
+	bool		have_waiters = false;
 	int			i;
 
-	dlist_init(&wakeup);
-
-	PRINT_LWDEBUG("LWLockRelease", lock);
-
 	/*
 	 * Remove lock from list of locks held.  Usually, but not always, it will
 	 * be the latest-acquired lock; so search array backwards.
 	 */
 	for (i = num_held_lwlocks; --i >= 0;)
 	{
-		if (lock == held_lwlocks[i])
+		if (lock == held_lwlocks[i].lock)
+		{
+			mode = held_lwlocks[i].mode;
 			break;
+		}
 	}
 	if (i < 0)
 		elog(ERROR, "lock %s %d is not held", T_NAME(lock), T_ID(lock));
@@ -1139,78 +1586,57 @@ LWLockRelease(LWLock *lock)
 	for (; i < num_held_lwlocks; i++)
 		held_lwlocks[i] = held_lwlocks[i + 1];
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* Release my hold on lock */
-	if (lock->exclusive > 0)
-		lock->exclusive--;
-	else
-	{
-		Assert(lock->shared > 0);
-		lock->shared--;
-	}
+	PRINT_LWDEBUG("LWLockRelease", lock, mode);
 
 	/*
-	 * See if I need to awaken any waiters.  If I released a non-last shared
-	 * hold, there cannot be anything to do.  Also, do not awaken any waiters
-	 * if someone has already awakened waiters that haven't yet acquired the
-	 * lock.
+	 * Release my hold on lock, after that it can immediately be acquired by
+	 * others, even if we still have to wakeup other waiters.
 	 */
-	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
-	{
-		/*
-		 * Remove the to-be-awakened PGPROCs from the queue.
-		 */
-		bool		releaseOK = true;
-		bool		wokeup_somebody = false;
-
-		dlist_foreach_modify(iter, &lock->waiters)
-		{
-			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-
-			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
-				continue;
-
-			dlist_delete(&waiter->lwWaitLink);
-			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
-
-			/*
-			 * Prevent additional wakeups until retryer gets to
-			 * run. Backends that are just waiting for the lock to become
-			 * free don't retry automatically.
-			 */
-			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
-			{
-				releaseOK = false;
-				wokeup_somebody = true;
-			}
+	if (mode == LW_EXCLUSIVE)
+		lockcount = pg_atomic_sub_fetch_u32(&lock->lockcount, EXCLUSIVE_LOCK);
+	else
+		lockcount = pg_atomic_sub_fetch_u32(&lock->lockcount, 1);
 
-			if(waiter->lwWaitMode == LW_EXCLUSIVE)
-				break;
-		}
-		lock->releaseOK = releaseOK;
-	}
+	/* nobody else can have that kind of lock */
+	Assert(lockcount < EXCLUSIVE_LOCK);
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
+	/*
+	 * Anybody we need to wakeup needs to have started queueing before we
+	 * removed ourselves from the queue and the atomic operations above are
+	 * full barriers. So we can just do plain read.
+	 */
+	if (pg_atomic_read_u32(&lock->nwaiters) > 0)
+		have_waiters = true;
 
-	TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+	/*
+	 * If we just released an exclusive lock always wakeup waiters, even if
+	 * lockcount is still > 0. A shared acquisition temporarily (and
+	 * spuriously) might have increased the value.
+	 */
+	if (mode == LW_EXCLUSIVE && have_waiters)
+		check_waiters = true;
+	/*
+	 * nobody has this locked anymore, potential exclusive lockers get a chance
+	 */
+	else if (lockcount == 0 && have_waiters)
+		check_waiters = true;
+	/* nobody queued or not free */
+	else
+		check_waiters = false;
 
 	/*
-	 * Awaken any waiters I removed from the queue.
+	 * As waking up waiters requires the spinlock to be acquired, only do so
+	 * if necessary.
 	 */
-	dlist_foreach_modify(iter, &wakeup)
+	if (check_waiters)
 	{
-		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-		LOG_LWDEBUG("LWLockRelease", T_NAME(lock), T_ID(lock),
-					"release waiter");
-		dlist_delete(&waiter->lwWaitLink);
-		pg_write_barrier();
-		waiter->lwWaiting = false;
-		PGSemaphoreUnlock(&waiter->sem);
+		/* XXX: remove before commit? */
+		LOG_LWDEBUG("LWLockRelease", lock, "releasing waiters");
+		LWLockWakeup(lock, mode);
 	}
 
+	TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+
 	/*
 	 * Now okay to allow cancel/die interrupts.
 	 */
@@ -1234,7 +1660,7 @@ LWLockReleaseAll(void)
 	{
 		HOLD_INTERRUPTS();		/* match the upcoming RESUME_INTERRUPTS */
 
-		LWLockRelease(held_lwlocks[num_held_lwlocks - 1]);
+		LWLockRelease(held_lwlocks[num_held_lwlocks - 1].lock);
 	}
 }
 
@@ -1242,8 +1668,8 @@ LWLockReleaseAll(void)
 /*
  * LWLockHeldByMe - test whether my process currently holds a lock
  *
- * This is meant as debug support only.  We do not distinguish whether the
- * lock is held shared or exclusive.
+ * This is meant as debug support only.  We currently do not distinguish
+ * whether the lock is held shared or exclusive.
  */
 bool
 LWLockHeldByMe(LWLock *l)
@@ -1252,7 +1678,7 @@ LWLockHeldByMe(LWLock *l)
 
 	for (i = 0; i < num_held_lwlocks; i++)
 	{
-		if (held_lwlocks[i] == l)
+		if (held_lwlocks[i].lock == l)
 			return true;
 	}
 	return false;
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index fea5d33..595e69d 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -16,6 +16,7 @@
 
 #include "lib/ilist.h"
 #include "storage/s_lock.h"
+#include "port/atomics.h"
 
 struct PGPROC;
 
@@ -48,10 +49,14 @@ typedef struct LWLock
 {
 	slock_t		mutex;			/* Protects LWLock and queue of PGPROCs */
 	bool		releaseOK;		/* T if ok to release waiters */
-	char		exclusive;		/* # of exclusive holders (0 or 1) */
-	int			shared;			/* # of shared holders (0..MaxBackends) */
+
+	pg_atomic_uint32 lockcount;	/* state of exlusive/nonexclusive lockers */
+	pg_atomic_uint32 nwaiters;	/* number of waiters */
 	int			tranche;		/* tranche ID */
 	dlist_head	waiters;		/* list of waiting PGPROCs */
+#ifdef LOCK_DEBUG
+	struct PGPROC *owner;		/* last exlusive owner of the lock */
+#endif
 } LWLock;
 
 /*
-- 
1.8.3.251.g1462b67

#65Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#62)
Re: Wait free LW_SHARED acquisition - v0.2

On Wed, Oct 8, 2014 at 8:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.
And the code is *so* much more readable.

I find the slist/dlist stuff actually quite difficult to get right
compared to a hand-rolled linked list. But the really big problem is
that the debugger can't do anything useful with it. You have to work
out the structure-member offset in order to walk the list and manually
cast to char *, adjust the pointer, and cast back. That sucks.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#66Andres Freund
andres@2ndquadrant.com
In reply to: Robert Haas (#65)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-08 13:13:33 -0400, Robert Haas wrote:

On Wed, Oct 8, 2014 at 8:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.
And the code is *so* much more readable.

I find the slist/dlist stuff actually quite difficult to get right
compared to a hand-rolled linked list.

Really? I've spent more than a day debugging things with the current
code. And Heikki introduced a bug in it. If you look at how the code
looks before/after I find the difference pretty clear.

But the really big problem is
that the debugger can't do anything useful with it. You have to work
out the structure-member offset in order to walk the list and manually
cast to char *, adjust the pointer, and cast back. That sucks.

Hm. I can just do that with the debugger here. Not sure if that's
because I added the right thing to my .gdbinit or because I use the
correct compiler flags.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#67Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Robert Haas (#65)
Re: Wait free LW_SHARED acquisition - v0.2

Robert Haas wrote:

On Wed, Oct 8, 2014 at 8:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.
And the code is *so* much more readable.

I find the slist/dlist stuff actually quite difficult to get right
compared to a hand-rolled linked list. But the really big problem is
that the debugger can't do anything useful with it. You have to work
out the structure-member offset in order to walk the list and manually
cast to char *, adjust the pointer, and cast back. That sucks.

As far as I recall you can get gdb to understand those pointer games
by defining some structs or macros. Maybe we can improve by documenting
this.

--
�lvaro Herrera http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#68Andres Freund
andres@2ndquadrant.com
In reply to: Alvaro Herrera (#67)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-08 14:23:44 -0300, Alvaro Herrera wrote:

Robert Haas wrote:

On Wed, Oct 8, 2014 at 8:47 AM, Andres Freund <andres@2ndquadrant.com> wrote:

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.
And the code is *so* much more readable.

I find the slist/dlist stuff actually quite difficult to get right
compared to a hand-rolled linked list. But the really big problem is
that the debugger can't do anything useful with it. You have to work
out the structure-member offset in order to walk the list and manually
cast to char *, adjust the pointer, and cast back. That sucks.

As far as I recall you can get gdb to understand those pointer games
by defining some structs or macros. Maybe we can improve by documenting
this.

So, what makes it work for me (among other unrelated stuff) seems to be
the following in .gdbinit, defineing away some things that gdb doesn't
handle:
macro define __builtin_offsetof(T, F) ((int) &(((T *) 0)->F))
macro define __extension__
macro define AssertVariableIsOfTypeMacro(x, y) ((void)0)

Additionally I have "-ggdb -g3" in CFLAGS. That way gdb knows about
postgres' macros. At least if you're in the right scope.

As an example, the following works:
(gdb) p dlist_is_empty(&BackendList) ? NULL : dlist_head_element(Backend, elem, &BackendList)

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#69Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#64)
Re: Wait free LW_SHARED acquisition - v0.9

On Wed, Oct 8, 2014 at 9:35 AM, Andres Freund <andres@2ndquadrant.com> wrote:

1) Convert PGPROC->lwWaitLink into a dlist. The old code was frail and
verbose. This also does:
* changes the logic in LWLockRelease() to release all shared lockers
when waking up any. This can yield some significant performance
improvements - and the fairness isn't really much worse than
before,
as we always allowed new shared lockers to jump the queue.

* adds a memory pg_write_barrier() in the wakeup paths between
dequeuing and unsetting ->lwWaiting. That was always required on
weakly ordered machines, but f4077cda2 made it more urgent. I can
reproduce crashes without it.

I think it's a really bad idea to mix a refactoring change (like
converting PGPROC->lwWaitLink into a dlist) with an attempted
performance enhancement (like changing the rules for jumping the lock
queue) and a bug fix (like adding pg_write_barrier where needed). I'd
suggest that the last of those be done first, and perhaps
back-patched.

The current coding, using a hand-rolled list, touches shared memory
fewer times. When many waiters are awoken at once, we clip them all
out of the list at one go. Your revision moves them to a
backend-private list one at a time, and then pops them off one at a
time. The backend-private memory accesses don't seem like they matter
much, but the shared memory accesses would be nice to avoid.

Does LWLockUpdateVar's wake-up loop need a write barrier per
iteration, or just one before the loop starts? How about commenting
the pg_write_barrier() with the read-fence to which it pairs?

+ if(waiter->lwWaitMode == LW_EXCLUSIVE)

Whitespace.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#70Andres Freund
andres@2ndquadrant.com
In reply to: Robert Haas (#69)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-08 15:23:22 -0400, Robert Haas wrote:

On Wed, Oct 8, 2014 at 9:35 AM, Andres Freund <andres@2ndquadrant.com> wrote:

1) Convert PGPROC->lwWaitLink into a dlist. The old code was frail and
verbose. This also does:
* changes the logic in LWLockRelease() to release all shared lockers
when waking up any. This can yield some significant performance
improvements - and the fairness isn't really much worse than
before,
as we always allowed new shared lockers to jump the queue.

* adds a memory pg_write_barrier() in the wakeup paths between
dequeuing and unsetting ->lwWaiting. That was always required on
weakly ordered machines, but f4077cda2 made it more urgent. I can
reproduce crashes without it.

I think it's a really bad idea to mix a refactoring change (like
converting PGPROC->lwWaitLink into a dlist) with an attempted
performance enhancement (like changing the rules for jumping the lock
queue) and a bug fix (like adding pg_write_barrier where needed). I'd
suggest that the last of those be done first, and perhaps
back-patched.

I think it makes sense to separate out the write barrier one. I don't
really see the point of separating the other two changes.

I've indeed previously started a thread
(http://archives.postgresql.org/message-id/20140210134625.GA15246%40awork2.anarazel.de)
about the barrier issue. IIRC you argued that that might be to
expensive.

The current coding, using a hand-rolled list, touches shared memory
fewer times. When many waiters are awoken at once, we clip them all
out of the list at one go. Your revision moves them to a
backend-private list one at a time, and then pops them off one at a
time. The backend-private memory accesses don't seem like they matter
much, but the shared memory accesses would be nice to avoid.

I can't imagine this to matter. We're entering the kernel for each PROC
for the PGSemaphoreUnlock() and we're dirtying the cacheline for
proc->lwWaiting = false anyway. This really is the slow path.

Does LWLockUpdateVar's wake-up loop need a write barrier per
iteration, or just one before the loop starts? How about commenting
the pg_write_barrier() with the read-fence to which it pairs?

Hm. Are you picking out LWLockUpdateVar for a reason or just as an
example? Because I don't see a difference between the different wakeup
loops?
It needs to be a barrier per iteration.

Currently the loop looks like
while (head != NULL)
{
proc = head;
head = proc->lwWaitLink;
proc->lwWaitLink = NULL;
proc->lwWaiting = false;
PGSemaphoreUnlock(&proc->sem);
}

Consider what happens if either the compiler or the cpu reorders this
to:
proc->lwWaiting = false;
head = proc->lwWaitLink;
proc->lwWaitLink = NULL;
PGSemaphoreUnlock(&proc->sem);

as soon as lwWaiting = false, 'proc' can wake up and acquire a new
lock. Backends can wake up prematurely because proc->sem is used for
other purposes than this (that's why the loops around PGSemaphoreLock
exist). Then it could reset lwWaitLink while acquiring a new lock. And
some processes wouldn't be woken up anymore.

The barrier it pairs with is the spinlock acquiration before
requeuing. To be more obviously correct we could add a read barrier
before
if (!proc->lwWaiting)
break;
but I don't think it's needed.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#71Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#64)
Re: Wait free LW_SHARED acquisition - v0.9

On Wed, Oct 8, 2014 at 9:35 AM, Andres Freund <andres@2ndquadrant.com> wrote:

2) Implement the wait free LW_SHARED algorithm.

+ * too high for workloads/locks that were locked in shared mode very

s/locked/taken/?

+ * frequently. Often we were spinning in the (obviously exlusive) spinlock,

exclusive.

+ * acquiration for locks that aren't exclusively locked.

acquisition.

+ * For exlusive lock acquisition we use an atomic compare-and-exchange on the

exclusive.

+ * lockcount variable swapping in EXCLUSIVE_LOCK/1<<31-1/0x7FFFFFFF if and only

Add comma after variable. Find some way of describing the special
value (maybe "a sentinel value, EXCLUSIVE_LOCK") just once, instead of
three times.

+ * if the current value of lockcount is 0. If the swap was not successfull, we

successful.

+ * by 1 again. If so, we have to wait for the exlusive locker to release the

exclusive.

+ * The attentive reader probably might have noticed that naively doing the

"probably might" is redundant. Delete probably.

+ * notice that we have to wait. Unfortunately until we have finished queuing,

until -> by the time

+ * Phase 2: Add us too the waitqueue of the lock

too -> to. And maybe us -> ourselves.

+ * get queued in Phase 2 and we can wake them up if neccessary or they will

necessary.

+ * When acquiring shared locks it's possible that we disturb an exclusive
+ * waiter. If that's a problem for the specific user, pass in a valid pointer
+ * for 'potentially_spurious'. Its value will be set to true if we possibly
+ * did so. The caller then has to handle that scenario.

"disturb" is not clear enough.

+ /* yipeyyahee */

Although this will be clear to individuals with a good command of
English, I suggest avoiding such usages.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#72Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#68)
Re: Wait free LW_SHARED acquisition - v0.2

On Wed, Oct 8, 2014 at 2:04 PM, Andres Freund <andres@2ndquadrant.com> wrote:

So, what makes it work for me (among other unrelated stuff) seems to be
the following in .gdbinit, defineing away some things that gdb doesn't
handle:
macro define __builtin_offsetof(T, F) ((int) &(((T *) 0)->F))
macro define __extension__
macro define AssertVariableIsOfTypeMacro(x, y) ((void)0)

Additionally I have "-ggdb -g3" in CFLAGS. That way gdb knows about
postgres' macros. At least if you're in the right scope.

As an example, the following works:
(gdb) p dlist_is_empty(&BackendList) ? NULL : dlist_head_element(Backend, elem, &BackendList)

Ah, cool. I'll try that.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#73Jim Nasby
Jim.Nasby@BlueTreble.com
In reply to: Andres Freund (#64)
Re: Wait free LW_SHARED acquisition - v0.9

On 10/8/14, 8:35 AM, Andres Freund wrote:

+#define EXCLUSIVE_LOCK (((uint32) 1) << (31 - 1))
+
+/* Must be greater than MAX_BACKENDS - which is 2^23-1, so we're fine. */
+#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)

There should at least be a comment where we define MAX_BACKENDS about the relationship here... or better yet, validate that MAX_BACKENDS > SHARED_LOCK_MASK during postmaster startup. (For those that think that's too pedantic, I'll argue that it's no worse than the patch verifying that MyProc != NULL in LWLockQueueSelf()).

+/*
+ * Internal function that tries to atomically acquire the lwlock in the passed
+ * in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * callers job.
+ *
+ * Returns true if the lock isn't free and we need to wait.
+ *
+ * When acquiring shared locks it's possible that we disturb an exclusive
+ * waiter. If that's a problem for the specific user, pass in a valid pointer
+ * for 'potentially_spurious'. Its value will be set to true if we possibly
+ * did so. The caller then has to handle that scenario.
+ */
+static bool
+LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)

We should invert the return of this function. Current code returns true if the lock is actually acquired (see below), and I think that's true of other locking code as well. IMHO it makes more sense that way, plus consistency is good.

(From 9.3)
* LWLockConditionalAcquire - acquire a lightweight lock in the specified mode
*
* If the lock is not available, return FALSE with no side-effects.
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#74Andres Freund
andres@2ndquadrant.com
In reply to: Jim Nasby (#73)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-09 16:52:46 -0500, Jim Nasby wrote:

On 10/8/14, 8:35 AM, Andres Freund wrote:

+#define EXCLUSIVE_LOCK (((uint32) 1) << (31 - 1))
+
+/* Must be greater than MAX_BACKENDS - which is 2^23-1, so we're fine. */
+#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)

There should at least be a comment where we define MAX_BACKENDS about the relationship here... or better yet, validate that MAX_BACKENDS > SHARED_LOCK_MASK during postmaster startup. (For those that think that's too pedantic, I'll argue that it's no worse than the patch verifying that MyProc != NULL in LWLockQueueSelf()).

If you modify either, you better grep for them... I don't think that's
going to happen anyway. Requiring it during startup would mean exposing
SHARED_LOCK_MASK outside of lwlock.c which'd be ugly. We could possibly
stick a StaticAssert() someplace in lwlock.c.

And no, it's not comparable at all to MyProc != NULL - the lwlock code
initially *does* run when MyProc isn't setup. We just better not
conflict against any other lockers at that stage.

+/*
+ * Internal function that tries to atomically acquire the lwlock in the passed
+ * in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * callers job.
+ *
+ * Returns true if the lock isn't free and we need to wait.
+ *
+ * When acquiring shared locks it's possible that we disturb an exclusive
+ * waiter. If that's a problem for the specific user, pass in a valid pointer
+ * for 'potentially_spurious'. Its value will be set to true if we possibly
+ * did so. The caller then has to handle that scenario.
+ */
+static bool
+LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)

We should invert the return of this function. Current code returns
true if the lock is actually acquired (see below), and I think that's
true of other locking code as well. IMHO it makes more sense that way,
plus consistency is good.

I don't think so. I've wondered about it as well, but the way the
function is used its more consistent imo if it returns whether we must
wait. Note that it's not an exported function.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#75Jim Nasby
Jim.Nasby@BlueTreble.com
In reply to: Andres Freund (#74)
Re: Wait free LW_SHARED acquisition - v0.9

On 10/9/14, 4:57 PM, Andres Freund wrote:

If you modify either, you better grep for them... I don't think that's
going to happen anyway. Requiring it during startup would mean exposing
SHARED_LOCK_MASK outside of lwlock.c which'd be ugly. We could possibly
stick a StaticAssert() someplace in lwlock.c.

Ahh, yeah, exposing it would be ugly.

I just get the heeby-jeebies when I see assumptions like this though. I fear there's a bunch of cases where changing something will break a completely unrelated part of the system with no warning.

Maybe add an assert() to check it?

And no, it's not comparable at all to MyProc != NULL - the lwlock code
initially*does* run when MyProc isn't setup. We just better not
conflict against any other lockers at that stage.

Ahh, can you maybe add that detail to the comment? That wasn't clear to me.

+/*
+ * Internal function that tries to atomically acquire the lwlock in the passed
+ * in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * callers job.
+ *
+ * Returns true if the lock isn't free and we need to wait.
+ *
+ * When acquiring shared locks it's possible that we disturb an exclusive
+ * waiter. If that's a problem for the specific user, pass in a valid pointer
+ * for 'potentially_spurious'. Its value will be set to true if we possibly
+ * did so. The caller then has to handle that scenario.
+ */
+static bool
+LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)

We should invert the return of this function. Current code returns
true if the lock is actually acquired (see below), and I think that's
true of other locking code as well. IMHO it makes more sense that way,
plus consistency is good.

I don't think so. I've wondered about it as well, but the way the
function is used its more consistent imo if it returns whether we must
wait. Note that it's not an exported function.

ISTM that a function attempting a lock would return success, not failure. Even though it's internal now it could certainly be made external at some point in the future. But I suppose it's ultimately a matter of preference...
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#76Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#64)
1 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.9

On Wed, Oct 8, 2014 at 7:05 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

Hi,

Attached you can find the next version of my LW_SHARED patchset. Now
that atomics are committed, it seems like a good idea to also add their
raison d'être.

Since the last public version I have:
* Addressed lots of Amit's comments. Thanks!
* Peformed a fair amount of testing.
* Rebased the code. The volatile removal made that not entirely
trivial...
* Significantly cleaned up and simplified the code.
* Updated comments and such
* Fixed a minor bug (unpaired HOLD/RESUME_INTERRUPTS in a corner case)

The feature currently consists out of two patches:
1) Convert PGPROC->lwWaitLink into a dlist. The old code was frail and
verbose. This also does:
* changes the logic in LWLockRelease() to release all shared lockers
when waking up any. This can yield some significant performance
improvements - and the fairness isn't really much worse than
before,
as we always allowed new shared lockers to jump the queue.

* adds a memory pg_write_barrier() in the wakeup paths between
dequeuing and unsetting ->lwWaiting. That was always required on
weakly ordered machines, but f4077cda2 made it more urgent. I can
reproduce crashes without it.
2) Implement the wait free LW_SHARED algorithm.

I have done few performance tests for above patches and results of
same is as below:

Performance Data
------------------------------
IBM POWER-7 16 cores, 64 hardware threads
RAM = 64GB
max_connections =210
Database Locale =C
checkpoint_segments=256
checkpoint_timeout =35min
shared_buffers=8GB
Client Count = number of concurrent sessions and threads (ex. -c 8 -j 8)
Duration of each individual run = 5mins
Test type - read only pgbench with -M prepared
Other Related information about test
a. This is the data for median of 3 runs, the detailed data of individual
run
is attached with mail.
b. I have applied both the patches to take performance data.

Scale Factor - 100

Patch_ver/Client_count 1 8 16 32 64 128 HEAD 13344 106921 196629 295123
377846 333928 PATCH 13662 106179 203960 298955 452638 465671

Scale Factor - 3000

Patch_ver/Client_count 8 16 32 64 128 160 HEAD 86920 152417 231668
280827 257093 255122 PATCH 87552 160313 230677 276186 248609 244372

Observations
----------------------
a. The patch performs really well (increase upto ~40%) incase all the
data fits in shared buffers (scale factor -100).
b. Incase data doesn't fit in shared buffers, but fits in RAM
(scale factor -3000), there is performance increase upto 16 client count,
however after that it starts dipping (in above config unto ~4.4%).

The above data shows that the patch improves performance for cases
when there is shared LWLock contention, however there is a slight
performance dip in case of Exclusive LWLocks (at scale factor 3000,
it needs exclusive LWLocks for buf mapping tables). Now I am not
sure if this is the worst case dip or under similar configurations the
performance dip can be higher, because the trend shows that dip is
increasing with more client counts.

Brief Analysis of code w.r.t performance dip
---------------------------------------------------------------------
Extra Instructions w.r.t Head in Acquire Exclusive lock path
a. Attempt lock twice
b. atomic operations for nwaiters in LWLockQueueSelf() and
LWLockAcquireCommon()
c. Now we need to take spinlock twice, once for self queuing and then
again for setting releaseOK.
d. few function calls and some extra checks

Similarly there seems to be few additional instructions in
LWLockRelease() path.

Now probably these shouldn't matter much in case backend needs to
wait for other Exclusive locker, but I am not sure what else could be
the reason for dip in case we need to have Exclusive LWLocks.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

Attachments:

perf_lwlock_contention_data_v1.odsapplication/vnd.oasis.opendocument.spreadsheet; name=perf_lwlock_contention_data_v1.odsDownload
PKc$JE�l9�..mimetypeapplication/vnd.oasis.opendocument.spreadsheetPKc$JE�]�'RRmeta.xml<?xml version="1.0" encoding="UTF-8"?>
<office:document-meta xmlns:office="urn:oasis:names:tc:opendocument:xmlns:office:1.0" xmlns:xlink="http://www.w3.org/1999/xlink" xmlns:dc="http://purl.org/dc/elements/1.1/" xmlns:meta="urn:oasis:names:tc:opendocument:xmlns:meta:1.0" xmlns:ooo="http://openoffice.org/2004/office" xmlns:grddl="http://www.w3.org/2003/g/data-view#" office:version="1.2"><office:meta><meta:creation-date>2014-03-31T15:29:55</meta:creation-date><dc:date>2014-10-10T10:05:07</dc:date><meta:editing-duration>P3DT1H41M3S</meta:editing-duration><meta:editing-cycles>70</meta:editing-cycles><meta:generator>LibreOffice/3.5$MacOSX_x86 LibreOffice_project/7e68ba2-a744ebf-1f241b7-c506db1-7d53735</meta:generator><meta:document-statistic meta:table-count="1" meta:cell-count="121" meta:object-count="0"/></office:meta></office:document-meta>PKc$JEsettings.xml�Z�s�6~�_����#9z�n�k.�i.�I�����D�z$����0d�c0V���)��~���>�����������N���T��P>m;w���g�K���L�n�~WU	J�-����t����"�T��� ]������������#��������Zm�X�,NOPLk�����ju��G>��}U%��U!��"#��R����j�g��6r�5M������w.�
�?U� 4������V��),���d�=�����t�F�fQ-#�H�r:���K����a�� �E5��n�������Ng��7?5Z�����$�R�#i]����JF��X�c1,A�L��N�c�q��4e��h��2��Hb�"$��J@�#���(nQR����%��s����
��
��"��Ub�H�P�� �m����x����m�����V�C��9�i��[2�m`�y�r$n�L�A��Rf��i1���H�������8�����N S0<�*z� �7��7A�>Fd@��Q"�l����7G�����=[+���]�,4��{�7="/>�-I6�����[�@�K�	a2'��Q�����g�>�7�lA;���
j�4Cs���i���UB\-x��Hmz�!.���a����(�9��{�t��2#��o�R�x�D#� �2NV<��F���!H��R\�[AJ�g����{�8�s*s�/	<������w�����	��'�]]�nR�7HP��!�� &��$��CM/���Bj�E�L�K�k#��/�numd�T����DB��G9K�v���������+C����u���]SR�e���16�
��A	O����=�R~�)�0�o<xU~�Q������������
���z�X}�}����jp�?���rb��n��O�������u�����z��v�R;j�'�����O�y����P�lY�5���P��#�����s����n������
�����]�i�6���I���t��5@�~��{�7�4������=u0}#��O��Y��6���.c:���~�q����;����8��8����^���������PK�-`�,:#PKc$JEcontent.xml�]��������H�j��e�� �M��&�����m��Dy<�_}��a��$%��G���cyF;�������������w�Q(=��I<�7�H����A<��~����������d6<|�'�2�q.{I���%�:&�E�d��������(��6�n���V�b�[>V���Oa�����:��y���n�-�v�W[�Zum��R���gI���$�g	�z��<�B����h����x�Z�nV�M������c^Z��z�2y-�����X�Q����QW|��)^FS�uV
���UM3Lh:]f��:����a����-j�(�lg�r�Tt������6B��e}��=-�����Uu���������4��b�$I*��A����(��������WY��L��������4�DMJ���1�!�f��1E����(�*�������[�m*/W����(�h&c��:Ss��4��J1���KWK��-�(l�V��:�|��*���)�P'����������J��8\{IU��~Lq������4jIz+���k=v��D���{�r�=o����cV&��B	�I���nC�!�����Ry�<,��������R!3����Y@�EYj]�S�x�bS�nn]���Y�<�1��:�
yi��IL���1���k�A�Q�����4���6��m��vZ�=��	K��P��	�v�b�t7�'���t��"a��SUp���6[.��eN�O��TF���z�Z
V��A�r�������PN���,0�f��4���<��}i�l�u�e�U�����h��E�g6
����M��fX{b3���~�L8l����-k��,Ym�ODTE{���|A��~�;l�v^,'iD(���y����n��p+7����n��p��o�`�[��0};6�5���aD��8�S4�r����e�oMJ�PqF������S����.G4�NFa&��=h��%V{�e,9����Y�-Ye(�*eF&���-,�
��$��!?N�=���mq
�r�Xi.��h�K_)��>������9����Sm+K��(�/�LF8��d���O���2=��xk���a]��Y�'�����C�	�c[�U��N�"��
n���P0c�U@��+�����$��o�,�0�i�3�z{.��,Y���v�!H��������j�!���8
�@>���#��eL��c�a[w�*iz�_c��Y�7�����U���VMKm5w[m��ui���L�0����A?;*p@�e7B���aU���Z�p��(]����d�����Z�S�W(}e�7�8�u^�?4�0���9���Q�����y�jq�M-�F����������<	����V�����a�?���T���5F������D������1��8h6�z��y�$���(��L�e������u�N1��	U���N(4�������(L;~�aq����j��!"�$"o}�<��W�O�
=nc.�=b?�q{<x�����S�?Q~�:7�30������a�30��0���a>?�%�������nv�����`�������V�q�$zc��K"M&�6e2������i�?m���F>Y`���+�{�����b�T!��>{�Bbo&��R1b��g����
U����jYg���������}�K�U�\ �OF�F����9p��'��wY��E���Q�d�0$�e�����
�>��kA������6&y�s���Y������	Q��{���`���;z�}��T��s����j|W�;���6�����o������#���%&�,CZ��_���D��8������4�����Q�|��4EK�$��5yTw�{_�$��?��W9'�����<�0��Je���7����8Q�r��n ���l[B�[�=�5������N_"��W���r!�����${�#����o��P|X�$#o!�<��R��)0�N��z�����</8���������w�2h=�k���_�c�u����2��Y���"]��R=�,;p�_��������:^��9S�������W���>���?���gd������'��?!/O2��j�������H�Q����:�{r#�:�>.)�~����Cf]Ga7���	��%T+T*���f`v��0eth]��k�C[�fh��U�&�5����`W_;��S�qZ*��!�:����`L\������
ASE��B�����a�Kg���
�n��`��/��{E�a�������r���r5q��eX�e���L�m9"Y0�,,M1E�`20Yh���d���,��6�Rvw�\�&[�jdAe`�0��W$����1d�jz�,�LN�2�,tW������V8<��� t��u<k��Z�����C����I���U�x_��������C������<��tCn����!7����q�������
�Y���'H�1k'H*��u�N��|�t\�O�L�`��ilZ��,CS�Cl!A�u������Ni[V-KEEh�Qm��1T�� 9����0����Lf�����0�.�l!{��t�-����b:��	?���0�*�l!3�k��,8��������20�������~1|�lgF}�������R��2�-�hC�t
?ym��?/?E��������i/KW��am��U�
Y��F]3k2��v]��D!�B0�vy�����:�&fh�<�x���\M�f.B���+5V�2�?h�-&�
��8�HL\f�����������<�x~�qL[�+p�l��[\�+�V�>6��YA����e`����W`24+��^��O�p������[��\�t�:��R�y�@���K�I_���t�����C��t��u���#�m�P1E�e�#�����L�N�jb���a!���75���MU����!��&y%����MSpJ.B�]�k������X�h�\�Ku����}��1Y�e`�n��.��|�i��E.�kM��}��dpV��:+������#C!'KU��,e20+��U�,0��XA=Qf���(U�lJ��H������<�J�1WOe-����IT����t�9-w����@�T���
���3�������\��g�Nc��j���g��]�w��k�
v�=Q��g�k�����~�{�:<\�]b����{_W�g?������&�P������d�����:n\����X�����6�g�)�M�;���*������;���O9�����Hwj���jv;3+�"��}����`�����;]��@l��Z�[��EB��Y���l�T�W�_�:F~����a���_nOO�=I���8J����_���9�N�+��DU���O������ C��6h�����N[�Y_�Y����[���V6�'���n�
���1l�:�XU��{������O���q|?�&$Hb�Z�o���d�q�y8M����'�2�q.��ax�PK�_���
��PKc$JE@,3�T&T&Thumbnails/thumbnail.png�PNG


IHDR�����&IDATx���	XG�'J$ 	���(**�����@��-~�J��]<�>�h���V��j���ry�x`=���
�$����&�ewg	(���{��;;����3;�l�1Q�T�`��S��.r��`�����sX��
�d(���V.���8�G�����?���^����INL>H�Tr2�)zY�r���p�9(�B�����'`���{��%sJ>�S��37����d����60��~C��OF
W�����gA����#���rz�x��2��?����W���6�o>X�3~f������<�TW8��P�;�k�s ��8��q���M;Y����+���&��~��r��Au����Z����!n|u�����y�9���&���&�uk>V9��C�G�3�=�����X�ico����z�9�g��J�ECf?���b�[�O3[����u�Z{�//l�x�F�oF�k�'�y������5���U��v�=�W����W�m������}Z�h<9��a�a�B�X-����bVV6���O8������(��=��s�3#�1��jI����=j�����'g<4q�FY@�o����C��>�M�N� �����/��;�y`��'��z��]��m����[X�Oz��<��7h���1�F�1��g�����g/�p%<e��B���Jo�};�>&�O~����g��F�����������+�Z�[��BsPzf��.�Rb��53���eb���a��7�?����Ji~��D�nU�,_���E�	�L�
�@�_�;�%P�%����x������Qs�����\�V��5���9_|�����
_�P��b��|���Q����W|�U	�����B�X"�]��cW�,W��d��8��Z	Mf�l��v�k������X%W�:	�.�����{�G��@�KoV��c/	hP�{��yw��<iH;�%��]��~]��`��)�����-[�{�\����j�f����n��.�JY�����������M�j(<l^'����:��yJ�����������9��B	�.E�t�yr2R8�`��3�o�����K����"gZo�z���5�y�90��n�6�L��?=���}Rw�#��{��-{��M��M	��[�pm�����^�������g^]�����#L	�,k+7�)���������:����O����f-�f)h������.y+#3n���xr�p�����K�;l2O��Y�:��<m�*���^�����O%�_:��{�g�~<0�#V��cG�]�q
���W[�
����m%6������������
��y����0����N��z�hET��z��hoA����;�g=�����s���6�D�c�����^���s��Y�e�
��j$N5t�������~���R��,���y���3q%���8��5o9����������]7?=�{c�73���8�$r����r��MZj~�FT���}�6��7���mt���fP�����:��~�=���q���_�x����8��8����_�[�o��o���6���F|������~a���ov;�osrA�Y�Y�g�������K�\�NA��G�_�=.K���>���� ���I�d{�)	/�����M���a�]G���9�/�����<�>�[������������
�� g��S/���_]�<tA�1��_���.ea�Kd��/��^4}�����.����9-�Y5�U�d�y������|��*(y���@�j>��m�t�����y���k\�g2�F����7/=��aw�����&��en�n���@�T�6�k����g}��Yv?c��XZ��pS`*rTOf������Vn�����g����5��������9>�[����k�6s��5����N�{�V�����������O�3����>-����������F�+��%X�
�M*>�T"p��tF�����n���S���8�oS�7?�����R^���9P�|Y��NT��G��cycc�B}r�9���2�y_/���[p/����w����y�}??9b������#FyA�����l��50yQ��9k����:�>�]�{Vuh�5�?6������M�2+*�I���2��z��|�?�q�����������K�d,�?lA�U����M ��k�al����zk���Y�c��k�}��S�����u�����S����Z~�@mw�����&i��5;o���~y����A���W���'��	s��.o�kO����U������m��3��2��4(z)��|���i��3G@�X�;��0j�����S<�zb�{�.��o.���f0����<�>�����q���k U W[�����c\�g���{%TS�@��q�9��,Ihei�K�=&_.7�\��Q�������k���A�!���g�M�i����f����)���S���L�_����������O��Ndf�
�@����3����$�?���d>��
��2�U��4e�M/+���r�m�&������9Kj������2�~�����%B�v��r�Cg{���aA���\j{�R���P� �j��"&Lhb{m��_�h��������7�z�jIB�M���!�n�z���{����S��2�0i�\!�o���i:���[i�������~D�#�5_���@���>)�.��M�7���2�n������iS�i�f�N�s3��@h�����YlO:(��N��";�;�?��x�s!��;�	59`�9:�J�������t����?�L}�&������q����zc������t&5��WF���=K��
���]��{���/E[���5���E[G���v�6��_����\�O��go?r����\8��a$��e'�p�9�/�P�y�/<)�����)�m9(���k�%��K��c}_���[�AY�q�`�����|���e�-� ������&(�E@�F���!� �����7~�oW�?s� �(/r�7k�$�|�=�V.:�# �@@��s����5;�g�D90��C��*�@[�������'�����j����!�]We��E
	�:y�`H��[�g��$w�j�����-�`�MY��|�\��fZ���~��_��)�`��R�/g����'8�����s���X�S��B���!��-P�	�'M��!�1f���?&S>ZU#�+�A�$(+��Z����|��(������XN������s0��/������	/��Z��\q���~�S��|�h�
�����,n�(��9U;\q�*NP4��z�����G`�U�1�C�a8e�K�j����Q:�j��s|t��#>'�����������\{���ZFyV���%\q�J�9%
���k��]����E��8#�s|`���^���Z���$��>q�FF��]��d[������s��l�b��������p>D�@��#�A���';�2��v���.�w�w������wNMlu��2���e�%����Y�h����:��}�
���Pt3����yY^���
�/��Q�0Y��Bkg{6��sH�l�nv�h���N�������<b�e��e�&M�(��|�HF3g�K����5�r2��m8������.i,��T��"�����%�X����I�MuX�S�����)IL�	d�����L�ML
<��F��	��3D�R����������'��RU���������$c���DLRmMK��u&|�t���u�����Hk�.g�8���7|�b������+2�075-���P�����G_��dn
���M�3���e�� �E����`jy�N�z�rr{��d������<0���?�]�\z4��_lB��GF/��[5��{7���V��n�}k�pu8������e"o����V�g���@��+���]5r���3j��m���E�Hu>�rD�j��:�jd������>zz���K����<�����b;������5�����J��jq��S����&i���/�9����dR����>���io�J�6's��9�C��yj�k����D�V�Z�Yd�u��������5�SW��0��o��A[�hs�{Zy2zMR��}V5�#��
���wC�{�O��&Z�+�!0�%���g��t��b�\vf�"J������F����������������)��%O��y�����|��G���;h����H\�����9���������Q�������f����Ph�d���@q
�e�m'��C��
��S�����2���.1A_�O�?����������7���_�����"�z�*��x��^x�����J�Me"��!/������u������Uh�j��C����V=|Y���yz��^}�T�<M���?��p(�`]��C~��������z"�%��Z4`.���k��������������s�n�x
:6�>G���<\w�T�L Mh@![>����_��zcM�3�sb@��}mk����	�+-.�YZ���<�.��+��_+�&������wn:]PC����%�M�����<
�a�>����w�R��=bV����$P�l]�~hmk�R(���&����zs�I�J�T�tA4Uf<M��$��?U�������DA�<	��T5��y������������P��$S�P*��1�;��:�_=�|��A�+�����|�/�&Mz�/o]�g��aO��#�@^�M�_��dF-��Vb��I��l%���H�����*�%�m���#�%�]��
�Psc�o�����93+���|��0�;�c�%�V�Cy�7W��C,��������T-*���9j��?��9z������u3�M���,�j��J���Y-����K�k���s���JmAN�\{��� ��s5�?<��������U!��+�
�����D�t��`}�4-�6�I�j�e��d��?Ox�K�����KT�,���|����N�m�n#�����W��\qP��^U���[��y�8��-�k�Y��;!)J�,?�o���I�bq��O���
*�?$U�5�V&�/���cx��y\��=,o�3vE�M���C�2s�����L|:7#KQj�MO<Xg���m�N�����=��6M(��=�|���&�9f����rnVNF���0u�����7�Bqco��T���������`f��.vbUhNT�������mB�y�Q5;S<��T]L����^�S|R��h��r�.��{�O�M����M�,N��-�����P#"Bs��m�Cd��A7�Q2u1=%x�P�����2��nK
cj^����i���	t�<�v.�Y��X��nq�h�@�!'�M2���9�GE��8�����&}�����>�i����c����taP)9�4����qk���9�n��P<�������)��EbI��}N���P�KM��_�nz1e�LC�b�����-�eX�;mS����{���Y��R��zO �N������;���%N<���������;���<��V��]y��g����w�/���s����+�E�4[J
�TD��l

�Q(v���?��jR�8�u;��$:5)�'��b8�9���R{�h��I[J��E�v�����0�T�I[r����Z�N���S
���bd�4� ����=�����rJ�SY���G�e�\
�)��k��"��H�HM��������%��:���9>�=��Z?G�G/���j;��������>��L~;7�h����{��
�vmN`��z����zl��a
f������OO"�k&l� ����!l{���H�{u-�����Ig��6�c~�q���������E�;=�r>��	���q��s*�x���0�j�����D����"#�����'��:�f�������0;/��lz��$2���������.�t��3i�	���Ee&��Mq,�_��i���zO���wn�	��tx��.b��nG�?�o�S�v���y+�m��b����*�=X�g��JZ�8�!	��L�a;�4��s9�������l�p�,��:������#������<��@��CFNc��	�-����I��U7��n���������J�}h�#�IM��v����"��d����{�<i�)�kb�4����,��}��
��vw|~�.�EG����1!+�?���}JJ�Zg�{&��by�hP�p*�!g�g�V��{�s���1����~�[���Y�*T�x|�;y[}�(��l�F�����.1��"�������O�L���������v8:�u��]���x��.g852j�$_��y=���J�$����S���7.9�)>H�����w#�nE���-�����������Fqpu�S��<�h��	�:a��Jb��I�kj
�M���t$5��;/��,J���������{�X6�l7� �?���-���d�8���.�i)����t���^����4^�w�����sB[��T�(~c�������������/3��X)��@<�|��)v��fWkh���^���,�����6�����<klp�jk���Cuj���G�q�9�:.�1"��UJ�/��_�P�������v��SS�5�P�?I}i�z7�������wi����)��!�������m��A
BYr�z��P�f����[j��������vo���}���������B#�,�&W���\<��6�����Y���/~���Z��
��9���J]�v^}B��=���#zF�mzNW�y��*�q[
Z	��X%AR���w���>��l
�f�$�z�:#y~�T�5����*�3�����cS�k�0��W���w���/*����v�g���)E�����vm��L%@������LE�t�B%�>�Ql3������f�����N�������+K���1�l<������>�������[B�:��[���hL�`�+qML�Y��r�UR�b��0s;�J#F��{rz������DS�]���������V�hWv����a+M,�g���Q���\�[��J }�G�>�2o��!��
��k!�M�����d���M0#�~��}�j���%.�KU�/vm�Z���2��j43���^���#�+�Mi!�I��6�!�C#(��
)��:�siY��q{�L��@��HG���u���~=��'�mu�� '.��@�5�,ZS��k����v�!���@p�(�9P�s � �@@A�����9
r�(�9P�s � �@@A�����9
r�(�9P�s � �@@A�����9
r�(�9P�s � �@@A�����9
r�(�9P�s � �@@A���Bu������;���[�{
���XB����L�h�����F�/o��v/3=�qrV�M���"_)/����m����D>E��]�)���Nd�6����Q^���8q��Uy_����L��X2Z�`���a�\�@��m�'!����}�����a �R���I?dl �5$gCl��e/Q�R9
r��A*��Mr,�C+�Y����uo.��������v�<��[��bl�r�3�Z����+1�R����S2����RM�Xz.�v�������"���4��fp��'(k��dH��1�i,f���W������@��e�r9;cC���	��8C�.���ky���l��r���8T��u�4<�Gx@kA���������a@�)��!��Q�@nUH�<�x�0�Y�
�F���s��|�{�"�qy����R�*���v���ppC|*���Ow���`	l�A
)
U!5
�Q5�u�9��U�F!�tZ�i�Ua�0�}N|��$��NPR��g���j�B�"�!��O���(�9Pt�2�1�]@SP��dD0�_)��.�y(_�#�0"���P���/����3�6���x]�!0��F8`�AX��,�{���&��R=�F�h)������:�+~c�D8���'4����-������/���?�����?�#���}��O����o�69r�5\pP@��M('`��Y�H��@?!��,��e�5�Y��Q.��K8�g"��t]�1��\0������Y�2f�xa���s|Z�`,��
���,��
8���b��L��\��e>W��T3�@��P�R1.�R�*X^�8��;���+�=����!�F�E,��"�q��@�
���Y�"p��	�=_���.�#D�����!C�j�$G!M�
�%2�a��q��2>�3��f�d���}V(�T���H�n�Q�
��~�jJg�P��[fz��`T�`	lH��K�`�VP�[
��H�1��
)�������TNE��$��J���f�����/�B�q�`�R��)���*kF�N
���SH��3(B$��4�Z:�c�B���9(��C!�k��E��Z�Z5PJd��#(�F�8ej���t���g�O�KDKY��R��%\� ��,��
8hZA@A
i�N�*��Y0�!�R!���%�*0�R�BX����!�PS;Y7"�NUi�G�v�����*�M(
)E*����=A������K$�`�i���3��_�ua��(T��
)�B:�{��X���
)e�3�4P�EKY��R��%\���Xf
D����(�9P�s � �@@A�����9
r�(�9P�s � �@@A�����9
r�(�9P�s � �@@A�����9
r�(�9P�s � �@@A�����9
r�(�9P�}nx�]C����Tqh����E�-�a�va+����=7C��~a�P�#K)U{����>�@��N��L��^�N�����V�q���n����l@9H�C���	,����c�7��S?0C���#�T�s�~	�4@wg�
���9�1�!A�a1���� K|w��c�efe,0�>������V&����\6�Y:������,����h)k���
9
r�(T��~eT<
���`m��2�1\=3���U2�~+����k���t�����.r��E[��'�X1���r���f���T�����w�fQ���"L(��>��BOO�\a�����2��=����_�Xn�P�4���C��r��d%�sglh�4�I�en����q���EfdC��hG����x����>�Z���e��n����r`��X�(3
�f��)�Q>7��CIc��i��0��������[.X�`nR��lU����X-1�����T�W+��<�q�.����� ��D��_�����J7
�
��0y�*�Bp�sT�W���*��TC�dD5�AcQi�g��@�s � �@@1����:�41�:�W�|m��]
�7��?��i����+f��[�t<�#�M�`�(�H@|:����f�����R�k0��c��AO����Z�"� �@@������'�|��	�������
�#'�}c�.,�>�o���������0���>�a�#w�Q����!`�5�u�0�1�;P��O�+K-e�g
z,?sB�.������Hhs]�bLf�����60vhE���1A5��.���K�`��RJ���:DIEND�B`�PKc$JEConfigurations2/images/Bitmaps/PKc$JEConfigurations2/popupmenu/PKc$JEConfigurations2/toolpanel/PKc$JEConfigurations2/statusbar/PKc$JEConfigurations2/progressbar/PKc$JEConfigurations2/toolbar/PKc$JEConfigurations2/menubar/PKc$JE'Configurations2/accelerator/current.xmlPKPKc$JEConfigurations2/floater/PKc$JE
styles.xml�Ymo�6��_a(@��);�������KQd�`$J&J�I�q���H��V��tK���=w��^��{s�����Iy������<�1���������
���IB#��yTf$W�TF��s������cI�:��k�yAr��n���U�b�MU7���"Oj����t��t�����OU�X�iS=�S��$��G<+��O���7�V�b��~���/�\�h�Z���:����`G�0��I��/��fD���4�I)/�G"&�+��j!�lW��4CM�V}�����KG�m��\g�.��xz�\�M���H~�������u%���4��H�b�6+tS�s��j���
�e\��������UD4��Ix�Y�"����n����.y�z����"A
.�#�L?� :K��[���V�RMEB��%�����w��/Zg���������R
�x�GZ������n�K/�WB��:HpD��DL�7U+��Y�����~�C�YyF��^FgU1�{��%Mu-�S�A�h��Jy��{�K�����s�DTE��	}"��
�m�y~wtu����}�)\�dv���W���]X��&w��d/a���<�������X-����cY�$�%�g k��d���c��X�T�b�ZR(
�S%4X��S�p�g(���Qv�n���!�*�������Fr��2WX����*�pt�|0�h�Y�����V����m}�2�������[���D�Ygf ��M5_�cF+UL>^4��2��4W$���TI�q4`���J!`*>�ZW������R��x�#g�M;c��Zx/��Z����ri��_0�c���;�w+M/3�����w?5(�X��d���$	�G��mjl�;���?�*.\��1�f�f%y��U��m\��`uD�V���G�����U�x���3���){���E���'�4��;-��������2T��H�0���g�e/B�b�.��dOc=o�R�D]������8�	M�0SAm�_��������s,���_	���Mb���U�C�Is_�R���"k�k�u0��r_��l���
!?�x(��������~������*�e��#�xVHm:��T��U��P�_}��3��������F~{�)����������"[x�>)���38 6�z<�d�;^����sH_'a����I�?���,c����D��E�n�>r���`��+4�������Ip�b���`B�����i�#��~i9�Z�[}��.�c��o0�y���Z8��o�<�'�E�?.:
��)��Io��~!}��:_�h������<�ZfQ[:55�`�����7�����[n	������
�.�+E'���4���J�0�������Q��~���10\X��k���P/��B{_7v"��^d����L��`�^W8EkB��7�@�<��Lu��EF'*���7S<tV���|�0��A�B��g�0����8�C�������5���*\���j�o�=4��h���?PK&��L��PKc$JEMETA-INF/manifest.xml�S�n� ��+"����	5�a����F�	��j�~I����I�����=c��������El��|���������E���M0�: ��������QGC�4������hKd�5_�J�2��Uu����z���%;@�L�C�F������ju�V��������t��$/�]��N��PB�e���jl�6[6r�}$)�8)z8����������A%�o�%|�q��C9e��r��5b����u��2�����l��}_1*8�������������7��7�~PK����PKc$JE�l9�..mimetypePKc$JE�]�'RRTmeta.xmlPKc$JE�-`�,:#�settings.xmlPKc$JE�_���
��2content.xmlPKc$JE@,3�T&T&Thumbnails/thumbnail.pngPKc$JE�<Configurations2/images/Bitmaps/PKc$JE�<Configurations2/popupmenu/PKc$JE=Configurations2/toolpanel/PKc$JE9=Configurations2/statusbar/PKc$JEq=Configurations2/progressbar/PKc$JE�=Configurations2/toolbar/PKc$JE�=Configurations2/menubar/PKc$JE'>Configurations2/accelerator/current.xmlPKc$JEn>Configurations2/floater/PKc$JE&��L��
�>styles.xmlPKc$JE�����DMETA-INF/manifest.xmlPK6F
#77Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#76)
Re: Wait free LW_SHARED acquisition - v0.9

Hi,

On 2014-10-10 10:13:03 +0530, Amit Kapila wrote:

I have done few performance tests for above patches and results of
same is as below:

Cool, thanks.

Performance Data
------------------------------
IBM POWER-7 16 cores, 64 hardware threads
RAM = 64GB
max_connections =210
Database Locale =C
checkpoint_segments=256
checkpoint_timeout =35min
shared_buffers=8GB
Client Count = number of concurrent sessions and threads (ex. -c 8 -j 8)
Duration of each individual run = 5mins
Test type - read only pgbench with -M prepared
Other Related information about test
a. This is the data for median of 3 runs, the detailed data of individual
run
is attached with mail.
b. I have applied both the patches to take performance data.

Scale Factor - 100

Patch_ver/Client_count 1 8 16 32 64 128 HEAD 13344 106921 196629 295123
377846 333928 PATCH 13662 106179 203960 298955 452638 465671

Scale Factor - 3000

Patch_ver/Client_count 8 16 32 64 128 160 HEAD 86920 152417 231668
280827 257093 255122 PATCH 87552 160313 230677 276186 248609 244372

Observations
----------------------
a. The patch performs really well (increase upto ~40%) incase all the
data fits in shared buffers (scale factor -100).
b. Incase data doesn't fit in shared buffers, but fits in RAM
(scale factor -3000), there is performance increase upto 16 client count,
however after that it starts dipping (in above config unto ~4.4%).

Hm. Interesting. I don't see that dip on x86.

The above data shows that the patch improves performance for cases
when there is shared LWLock contention, however there is a slight
performance dip in case of Exclusive LWLocks (at scale factor 3000,
it needs exclusive LWLocks for buf mapping tables). Now I am not
sure if this is the worst case dip or under similar configurations the
performance dip can be higher, because the trend shows that dip is
increasing with more client counts.

Brief Analysis of code w.r.t performance dip
---------------------------------------------------------------------
Extra Instructions w.r.t Head in Acquire Exclusive lock path
a. Attempt lock twice
b. atomic operations for nwaiters in LWLockQueueSelf() and
LWLockAcquireCommon()
c. Now we need to take spinlock twice, once for self queuing and then
again for setting releaseOK.
d. few function calls and some extra checks

Hm. I can't really see the number of atomics itself matter - a spinning
lock will do many more atomic ops than this. But I wonder whether we
could get rid of the releaseOK lock. Should be quite possible.

Now probably these shouldn't matter much in case backend needs to
wait for other Exclusive locker, but I am not sure what else could be
the reason for dip in case we need to have Exclusive LWLocks.

Any chance to get a profile?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#78Andres Freund
andres@2ndquadrant.com
In reply to: Robert Haas (#72)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-08 20:07:35 -0400, Robert Haas wrote:

On Wed, Oct 8, 2014 at 2:04 PM, Andres Freund <andres@2ndquadrant.com> wrote:

So, what makes it work for me (among other unrelated stuff) seems to be
the following in .gdbinit, defineing away some things that gdb doesn't
handle:
macro define __builtin_offsetof(T, F) ((int) &(((T *) 0)->F))
macro define __extension__
macro define AssertVariableIsOfTypeMacro(x, y) ((void)0)

Additionally I have "-ggdb -g3" in CFLAGS. That way gdb knows about
postgres' macros. At least if you're in the right scope.

As an example, the following works:
(gdb) p dlist_is_empty(&BackendList) ? NULL : dlist_head_element(Backend, elem, &BackendList)

Ah, cool. I'll try that.

If that works for you, should we put it somewhere in the docs? If so,
where?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#79Andres Freund
andres@2ndquadrant.com
In reply to: Robert Haas (#71)
4 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.9

Hi Robert,

On 2014-10-08 16:01:53 -0400, Robert Haas wrote:

[ comment fixes ]

Thanks, I've incorporated these + a bit more.

Could you otherwise make sense of the explanation and the algorithm?

+ /* yipeyyahee */

Although this will be clear to individuals with a good command of
English, I suggest avoiding such usages.

I've removed them with a heavy heart. These are heartfelt emotions from
getting the algorithm to work.... (:P)

I've attached these fixes + the removal of spinlocks around releaseOK as
follow up patches. Obviously they'll be merged into the other patch, but
sounds useful to be able see them separately.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-Convert-the-PGPROC-lwWaitLink-list-into-a-dlist-inst.patchtext/x-patch; charset=us-asciiDownload
>From 6885a15cc6f2e193ff575a4463d90ad252d74f5e Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Tue, 7 Oct 2014 15:32:34 +0200
Subject: [PATCH 1/4] Convert the PGPROC->lwWaitLink list into a dlist instead
 of open coding it.

Besides being shorter and much easier to read it:

* changes the logic in LWLockRelease() to release all shared lockers
  when waking up any. This can yield some significant performance
  improvements - and the fairness isn't really much worse than before,
  as we always allowed new shared lockers to jump the queue.

* adds a memory pg_write_barrier() in the wakeup paths between
  dequeuing and unsetting ->lwWaiting. That was always required on
  weakly ordered machines, but f4077cda2 made it more urgent.

Author: Andres Freund
---
 src/backend/access/transam/twophase.c |   1 -
 src/backend/storage/lmgr/lwlock.c     | 151 +++++++++++++---------------------
 src/backend/storage/lmgr/proc.c       |   2 -
 src/include/storage/lwlock.h          |   5 +-
 src/include/storage/proc.h            |   3 +-
 5 files changed, 60 insertions(+), 102 deletions(-)

diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index d5409a6..6401943 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -389,7 +389,6 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 	proc->roleId = owner;
 	proc->lwWaiting = false;
 	proc->lwWaitMode = 0;
-	proc->lwWaitLink = NULL;
 	proc->waitLock = NULL;
 	proc->waitProcLock = NULL;
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 9fe6855..e6f9158 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -35,6 +35,7 @@
 #include "miscadmin.h"
 #include "pg_trace.h"
 #include "replication/slot.h"
+#include "storage/barrier.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
@@ -115,9 +116,9 @@ inline static void
 PRINT_LWDEBUG(const char *where, const LWLock *lock)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): excl %d shared %d head %p rOK %d",
+		elog(LOG, "%s(%s %d): excl %d shared %d rOK %d",
 			 where, T_NAME(lock), T_ID(lock),
-			 (int) lock->exclusive, lock->shared, lock->head,
+			 (int) lock->exclusive, lock->shared,
 			 (int) lock->releaseOK);
 }
 
@@ -475,8 +476,7 @@ LWLockInitialize(LWLock *lock, int tranche_id)
 	lock->exclusive = 0;
 	lock->shared = 0;
 	lock->tranche = tranche_id;
-	lock->head = NULL;
-	lock->tail = NULL;
+	dlist_init(&lock->waiters);
 }
 
 
@@ -615,12 +615,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = mode;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -836,12 +831,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -997,13 +987,8 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		 */
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-
 		/* waiters are added to the front of the queue */
-		proc->lwWaitLink = lock->head;
-		if (lock->head == NULL)
-			lock->tail = proc;
-		lock->head = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -1079,9 +1064,10 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 void
 LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 {
-	PGPROC	   *head;
-	PGPROC	   *proc;
-	PGPROC	   *next;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+
+	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
 	SpinLockAcquire(&lock->mutex);
@@ -1096,24 +1082,16 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	 * See if there are any LW_WAIT_UNTIL_FREE waiters that need to be woken
 	 * up. They are always in the front of the queue.
 	 */
-	head = lock->head;
-
-	if (head != NULL && head->lwWaitMode == LW_WAIT_UNTIL_FREE)
+	dlist_foreach_modify(iter, &lock->waiters)
 	{
-		proc = head;
-		next = proc->lwWaitLink;
-		while (next && next->lwWaitMode == LW_WAIT_UNTIL_FREE)
-		{
-			proc = next;
-			next = next->lwWaitLink;
-		}
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-		/* proc is now the last PGPROC to be released */
-		lock->head = next;
-		proc->lwWaitLink = NULL;
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+			break;
+
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 	}
-	else
-		head = NULL;
 
 	/* We are done updating shared state of the lock itself. */
 	SpinLockRelease(&lock->mutex);
@@ -1121,13 +1099,13 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, &wakeup)
 	{
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 }
 
@@ -1138,10 +1116,12 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 void
 LWLockRelease(LWLock *lock)
 {
-	PGPROC	   *head;
-	PGPROC	   *proc;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
 	int			i;
 
+	dlist_init(&wakeup);
+
 	PRINT_LWDEBUG("LWLockRelease", lock);
 
 	/*
@@ -1177,58 +1157,39 @@ LWLockRelease(LWLock *lock)
 	 * if someone has already awakened waiters that haven't yet acquired the
 	 * lock.
 	 */
-	head = lock->head;
-	if (head != NULL)
+	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
 	{
-		if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
+		/*
+		 * Remove the to-be-awakened PGPROCs from the queue.
+		 */
+		bool		releaseOK = true;
+		bool		wokeup_somebody = false;
+
+		dlist_foreach_modify(iter, &lock->waiters)
 		{
-			/*
-			 * Remove the to-be-awakened PGPROCs from the queue.
-			 */
-			bool		releaseOK = true;
+			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-			proc = head;
+			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+				continue;
 
-			/*
-			 * First wake up any backends that want to be woken up without
-			 * acquiring the lock.
-			 */
-			while (proc->lwWaitMode == LW_WAIT_UNTIL_FREE && proc->lwWaitLink)
-				proc = proc->lwWaitLink;
+			dlist_delete(&waiter->lwWaitLink);
+			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 
 			/*
-			 * If the front waiter wants exclusive lock, awaken him only.
-			 * Otherwise awaken as many waiters as want shared access.
+			 * Prevent additional wakeups until retryer gets to
+			 * run. Backends that are just waiting for the lock to become
+			 * free don't retry automatically.
 			 */
-			if (proc->lwWaitMode != LW_EXCLUSIVE)
+			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
 			{
-				while (proc->lwWaitLink != NULL &&
-					   proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
-				{
-					if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
-						releaseOK = false;
-					proc = proc->lwWaitLink;
-				}
-			}
-			/* proc is now the last PGPROC to be released */
-			lock->head = proc->lwWaitLink;
-			proc->lwWaitLink = NULL;
-
-			/*
-			 * Prevent additional wakeups until retryer gets to run. Backends
-			 * that are just waiting for the lock to become free don't retry
-			 * automatically.
-			 */
-			if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
 				releaseOK = false;
+				wokeup_somebody = true;
+			}
 
-			lock->releaseOK = releaseOK;
-		}
-		else
-		{
-			/* lock is still held, can't awaken anything */
-			head = NULL;
+			if(waiter->lwWaitMode == LW_EXCLUSIVE)
+				break;
 		}
+		lock->releaseOK = releaseOK;
 	}
 
 	/* We are done updating shared state of the lock itself. */
@@ -1239,15 +1200,15 @@ LWLockRelease(LWLock *lock)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, &wakeup)
 	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 		LOG_LWDEBUG("LWLockRelease", T_NAME(lock), T_ID(lock),
 					"release waiter");
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 
 	/*
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index ea88a24..a4789fc 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -372,7 +372,6 @@ InitProcess(void)
 		MyPgXact->vacuumFlags |= PROC_IS_AUTOVACUUM;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
@@ -535,7 +534,6 @@ InitAuxiliaryProcess(void)
 	MyPgXact->vacuumFlags = 0;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 02c8f1a..fea5d33 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -14,6 +14,7 @@
 #ifndef LWLOCK_H
 #define LWLOCK_H
 
+#include "lib/ilist.h"
 #include "storage/s_lock.h"
 
 struct PGPROC;
@@ -50,9 +51,7 @@ typedef struct LWLock
 	char		exclusive;		/* # of exclusive holders (0 or 1) */
 	int			shared;			/* # of shared holders (0..MaxBackends) */
 	int			tranche;		/* tranche ID */
-	struct PGPROC *head;		/* head of list of waiting PGPROCs */
-	struct PGPROC *tail;		/* tail of list of waiting PGPROCs */
-	/* tail is undefined when head is NULL */
+	dlist_head	waiters;		/* list of waiting PGPROCs */
 } LWLock;
 
 /*
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index c23f4da..38758d3 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -15,6 +15,7 @@
 #define _PROC_H_
 
 #include "access/xlogdefs.h"
+#include "lib/ilist.h"
 #include "storage/latch.h"
 #include "storage/lock.h"
 #include "storage/pg_sema.h"
@@ -104,7 +105,7 @@ struct PGPROC
 	/* Info about LWLock the process is currently waiting for, if any. */
 	bool		lwWaiting;		/* true if waiting for an LW lock */
 	uint8		lwWaitMode;		/* lwlock mode being waited for */
-	struct PGPROC *lwWaitLink;	/* next waiter for same LW lock */
+	dlist_node	lwWaitLink;		/* position in LW lock wait list */
 
 	/* Info about lock the process is currently waiting for, if any. */
 	/* waitLock and waitProcLock are NULL if not currently waiting. */
-- 
1.8.3.251.g1462b67

0002-Wait-free-LW_SHARED-LWLock-acquisition.patchtext/x-patch; charset=us-asciiDownload
>From cbd80574f5dfe631f492dd497700dd05b211b3e3 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Thu, 18 Sep 2014 16:14:16 +0200
Subject: [PATCH 2/4] Wait free LW_SHARED LWLock acquisition.

The old LWLock implementation had the problem that concurrent shared
lock acquisitions required exclusively acquiring a spinlock. Often
that could lead to acquirers waiting behind the spinlock, even if the
actual LWLock was free.

The new implementation doesn't acquire the spinlock when acquiring the
lock itself. Instead the new atomic operations are used to atomically
manipulate the state. Only the waitqueue, only used in the slow path,
is still protected by the spinlock. Check lwlock.c's header for an
explanation about the used algorithm.

For some common workloads on larger machines this can yield
significant performance improvements. Particularly in read mostly
workloads.

Reviewed-By: Amit Kapila
Author: Andres Freund

Discussion: 20130926225545.GB26663@awork2.anarazel.de
---
 src/backend/storage/lmgr/lwlock.c | 968 +++++++++++++++++++++++++++-----------
 src/include/storage/lwlock.h      |   9 +-
 2 files changed, 704 insertions(+), 273 deletions(-)

diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index e6f9158..b10c121 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -24,6 +24,78 @@
  * IDENTIFICATION
  *	  src/backend/storage/lmgr/lwlock.c
  *
+ * NOTES:
+ *
+ * This used to be a pretty straight forward reader-writer lock
+ * implementation, in which the internal state was protected by a
+ * spinlock. Unfortunately the overhead of taking the spinlock proved to be
+ * too high for workloads/locks that were locked in shared mode very
+ * frequently. Often we were spinning in the (obviously exlusive) spinlock,
+ * while trying to acquire a shared lock that was actually free.
+ *
+ * Thus a new implementation was devised that provides wait-free shared lock
+ * acquiration for locks that aren't exclusively locked.
+ *
+ * The basic idea is to have a single atomic variable 'lockcount' instead of
+ * the formerly separate shared and exclusive counters and to use an atomic
+ * increment to acquire the lock. That's fairly easy to do for rw-spinlocks,
+ * but a lot harder for something like LWLocks that want to wait in the OS.
+ *
+ * For exlusive lock acquisition we use an atomic compare-and-exchange on the
+ * lockcount variable swapping in EXCLUSIVE_LOCK/1<<31-1/0x7FFFFFFF if and only
+ * if the current value of lockcount is 0. If the swap was not successfull, we
+ * have to wait.
+ *
+ * For shared lock acquisition we use an atomic add to the lockcount variable
+ * to add 1. If the value is bigger than EXCLUSIVE_LOCK we know that somebody
+ * actually has an exclusive lock, and we back out by atomically decrementing
+ * by 1 again. If so, we have to wait for the exlusive locker to release the
+ * lock.
+ *
+ * To release the lock we use an atomic decrement to release the lock. If the
+ * new value is zero (we get that atomically), we know we have to release
+ * waiters.
+ *
+ * The attentive reader probably might have noticed that naively doing the
+ * above has two glaring race conditions:
+ *
+ * 1) too-quick-for-queueing: We try to lock using the atomic operations and
+ * notice that we have to wait. Unfortunately until we have finished queuing,
+ * the former locker very well might have already finished it's work. That's
+ * problematic because we're now stuck waiting inside the OS.
+ *
+ * 2) spurious failed locks: Due to the logic of backing out of shared
+ * locks after we unconditionally added a 1 to lockcount, we might have
+ * prevented another exclusive locker from getting the lock:
+ *   1) Session A: LWLockAcquire(LW_EXCLUSIVE) - success
+ *   2) Session B: LWLockAcquire(LW_SHARED) - lockcount += 1
+ *   3) Session B: LWLockAcquire(LW_SHARED) - oops, bigger than EXCLUSIVE_LOCK
+ *   4) Session A: LWLockRelease()
+ *   5) Session C: LWLockAcquire(LW_EXCLUSIVE) - check if lockcount = 0, no. wait.
+ *   6) Session B: LWLockAcquire(LW_SHARED) - lockcount -= 1
+ *   7) Session B: LWLockAcquire(LW_SHARED) - wait
+ *
+ * So we'd now have both B) and C) waiting on a lock that nobody is holding
+ * anymore. Not good.
+ *
+ * To mitigate those races we use a two phased attempt at locking:
+ *   Phase 1: Try to do it atomically, if we succeed, nice
+ *   Phase 2: Add us too the waitqueue of the lock
+ *   Phase 3: Try to grab the lock again, if we succeed, remove ourselves from
+ *            the queue
+ *   Phase 4: Sleep till wakeup, goto Phase 1
+ *
+ * This protects us against both problems from above:
+ * 1) Nobody can release too quick, before we're queued, since after Phase 2
+ *    we're already queued.
+ * 2) If somebody spuriously got blocked from acquiring the lock, they will
+ *    get queued in Phase 2 and we can wake them up if neccessary or they will
+ *    have gotten the lock in Phase 3.
+ *
+ * There above algorithm only works for LWLockAcquire, not directly for
+ * LWLockAcquireConditional where we don't want to wait. In that case we just
+ * need to retry acquiring the lock until we're sure we didn't disturb anybody
+ * in doing so.
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -35,7 +107,6 @@
 #include "miscadmin.h"
 #include "pg_trace.h"
 #include "replication/slot.h"
-#include "storage/barrier.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
@@ -50,6 +121,11 @@
 /* We use the ShmemLock spinlock to protect LWLockAssign */
 extern slock_t *ShmemLock;
 
+#define EXCLUSIVE_LOCK (((uint32) 1) << (31 - 1))
+
+/* Must be greater than MAX_BACKENDS - which is 2^23-1, so we're fine. */
+#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)
+
 /*
  * This is indexed by tranche ID and stores metadata for all tranches known
  * to the current backend.
@@ -80,8 +156,14 @@ static LWLockTranche MainLWLockTranche;
  */
 #define MAX_SIMUL_LWLOCKS	200
 
+typedef struct LWLockHandle
+{
+	LWLock *lock;
+	LWLockMode	mode;
+} LWLockHandle;
+
 static int	num_held_lwlocks = 0;
-static LWLock *held_lwlocks[MAX_SIMUL_LWLOCKS];
+static LWLockHandle held_lwlocks[MAX_SIMUL_LWLOCKS];
 
 static int	lock_addin_request = 0;
 static bool lock_addin_request_allowed = true;
@@ -100,8 +182,10 @@ typedef struct lwlock_stats
 {
 	lwlock_stats_key key;
 	int			sh_acquire_count;
+	int			sh_attempt_backout;
 	int			ex_acquire_count;
 	int			block_count;
+	int			dequeue_self_count;
 	int			spin_delay_count;
 }	lwlock_stats;
 
@@ -113,24 +197,32 @@ static lwlock_stats lwlock_stats_dummy;
 bool		Trace_lwlocks = false;
 
 inline static void
-PRINT_LWDEBUG(const char *where, const LWLock *lock)
+PRINT_LWDEBUG(const char *where, LWLock *lock, LWLockMode mode)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): excl %d shared %d rOK %d",
+	{
+		uint32 lockcount = pg_atomic_read_u32(&lock->lockcount);
+
+		elog(LOG, "%d: %s(%s %d): excl %u shared %u waiters %u rOK %d\n",
+			 MyProcPid,
 			 where, T_NAME(lock), T_ID(lock),
-			 (int) lock->exclusive, lock->shared,
+			 lockcount >= EXCLUSIVE_LOCK,
+			 lockcount & SHARED_LOCK_MASK,
+			 pg_atomic_read_u32(&lock->nwaiters),
 			 (int) lock->releaseOK);
+	}
 }
 
 inline static void
-LOG_LWDEBUG(const char *where, const char *name, int index, const char *msg)
+LOG_LWDEBUG(const char *where, LWLock *lock, const char *msg)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): %s", where, name, index, msg);
+		elog(LOG, "%s(%s %d): %s", where, T_NAME(lock), T_ID(lock), msg);
 }
+
 #else							/* not LOCK_DEBUG */
-#define PRINT_LWDEBUG(a,b)
-#define LOG_LWDEBUG(a,b,c,d)
+#define PRINT_LWDEBUG(a,b,c)
+#define LOG_LWDEBUG(a,b,c)
 #endif   /* LOCK_DEBUG */
 
 #ifdef LWLOCK_STATS
@@ -192,11 +284,12 @@ print_lwlock_stats(int code, Datum arg)
 	while ((lwstats = (lwlock_stats *) hash_seq_search(&scan)) != NULL)
 	{
 		fprintf(stderr,
-			  "PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u\n",
+				"PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u backout %u dequeue self %u\n",
 				MyProcPid, LWLockTrancheArray[lwstats->key.tranche]->name,
 				lwstats->key.instance, lwstats->sh_acquire_count,
 				lwstats->ex_acquire_count, lwstats->block_count,
-				lwstats->spin_delay_count);
+				lwstats->spin_delay_count, lwstats->sh_attempt_backout,
+				lwstats->dequeue_self_count);
 	}
 
 	LWLockRelease(&MainLWLockArray[0].lock);
@@ -224,8 +317,10 @@ get_lwlock_stats_entry(LWLock *lock)
 	if (!found)
 	{
 		lwstats->sh_acquire_count = 0;
+		lwstats->sh_attempt_backout = 0;
 		lwstats->ex_acquire_count = 0;
 		lwstats->block_count = 0;
+		lwstats->dequeue_self_count = 0;
 		lwstats->spin_delay_count = 0;
 	}
 	return lwstats;
@@ -473,12 +568,309 @@ LWLockInitialize(LWLock *lock, int tranche_id)
 {
 	SpinLockInit(&lock->mutex);
 	lock->releaseOK = true;
-	lock->exclusive = 0;
-	lock->shared = 0;
+	pg_atomic_init_u32(&lock->lockcount, 0);
+	pg_atomic_init_u32(&lock->nwaiters, 0);
 	lock->tranche = tranche_id;
 	dlist_init(&lock->waiters);
 }
 
+/*
+ * Internal function that tries to atomically acquire the lwlock in the passed
+ * in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * callers job.
+ *
+ * Returns true if the lock isn't free and we need to wait.
+ *
+ * When acquiring shared locks it's possible that we disturb an exclusive
+ * waiter. If that's a problem for the specific user, pass in a valid pointer
+ * for 'potentially_spurious'. Its value will be set to true if we possibly
+ * did so. The caller then has to handle that scenario.
+ */
+static bool
+LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)
+{
+	uint32		oldstate;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	AssertArg(mode == LW_EXCLUSIVE || mode == LW_SHARED);
+
+	if (potentially_spurious != NULL)
+		*potentially_spurious = false;
+
+	if (mode == LW_EXCLUSIVE)
+	{
+		uint32 expected;
+
+		/*
+		 * First check whether the variable is free without a atomic
+		 * operation; it's often quite a bit cheaper for contended
+		 * locks. Doing so can cause a superflous shared-exclusive cacheline
+		 * transition, but benchmarks show that it's still worth doing so.
+		 */
+		expected = pg_atomic_read_u32(&lock->lockcount);
+
+		if (expected != 0)
+			return true;
+		else if (!pg_atomic_compare_exchange_u32(&lock->lockcount,
+												 &expected, EXCLUSIVE_LOCK))
+		{
+			/*
+			 * No can do. Between the pg_atomic_read() above and the CAS
+			 * somebody else acquired the lock.
+			 */
+			return true;
+		}
+		else
+		{
+			/* yipeyyahee */
+#ifdef LOCK_DEBUG
+			lock->owner = MyProc;
+#endif
+			Assert(expected == 0);
+			return false;
+		}
+	}
+	else
+	{
+		/*
+		 * If the caller is interested in spurious locks, do an unlocked check
+		 * first.  This is useful if potentially spurious results have a
+		 * noticeable cost.
+		 */
+		if (potentially_spurious != NULL &&
+			pg_atomic_read_u32(&lock->lockcount) >= EXCLUSIVE_LOCK)
+			return true;
+
+		/*
+		 * Acquire the share lock unconditionally using an atomic addition. We
+		 * might have to back out again if it turns out somebody else has an
+		 * exclusive lock.
+		 */
+		oldstate = pg_atomic_fetch_add_u32(&lock->lockcount, 1);
+
+		if (oldstate >= EXCLUSIVE_LOCK)
+		{
+			/*
+			 * Ok, somebody else holds the lock exclusively. We need to back
+			 * away from the shared lock, since we don't actually hold it right
+			 * now.  Since there's a window between lockcount += 1 and lockcount
+			 * -= 1, the previous exclusive locker could have released and
+			 * another exclusive locker could have seen our +1. We need to
+			 * signal that to the upper layers so they can deal with the race
+			 * condition.
+			 */
+
+			/*
+			 * XXX: We could check the return value if (double_check), it's
+			 * not spurious if still exclusively locked. Should work for the
+			 * current callers. There might be some cases where ABA like
+			 * problems exist.
+			 */
+			pg_atomic_fetch_sub_u32(&lock->lockcount, 1);
+
+
+			if (potentially_spurious != NULL)
+				*potentially_spurious = true;
+#ifdef LWLOCK_STATS
+			lwstats->sh_attempt_backout++;
+#endif
+			return true;
+		}
+		else
+		{
+			/* yipeyyahee */
+			return false;
+		}
+	}
+
+	pg_unreachable();
+}
+
+/*
+ * Wakeup all the lockers that currently have a chance to run.
+ */
+static void
+LWLockWakeup(LWLock *lock, LWLockMode released_mode)
+{
+	bool		releaseOK;
+	bool		wokeup_somebody = false;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	dlist_init(&wakeup);
+
+	/* remove the to-be-awakened PGPROCs from the queue */
+	releaseOK = true;
+
+	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/*
+	 * We're still waiting for backends to get scheduled, don't wake them up
+	 * again.
+	 */
+	if (!lock->releaseOK)
+	{
+		SpinLockRelease(&lock->mutex);
+		LOG_LWDEBUG("LWLockRelease", lock, "skipping due to releaseOK");
+		return;
+	}
+
+	dlist_foreach_modify(iter, &lock->waiters)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+			continue;
+
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
+
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+		{
+			/*
+			 * Prevent additional wakeups until retryer gets to run. Backends
+			 * that are just waiting for the lock to become free don't retry
+			 * automatically.
+			 */
+			releaseOK = false;
+			/*
+			 * Don't wakeup (further) exclusive locks.
+			 */
+			wokeup_somebody = true;
+		}
+
+		/*
+		 * Once we've woken up an exclusive lock, there's no point in waking
+		 * up anybody else.
+		 */
+		if(waiter->lwWaitMode == LW_EXCLUSIVE)
+			break;
+	}
+	lock->releaseOK = releaseOK;
+
+
+	/* We are done updating shared state of the lock queue. */
+	SpinLockRelease(&lock->mutex);
+
+	/*
+	 * Awaken any waiters I removed from the queue.
+	 */
+	dlist_foreach_modify(iter, &wakeup)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		LOG_LWDEBUG("LWLockRelease", lock, "release waiter");
+		dlist_delete(&waiter->lwWaitLink);
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
+	}
+}
+
+/*
+ * Add ourselves to the end of the queue. Mode can be LW_WAIT_UNTIL_FREE here!
+ */
+static void
+LWLockQueueSelf(LWLock *lock, LWLockMode mode)
+{
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	/*
+	 * If we don't have a PGPROC structure, there's no way to wait. This
+	 * should never occur, since MyProc should only be null during shared
+	 * memory initialization.
+	 */
+	if (MyProc == NULL)
+		elog(PANIC, "cannot wait without a PGPROC structure");
+
+	pg_atomic_fetch_add_u32(&lock->nwaiters, 1);
+
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	if (MyProc->lwWaiting)
+		elog(PANIC, "queueing for lock while waiting on another one");
+
+	MyProc->lwWaiting = true;
+	MyProc->lwWaitMode = mode;
+
+	/* LW_WAIT_UNTIL_FREE waiters are always at the front of the queue */
+	if (mode == LW_WAIT_UNTIL_FREE)
+		dlist_push_head(&lock->waiters, &MyProc->lwWaitLink);
+	else
+		dlist_push_tail(&lock->waiters, &MyProc->lwWaitLink);
+
+	/* Can release the mutex now */
+	SpinLockRelease(&lock->mutex);
+}
+
+/*
+ * Remove ourselves from the waitlist.  This is used if we queued ourselves
+ * because we thought we needed to sleep but, after further checking, we
+ * discovered that we don't actually need to do so. Somebody else might have
+ * already woken us up though, in that case return false.
+ */
+static bool
+LWLockDequeueSelf(LWLock *lock)
+{
+	bool	found = false;
+	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/* need to iterate, somebody else could have unqueued us */
+	dlist_foreach_modify(iter, &lock->waiters)
+	{
+		PGPROC *proc = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		if (proc == MyProc)
+		{
+			found = true;
+			dlist_delete(&proc->lwWaitLink);
+			break;
+		}
+	}
+
+	/* clear waiting state again, nice for debugging */
+	if (found)
+		MyProc->lwWaiting = false;
+
+	SpinLockRelease(&lock->mutex);
+
+	pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+	return found;
+}
 
 /*
  * LWLockAcquire - acquire a lightweight lock in the specified mode
@@ -510,18 +902,19 @@ static inline bool
 LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 {
 	PGPROC	   *proc = MyProc;
-	bool		retry = false;
 	bool		result = true;
 	int			extraWaits = 0;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
 #endif
 
-	PRINT_LWDEBUG("LWLockAcquire", lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
 
-#ifdef LWLOCK_STATS
-	lwstats = get_lwlock_stats_entry(lock);
+	PRINT_LWDEBUG("LWLockAcquire", lock, mode);
 
+#ifdef LWLOCK_STATS
 	/* Count lock acquisition attempts */
 	if (mode == LW_EXCLUSIVE)
 		lwstats->ex_acquire_count++;
@@ -567,58 +960,78 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 	{
 		bool		mustwait;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
-#ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
-#else
-		SpinLockAcquire(&lock->mutex);
-#endif
-
-		/* If retrying, allow LWLockRelease to release waiters again */
-		if (retry)
-			lock->releaseOK = true;
+		/*
+		 * try to grab the lock the first time, we're not in the waitqueue yet.
+		 */
+		mustwait = LWLockAttemptLock(lock, mode, NULL);
 
-		/* If I can get the lock, do so quickly. */
-		if (mode == LW_EXCLUSIVE)
+		if (!mustwait)
 		{
-			if (lock->exclusive == 0 && lock->shared == 0)
-			{
-				lock->exclusive++;
-				mustwait = false;
-			}
-			else
-				mustwait = true;
+			/* XXX: remove before commit? */
+			LOG_LWDEBUG("LWLockAcquire", lock, "immediately acquired lock");
+			break;				/* got the lock */
 		}
-		else
+
+		/*
+		 * Ok, at this point we couldn't grab the lock on the first try. We
+		 * cannot simply queue ourselves to the end of the list and wait to be
+		 * woken up because by now the lock could long have been released.
+		 * Instead add us to the queue and try to grab the lock again. If we
+		 * succeed we need to revert the queuing and be happy, otherwise we
+		 * recheck the lock. If we still couldn't grab it, we know that the
+		 * other lock will see our queue entries when releasing since they
+		 * existed before we checked for the lock.
+		 */
+
+		/* add to the queue */
+		LWLockQueueSelf(lock, mode);
+
+		/* we're now guaranteed to be woken up if necessary */
+		mustwait = LWLockAttemptLock(lock, mode, NULL);
+
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
 		{
-			if (lock->exclusive == 0)
+			LOG_LWDEBUG("LWLockAcquire", lock, "acquired, undoing queue");
+
+#ifdef LWLOCK_STATS
+			lwstats->dequeue_self_count++;
+#endif
+			if (!LWLockDequeueSelf(lock))
 			{
-				lock->shared++;
-				mustwait = false;
+				/*
+				 * Somebody else dequeued us and has or will wake us up. Wait
+				 * for the correct wakeup, otherwise our ->lwWaiting would get
+				 * reset at some inconvenient point later, and releaseOk
+				 * wouldn't be managed correctly.
+				 */
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+				/*
+				 * Reset releaseOk - if somebody woke us they'll have set it
+				 * to false.
+				 */
+#ifdef LWLOCK_STATS
+				lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+				SpinLockAcquire(&lock->mutex);
+#endif
+				lock->releaseOK = true;
+				SpinLockRelease(&lock->mutex);
 			}
-			else
-				mustwait = true;
+			break;
 		}
 
-		if (!mustwait)
-			break;				/* got the lock */
-
 		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait. This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
+		 * NB: There's no need to deal with spurious lock attempts
+		 * here. Anyone we prevented from acquiring the lock will
+		 * enqueue themselves using the same protocol we used here.
 		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
-
-		proc->lwWaiting = true;
-		proc->lwWaitMode = mode;
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
-
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
 
 		/*
 		 * Wait until awakened.
@@ -632,7 +1045,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockAcquire", T_NAME(lock), T_ID(lock), "waiting");
+		LOG_LWDEBUG("LWLockAcquire", lock, "waiting");
 
 #ifdef LWLOCK_STATS
 		lwstats->block_count++;
@@ -649,12 +1062,22 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 			extraWaits++;
 		}
 
+		/* not waiting anymore */
+		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
-		LOG_LWDEBUG("LWLockAcquire", T_NAME(lock), T_ID(lock), "awakened");
+		LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
 
 		/* Now loop back and try to acquire lock again. */
-		retry = true;
+#ifdef LWLOCK_STATS
+		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+		SpinLockAcquire(&lock->mutex);
+#endif
+		lock->releaseOK = true;
+		SpinLockRelease(&lock->mutex);
+
 		result = false;
 	}
 
@@ -662,13 +1085,11 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 	if (valptr)
 		*valptr = val;
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), mode);
 
 	/* Add lock to list of locks held by this backend */
-	held_lwlocks[num_held_lwlocks++] = lock;
+	held_lwlocks[num_held_lwlocks].lock = lock;
+	held_lwlocks[num_held_lwlocks++].mode = mode;
 
 	/*
 	 * Fix the process wait semaphore's count for any absorbed wakeups.
@@ -690,8 +1111,11 @@ bool
 LWLockConditionalAcquire(LWLock *lock, LWLockMode mode)
 {
 	bool		mustwait;
+	bool		potentially_spurious;
 
-	PRINT_LWDEBUG("LWLockConditionalAcquire", lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockConditionalAcquire", lock, mode);
 
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
@@ -704,50 +1128,44 @@ LWLockConditionalAcquire(LWLock *lock, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
+retry:
+	/*
+	 * We need to check whether it's possible that we prevented somebody else
+	 * from acquiring the lock. If so, potentially_spurious will be set, and
+	 * we'll retry.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, &potentially_spurious);
 
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
+	if (mustwait)
 	{
-		if (lock->exclusive == 0)
+		/*
+		 * We ran into an exclusive lock and might have blocked another
+		 * exclusive lock from taking a shot because it took a time to back
+		 * off. Retry till we are either sure we didn't block somebody (because
+		 * somebody else certainly has the lock) or till we got it.
+		 *
+		 * We cannot rely on the two-step lock-acquisition protocol as in
+		 * LWLockAcquire because we're not using it.
+		 */
+		if (potentially_spurious)
 		{
-			lock->shared++;
-			mustwait = false;
+			SPIN_DELAY();
+			goto retry;
 		}
-		else
-			mustwait = true;
-	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
-	if (mustwait)
-	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockConditionalAcquire",
-					T_NAME(lock), T_ID(lock), "failed");
-		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock),
-												 T_ID(lock), mode);
+
+		LOG_LWDEBUG("LWLockConditionalAcquire", lock, "failed");
+		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock), T_ID(lock), mode);
 	}
 	else
 	{
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lock;
+		held_lwlocks[num_held_lwlocks].lock = lock;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(T_NAME(lock), T_ID(lock), mode);
 	}
-
 	return !mustwait;
 }
 
@@ -773,14 +1191,14 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	int			extraWaits = 0;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
-#endif
-
-	PRINT_LWDEBUG("LWLockAcquireOrWait", lock);
 
-#ifdef LWLOCK_STATS
 	lwstats = get_lwlock_stats_entry(lock);
 #endif
 
+	Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockAcquireOrWait", lock, mode);
+
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
 		elog(ERROR, "too many LWLocks taken");
@@ -792,81 +1210,61 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
+	/*
+	 * NB: We're using nearly the same twice-in-a-row lock acquisition
+	 * protocol as LWLockAcquire(). Check its comments for details.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode, NULL);
 
 	if (mustwait)
 	{
-		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait.  This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
-		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		mustwait = LWLockAttemptLock(lock, mode, NULL);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
-
-		/*
-		 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
-		 * wakups, because we share the semaphore with ProcWaitForSignal.
-		 */
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock),
-					"waiting");
+		if (mustwait)
+		{
+			/*
+			 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
+			 * wakups, because we share the semaphore with ProcWaitForSignal.
+			 */
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "waiting");
 
 #ifdef LWLOCK_STATS
-		lwstats->block_count++;
+			lwstats->block_count++;
 #endif
+			TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(l), T_ID(l), mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+			for (;;)
+			{
+				/* "false" means cannot accept cancel/die interrupt here. */
+				PGSemaphoreLock(&proc->sem, false);
+				if (!proc->lwWaiting)
+					break;
+				extraWaits++;
+			}
+			pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
 
-		for (;;)
-		{
-			/* "false" means cannot accept cancel/die interrupt here. */
-			PGSemaphoreLock(&proc->sem, false);
-			if (!proc->lwWaiting)
-				break;
-			extraWaits++;
-		}
+			TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
+		}
+		else
+		{
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "acquired, undoing queue");
 
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock),
-					"awakened");
-	}
-	else
-	{
-		/* We are done updating shared state of the lock itself. */
-		SpinLockRelease(&lock->mutex);
+			/* got lock in the second attempt, undo queueing */
+			if (!LWLockDequeueSelf(lock))
+			{
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+			}
+		}
 	}
 
 	/*
@@ -879,16 +1277,17 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock), "failed");
+		LOG_LWDEBUG("LWLockAcquireOrWait", lock, "failed");
 		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT_FAIL(T_NAME(lock), T_ID(lock),
 													 mode);
 	}
 	else
 	{
+		LOG_LWDEBUG("LWLockAcquireOrWait", lock, "succeeded");
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lock;
-		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock),
-												mode);
+		held_lwlocks[num_held_lwlocks].lock = lock;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
+		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock), mode);
 	}
 
 	return !mustwait;
@@ -920,13 +1319,11 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 	bool		result = false;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
-#endif
 
-	PRINT_LWDEBUG("LWLockWaitForVar", lock);
-
-#ifdef LWLOCK_STATS
 	lwstats = get_lwlock_stats_entry(lock);
-#endif   /* LWLOCK_STATS */
+#endif
+
+	PRINT_LWDEBUG("LWLockWaitForVar", lock, LW_WAIT_UNTIL_FREE);
 
 	/*
 	 * Quick test first to see if it the slot is free right now.
@@ -935,7 +1332,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 	 * barrier here as far as the current usage is concerned.  But that might
 	 * not be safe in general.
 	 */
-	if (lock->exclusive == 0)
+	if (pg_atomic_read_u32(&lock->lockcount) == 0)
 		return true;
 
 	/*
@@ -953,21 +1350,24 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		bool		mustwait;
 		uint64		value;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
+		mustwait = pg_atomic_read_u32(&lock->lockcount) != 0;
+
+		if (mustwait)
+		{
+			/*
+			 * Perform comparison using spinlock as we can't rely on atomic 64
+			 * bit reads/stores.
+			 */
 #ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+			lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
 #else
-		SpinLockAcquire(&lock->mutex);
+			SpinLockAcquire(&lock->mutex);
 #endif
 
-		/* Is the lock now free, and if not, does the value match? */
-		if (lock->exclusive == 0)
-		{
-			result = true;
-			mustwait = false;
-		}
-		else
-		{
+			/*
+			 * XXX: We can significantly optimize this on platforms with 64bit
+			 * atomics.
+			 */
 			value = *valptr;
 			if (value != oldval)
 			{
@@ -977,21 +1377,58 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 			}
 			else
 				mustwait = true;
+			SpinLockRelease(&lock->mutex);
 		}
+		else
+			mustwait = false;
 
 		if (!mustwait)
 			break;				/* the lock was free or value didn't match */
 
 		/*
-		 * Add myself to wait queue.
+		 * Add myself to wait queue. Note that this is racy, somebody else
+		 * could wakeup before we're finished queuing.
+		 * NB: We're using nearly the same twice-in-a-row lock acquisition
+		 * protocol as LWLockAcquire(). Check its comments for details.
 		 */
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		/* waiters are added to the front of the queue */
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
+		/*
+		 * We're now guaranteed to be woken up if necessary. Recheck the
+		 * lock's state.
+		 */
+		mustwait = pg_atomic_read_u32(&lock->lockcount) != 0;
+
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
+		{
+#ifdef LWLOCK_STATS
+			lwstats->dequeue_self_count++;
+#endif
+			LOG_LWDEBUG("LWLockWaitForVar", lock, "free, undoing queue");
+
+			if (!LWLockDequeueSelf(lock))
+			{
+				/*
+				 * Somebody else dequeued us and has or will wake us up. Wait
+				 * for the correct wakeup, otherwise our ->lwWaiting would get
+				 * reset at some inconvenient point later.
+				 */
+				for (;;)
+				{
+					PGSemaphoreLock(&proc->sem, false);
+					if (!proc->lwWaiting)
+						break;
+					extraWaits++;
+				}
+			}
+			break;
+		}
+
+		/*
+		 * NB: Just as in LWLockAcquireCommon() there's no need to deal with
+		 * spurious lock attempts here.
+		 */
 
 		/*
 		 * Wait until awakened.
@@ -1005,7 +1442,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockWaitForVar", T_NAME(lock), T_ID(lock), "waiting");
+		LOG_LWDEBUG("LWLockWaitForVar", lock, "waiting");
 
 #ifdef LWLOCK_STATS
 		lwstats->block_count++;
@@ -1023,17 +1460,16 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 			extraWaits++;
 		}
 
+		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock),
 										  LW_EXCLUSIVE);
 
-		LOG_LWDEBUG("LWLockWaitForVar", T_NAME(lock), T_ID(lock), "awakened");
+		LOG_LWDEBUG("LWLockWaitForVar", lock, "awakened");
 
 		/* Now loop back and check the status of the lock again. */
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), LW_EXCLUSIVE);
 
 	/*
@@ -1066,14 +1502,24 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 {
 	dlist_head	wakeup;
 	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	PRINT_LWDEBUG("LWLockUpdateVar", lock, LW_EXCLUSIVE);
 
 	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
 	SpinLockAcquire(&lock->mutex);
+#endif
 
-	/* we should hold the lock */
-	Assert(lock->exclusive == 1);
+	Assert(pg_atomic_read_u32(&lock->lockcount) >= EXCLUSIVE_LOCK);
 
 	/* Update the lock's value */
 	*valptr = val;
@@ -1116,22 +1562,23 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 void
 LWLockRelease(LWLock *lock)
 {
-	dlist_head	wakeup;
-	dlist_mutable_iter iter;
+	LWLockMode	mode;
+	uint32		lockcount;
+	bool		check_waiters;
+	bool		have_waiters = false;
 	int			i;
 
-	dlist_init(&wakeup);
-
-	PRINT_LWDEBUG("LWLockRelease", lock);
-
 	/*
 	 * Remove lock from list of locks held.  Usually, but not always, it will
 	 * be the latest-acquired lock; so search array backwards.
 	 */
 	for (i = num_held_lwlocks; --i >= 0;)
 	{
-		if (lock == held_lwlocks[i])
+		if (lock == held_lwlocks[i].lock)
+		{
+			mode = held_lwlocks[i].mode;
 			break;
+		}
 	}
 	if (i < 0)
 		elog(ERROR, "lock %s %d is not held", T_NAME(lock), T_ID(lock));
@@ -1139,78 +1586,57 @@ LWLockRelease(LWLock *lock)
 	for (; i < num_held_lwlocks; i++)
 		held_lwlocks[i] = held_lwlocks[i + 1];
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* Release my hold on lock */
-	if (lock->exclusive > 0)
-		lock->exclusive--;
-	else
-	{
-		Assert(lock->shared > 0);
-		lock->shared--;
-	}
+	PRINT_LWDEBUG("LWLockRelease", lock, mode);
 
 	/*
-	 * See if I need to awaken any waiters.  If I released a non-last shared
-	 * hold, there cannot be anything to do.  Also, do not awaken any waiters
-	 * if someone has already awakened waiters that haven't yet acquired the
-	 * lock.
+	 * Release my hold on lock, after that it can immediately be acquired by
+	 * others, even if we still have to wakeup other waiters.
 	 */
-	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
-	{
-		/*
-		 * Remove the to-be-awakened PGPROCs from the queue.
-		 */
-		bool		releaseOK = true;
-		bool		wokeup_somebody = false;
-
-		dlist_foreach_modify(iter, &lock->waiters)
-		{
-			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-
-			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
-				continue;
-
-			dlist_delete(&waiter->lwWaitLink);
-			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
-
-			/*
-			 * Prevent additional wakeups until retryer gets to
-			 * run. Backends that are just waiting for the lock to become
-			 * free don't retry automatically.
-			 */
-			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
-			{
-				releaseOK = false;
-				wokeup_somebody = true;
-			}
+	if (mode == LW_EXCLUSIVE)
+		lockcount = pg_atomic_sub_fetch_u32(&lock->lockcount, EXCLUSIVE_LOCK);
+	else
+		lockcount = pg_atomic_sub_fetch_u32(&lock->lockcount, 1);
 
-			if(waiter->lwWaitMode == LW_EXCLUSIVE)
-				break;
-		}
-		lock->releaseOK = releaseOK;
-	}
+	/* nobody else can have that kind of lock */
+	Assert(lockcount < EXCLUSIVE_LOCK);
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
+	/*
+	 * Anybody we need to wakeup needs to have started queueing before we
+	 * removed ourselves from the queue and the atomic operations above are
+	 * full barriers. So we can just do plain read.
+	 */
+	if (pg_atomic_read_u32(&lock->nwaiters) > 0)
+		have_waiters = true;
 
-	TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+	/*
+	 * If we just released an exclusive lock always wakeup waiters, even if
+	 * lockcount is still > 0. A shared acquisition temporarily (and
+	 * spuriously) might have increased the value.
+	 */
+	if (mode == LW_EXCLUSIVE && have_waiters)
+		check_waiters = true;
+	/*
+	 * nobody has this locked anymore, potential exclusive lockers get a chance
+	 */
+	else if (lockcount == 0 && have_waiters)
+		check_waiters = true;
+	/* nobody queued or not free */
+	else
+		check_waiters = false;
 
 	/*
-	 * Awaken any waiters I removed from the queue.
+	 * As waking up waiters requires the spinlock to be acquired, only do so
+	 * if necessary.
 	 */
-	dlist_foreach_modify(iter, &wakeup)
+	if (check_waiters)
 	{
-		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-		LOG_LWDEBUG("LWLockRelease", T_NAME(lock), T_ID(lock),
-					"release waiter");
-		dlist_delete(&waiter->lwWaitLink);
-		pg_write_barrier();
-		waiter->lwWaiting = false;
-		PGSemaphoreUnlock(&waiter->sem);
+		/* XXX: remove before commit? */
+		LOG_LWDEBUG("LWLockRelease", lock, "releasing waiters");
+		LWLockWakeup(lock, mode);
 	}
 
+	TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+
 	/*
 	 * Now okay to allow cancel/die interrupts.
 	 */
@@ -1234,7 +1660,7 @@ LWLockReleaseAll(void)
 	{
 		HOLD_INTERRUPTS();		/* match the upcoming RESUME_INTERRUPTS */
 
-		LWLockRelease(held_lwlocks[num_held_lwlocks - 1]);
+		LWLockRelease(held_lwlocks[num_held_lwlocks - 1].lock);
 	}
 }
 
@@ -1242,8 +1668,8 @@ LWLockReleaseAll(void)
 /*
  * LWLockHeldByMe - test whether my process currently holds a lock
  *
- * This is meant as debug support only.  We do not distinguish whether the
- * lock is held shared or exclusive.
+ * This is meant as debug support only.  We currently do not distinguish
+ * whether the lock is held shared or exclusive.
  */
 bool
 LWLockHeldByMe(LWLock *l)
@@ -1252,7 +1678,7 @@ LWLockHeldByMe(LWLock *l)
 
 	for (i = 0; i < num_held_lwlocks; i++)
 	{
-		if (held_lwlocks[i] == l)
+		if (held_lwlocks[i].lock == l)
 			return true;
 	}
 	return false;
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index fea5d33..595e69d 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -16,6 +16,7 @@
 
 #include "lib/ilist.h"
 #include "storage/s_lock.h"
+#include "port/atomics.h"
 
 struct PGPROC;
 
@@ -48,10 +49,14 @@ typedef struct LWLock
 {
 	slock_t		mutex;			/* Protects LWLock and queue of PGPROCs */
 	bool		releaseOK;		/* T if ok to release waiters */
-	char		exclusive;		/* # of exclusive holders (0 or 1) */
-	int			shared;			/* # of shared holders (0..MaxBackends) */
+
+	pg_atomic_uint32 lockcount;	/* state of exlusive/nonexclusive lockers */
+	pg_atomic_uint32 nwaiters;	/* number of waiters */
 	int			tranche;		/* tranche ID */
 	dlist_head	waiters;		/* list of waiting PGPROCs */
+#ifdef LOCK_DEBUG
+	struct PGPROC *owner;		/* last exlusive owner of the lock */
+#endif
 } LWLock;
 
 /*
-- 
1.8.3.251.g1462b67

0003-Don-t-use-a-spinlock-while-manipulating-a-lwlock-s-r.patchtext/x-patch; charset=us-asciiDownload
>From b15e5f486f84ad4aefe3c5282cdd2c535b9966b1 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Fri, 10 Oct 2014 13:02:17 +0200
Subject: [PATCH 3/4] Don't use a spinlock while manipulating a lwlock's
 releaseOK state.

Possibly it's relevant for concurrency because it increases the number
of spinlock acquisition by a significant factor. And it's not too hard
to avoid.
---
 src/backend/storage/lmgr/lwlock.c | 53 ++++++++++++++++++---------------------
 1 file changed, 24 insertions(+), 29 deletions(-)

diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index b10c121..2246fc3 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -156,6 +156,9 @@ static LWLockTranche MainLWLockTranche;
  */
 #define MAX_SIMUL_LWLOCKS	200
 
+/* force boolean variable `var' to be accessed once, be it for write or read */
+#define BOOL_ACCESS_ONCE(var)	(*((volatile bool *)&(var)))
+
 typedef struct LWLockHandle
 {
 	LWLock *lock;
@@ -710,27 +713,27 @@ LWLockWakeup(LWLock *lock, LWLockMode released_mode)
 
 	dlist_init(&wakeup);
 
-	/* remove the to-be-awakened PGPROCs from the queue */
-	releaseOK = true;
-
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-#ifdef LWLOCK_STATS
-	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
-#else
-	SpinLockAcquire(&lock->mutex);
-#endif
-
 	/*
 	 * We're still waiting for backends to get scheduled, don't wake them up
 	 * again.
 	 */
-	if (!lock->releaseOK)
+	pg_read_barrier(); /* pairs with nwaiters-- */
+	if (!BOOL_ACCESS_ONCE(lock->releaseOK))
 	{
-		SpinLockRelease(&lock->mutex);
 		LOG_LWDEBUG("LWLockRelease", lock, "skipping due to releaseOK");
 		return;
 	}
 
+	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/* remove the to-be-awakened PGPROCs from the queue */
+	releaseOK = true;
+
 	dlist_foreach_modify(iter, &lock->waiters)
 	{
 		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
@@ -762,8 +765,9 @@ LWLockWakeup(LWLock *lock, LWLockMode released_mode)
 		if(waiter->lwWaitMode == LW_EXCLUSIVE)
 			break;
 	}
-	lock->releaseOK = releaseOK;
 
+	/* The SpinLockRelease() below provides barrier semantics */
+	BOOL_ACCESS_ONCE(lock->releaseOK) = releaseOK;
 
 	/* We are done updating shared state of the lock queue. */
 	SpinLockRelease(&lock->mutex);
@@ -1012,17 +1016,13 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 						break;
 					extraWaits++;
 				}
+
 				/*
 				 * Reset releaseOk - if somebody woke us they'll have set it
-				 * to false.
+				 * to false. No need for a barrier here - we got the lock and
+				 * will perform wakeup if necessary.
 				 */
-#ifdef LWLOCK_STATS
-				lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
-#else
-				SpinLockAcquire(&lock->mutex);
-#endif
-				lock->releaseOK = true;
-				SpinLockRelease(&lock->mutex);
+				BOOL_ACCESS_ONCE(lock->releaseOK) = true;
 			}
 			break;
 		}
@@ -1062,6 +1062,9 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 			extraWaits++;
 		}
 
+		/* The atomic subtraction provides the barrier */
+		BOOL_ACCESS_ONCE(lock->releaseOK) = true;
+
 		/* not waiting anymore */
 		pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
 
@@ -1070,14 +1073,6 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 		LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
 
 		/* Now loop back and try to acquire lock again. */
-#ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
-#else
-		SpinLockAcquire(&lock->mutex);
-#endif
-		lock->releaseOK = true;
-		SpinLockRelease(&lock->mutex);
-
 		result = false;
 	}
 
-- 
1.8.3.251.g1462b67

0004-Spelling-and-other-comment-improvements-for-the-new-.patchtext/x-patch; charset=us-asciiDownload
>From bac195b2c359b3dd311e30d7f17f83d09ece4438 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Fri, 10 Oct 2014 13:03:56 +0200
Subject: [PATCH 4/4] Spelling and other comment improvements for the new
 lwlock code.

Robert Haas
---
 src/backend/storage/lmgr/lwlock.c | 50 ++++++++++++++++++++++-----------------
 1 file changed, 28 insertions(+), 22 deletions(-)

diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 2246fc3..d1f0f77 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -29,40 +29,46 @@
  * This used to be a pretty straight forward reader-writer lock
  * implementation, in which the internal state was protected by a
  * spinlock. Unfortunately the overhead of taking the spinlock proved to be
- * too high for workloads/locks that were locked in shared mode very
- * frequently. Often we were spinning in the (obviously exlusive) spinlock,
+ * too high for workloads/locks that were taken in shared mode very
+ * frequently. Often we were spinning in the (obviously exclusive) spinlock,
  * while trying to acquire a shared lock that was actually free.
  *
  * Thus a new implementation was devised that provides wait-free shared lock
- * acquiration for locks that aren't exclusively locked.
+ * acquisition for locks that aren't exclusively locked.
  *
  * The basic idea is to have a single atomic variable 'lockcount' instead of
  * the formerly separate shared and exclusive counters and to use an atomic
  * increment to acquire the lock. That's fairly easy to do for rw-spinlocks,
  * but a lot harder for something like LWLocks that want to wait in the OS.
  *
- * For exlusive lock acquisition we use an atomic compare-and-exchange on the
- * lockcount variable swapping in EXCLUSIVE_LOCK/1<<31-1/0x7FFFFFFF if and only
- * if the current value of lockcount is 0. If the swap was not successfull, we
- * have to wait.
+ * For exclusive lock acquisition we use an atomic compare-and-exchange on the
+ * lockcount variable, swapping in a sentinel value
+ * (EXCLUSIVE_LOCK/1<<31-1/0x7FFFFFFF) if and only if the current value of
+ * lockcount is 0. If the swap was not successful, we have to wait.
  *
  * For shared lock acquisition we use an atomic add to the lockcount variable
- * to add 1. If the value is bigger than EXCLUSIVE_LOCK we know that somebody
+ * to add 1. If the value returned by the atomic fetch-and-add operation is
+ * bigger than the value marking an exclusive locks we know that somebody else
  * actually has an exclusive lock, and we back out by atomically decrementing
- * by 1 again. If so, we have to wait for the exlusive locker to release the
+ * by 1 again. If so, we have to wait for the exclusive locker to release the
  * lock.
  *
  * To release the lock we use an atomic decrement to release the lock. If the
  * new value is zero (we get that atomically), we know we have to release
  * waiters.
  *
- * The attentive reader probably might have noticed that naively doing the
- * above has two glaring race conditions:
+ * Obviously it is important that the sentinel value for exclusive locks
+ * doesn't conflict with the maximum number of possible share lockers -
+ * luckily MAX_BACKENDS makes that easily possible.
+ *
+ *
+ * The attentive reader might have noticed that naively doing the above has
+ * two glaring race conditions:
  *
  * 1) too-quick-for-queueing: We try to lock using the atomic operations and
- * notice that we have to wait. Unfortunately until we have finished queuing,
- * the former locker very well might have already finished it's work. That's
- * problematic because we're now stuck waiting inside the OS.
+ * notice that we have to wait. Unfortunately by the time we have finished
+ * queuing, the former locker very well might have already finished it's
+ * work. That's problematic because we're now stuck waiting inside the OS.
  *
  * 2) spurious failed locks: Due to the logic of backing out of shared
  * locks after we unconditionally added a 1 to lockcount, we might have
@@ -80,23 +86,23 @@
  *
  * To mitigate those races we use a two phased attempt at locking:
  *   Phase 1: Try to do it atomically, if we succeed, nice
- *   Phase 2: Add us too the waitqueue of the lock
+ *   Phase 2: Add ourselves to the waitqueue of the lock
  *   Phase 3: Try to grab the lock again, if we succeed, remove ourselves from
  *            the queue
- *   Phase 4: Sleep till wakeup, goto Phase 1
+ *   Phase 4: Sleep till wake-up, goto Phase 1
  *
  * This protects us against both problems from above:
  * 1) Nobody can release too quick, before we're queued, since after Phase 2
  *    we're already queued.
  * 2) If somebody spuriously got blocked from acquiring the lock, they will
- *    get queued in Phase 2 and we can wake them up if neccessary or they will
+ *    get queued in Phase 2 and we can wake them up if necessary or they will
  *    have gotten the lock in Phase 3.
  *
  * There above algorithm only works for LWLockAcquire, not directly for
  * LWLockAcquireConditional where we don't want to wait. In that case we just
- * need to retry acquiring the lock until we're sure we didn't disturb anybody
- * in doing so.
- *-------------------------------------------------------------------------
+ * need to retry acquiring the lock until we're sure we didn't spuriously
+ * prevent anybody else from doing so.
+ * -------------------------------------------------------------------------
  */
 #include "postgres.h"
 
@@ -631,7 +637,7 @@ LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)
 		}
 		else
 		{
-			/* yipeyyahee */
+			/* Great! Got the lock. */
 #ifdef LOCK_DEBUG
 			lock->owner = MyProc;
 #endif
@@ -687,7 +693,7 @@ LWLockAttemptLock(LWLock* lock, LWLockMode mode, bool *potentially_spurious)
 		}
 		else
 		{
-			/* yipeyyahee */
+			/* Great! Got the lock. */
 			return false;
 		}
 	}
-- 
1.8.3.251.g1462b67

#80Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#77)
Re: Wait free LW_SHARED acquisition - v0.9

On Fri, Oct 10, 2014 at 1:27 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-10 10:13:03 +0530, Amit Kapila wrote:

I have done few performance tests for above patches and results of
same is as below:

Cool, thanks.

Performance Data
------------------------------
IBM POWER-7 16 cores, 64 hardware threads
RAM = 64GB
max_connections =210
Database Locale =C
checkpoint_segments=256
checkpoint_timeout =35min
shared_buffers=8GB
Client Count = number of concurrent sessions and threads (ex. -c 8 -j 8)
Duration of each individual run = 5mins
Test type - read only pgbench with -M prepared
Other Related information about test
a. This is the data for median of 3 runs, the detailed data of

individual

run
is attached with mail.
b. I have applied both the patches to take performance data.

Observations
----------------------
a. The patch performs really well (increase upto ~40%) incase all the
data fits in shared buffers (scale factor -100).
b. Incase data doesn't fit in shared buffers, but fits in RAM
(scale factor -3000), there is performance increase upto 16 client

count,

however after that it starts dipping (in above config unto ~4.4%).

Hm. Interesting. I don't see that dip on x86.

Is it possible that implementation of some atomic operation is costlier
for particular architecture?

I have tried again for scale factor 3000 and could see the dip and this
time I have even tried with 175 client count and the dip is approximately
5% which is slightly more than 160 client count.

Patch_ver/Client_count 175 HEAD 248374 PATCH 235669

Now probably these shouldn't matter much in case backend needs to
wait for other Exclusive locker, but I am not sure what else could be
the reason for dip in case we need to have Exclusive LWLocks.

Any chance to get a profile?

Here it goes..

HEAD - client_count=128
-----------------------------------------

+   7.53%         postgres  postgres               [.] GetSnapshotData
+   3.41%         postgres  postgres               [.] AllocSetAlloc
+   2.61%         postgres  postgres               [.] AllocSetFreeIndex
+   2.49%         postgres  postgres               [.] _bt_compare
+   2.43%         postgres  [kernel.kallsyms]      [k] .__copy_tofrom_user
+   2.40%         postgres  postgres               [.]
hash_search_with_hash_value
+   1.83%         postgres  postgres               [.] tas
+   1.29%         postgres  postgres               [.] pg_encoding_mbcliplen
+   1.27%         postgres  postgres               [.] MemoryContextCreate
+   1.22%         postgres  postgres               [.]
MemoryContextAllocZeroAligned
+   1.17%         postgres  postgres               [.] hash_seq_search
+   0.97%         postgres  postgres               [.] LWLockRelease
+   0.96%         postgres  postgres               [.]
MemoryContextAllocZero
+   0.91%         postgres  postgres               [.]
GetPrivateRefCountEntry
+   0.82%         postgres  postgres               [.] AllocSetFree
+   0.79%         postgres  postgres               [.] LWLockAcquireCommon
+   0.78%         postgres  postgres               [.] pfree

Detailed Data
-------------------------
- 7.53% postgres postgres [.] GetSnapshotData
- GetSnapshotData
- 7.46% GetSnapshotData
- 7.46% GetTransactionSnapshot
- 3.74% exec_bind_message
PostgresMain
BackendRun
BackendStartup
ServerLoop
PostmasterMain
main
generic_start_main.isra.0
__libc_start_main
0
- 3.72% PortalStart
exec_bind_message
PostgresMain
BackendRun
BackendStartup
ServerLoop
PostmasterMain
main
generic_start_main.isra.0
__libc_start_main
0
- 3.41% postgres postgres [.] AllocSetAlloc
- AllocSetAlloc
- 2.01% AllocSetAlloc
0.81% palloc
0.63% MemoryContextAlloc
- 2.61% postgres postgres [.] AllocSetFreeIndex
- AllocSetFreeIndex
1.59% AllocSetAlloc
0.79% AllocSetFree
- 2.49% postgres postgres [.] _bt_compare
- _bt_compare
- 1.80% _bt_binsrch
- 1.80% _bt_binsrch
- 1.21% _bt_search
_bt_first

Lwlock_contention patches - client_count=128
----------------------------------------------------------------------

+   7.95%      postgres  postgres               [.] GetSnapshotData
+   3.58%      postgres  postgres               [.] AllocSetAlloc
+   2.51%      postgres  postgres               [.] _bt_compare
+   2.44%      postgres  postgres               [.]
hash_search_with_hash_value
+   2.33%      postgres  [kernel.kallsyms]      [k] .__copy_tofrom_user
+   2.24%      postgres  postgres               [.] AllocSetFreeIndex
+   1.75%      postgres  postgres               [.]
pg_atomic_fetch_add_u32_impl
+   1.60%      postgres  postgres               [.] hash_seq_search
+   1.31%      postgres  postgres               [.] pg_encoding_mbcliplen
+   1.27%      postgres  postgres               [.]
MemoryContextAllocZeroAligned
+   1.26%      postgres  postgres               [.] MemoryContextCreate
+   0.98%      postgres  postgres               [.] GetPrivateRefCountEntry
+   0.97%      postgres  postgres               [.] MemoryContextAllocZero
+   0.87%      postgres  postgres               [.] LWLockRelease
+   0.82%      postgres  postgres               [.] AllocSetFree
+   0.79%      postgres  postgres               [.] SearchCatCache
+   0.70%      postgres  postgres               [.] palloc
    0.69%      postgres  postgres               [.] pfree
+   0.61%      postgres  postgres               [.] AllocSetDelete
+   0.57%      postgres  postgres               [.] hash_any
    0.57%      postgres  postgres               [.] MemoryContextAlloc
    0.56%      postgres  postgres               [.] FunctionCall2Coll
+   0.56%       swapper  [kernel.kallsyms]      [k]
.pseries_dedicated_idle_sleep
    0.56%      postgres  libc-2.14.90.so        [.] memcpy
+   0.55%      postgres  postgres               [.] AllocSetReset
    0.51%      postgres  postgres               [.] _bt_binsrch
    0.50%      postgres  postgres               [.] LWLockAcquireCommon

Detailed Data
----------------------------
- 7.95% postgres postgres [.] GetSnapshotData

- GetSnapshotData

- 7.87% GetSnapshotData

- 7.87% GetTransactionSnapshot

- 3.95% exec_bind_message

0

- 3.92% PortalStart

exec_bind_message

0

- 3.58% postgres postgres [.] AllocSetAlloc

- AllocSetAlloc

- 1.82% AllocSetAlloc

0.75% palloc

0.56% MemoryContextAlloc

- 2.51% postgres postgres [.] _bt_compare

- _bt_compare

- 1.75% _bt_binsrch

- 1.75% _bt_binsrch

- 1.18% _bt_search

_bt_first

btgettuple

- 2.44% postgres postgres [.]
hash_search_with_hash_value

- hash_search_with_hash_value

- 2.01% hash_search_with_hash_value

- 0.96% BufTableLookup

BufferAlloc

- 2.33% postgres [kernel.kallsyms] [k] .__copy_tofrom_user

- .__copy_tofrom_user

- 2.27% .file_read_actor

.generic_file_aio_read

.do_sync_read

- 2.24% postgres postgres [.] AllocSetFreeIndex

- AllocSetFreeIndex

1.23% AllocSetAlloc

0.68% AllocSetFree

- 1.75% postgres postgres [.]
pg_atomic_fetch_add_u32_impl

- pg_atomic_fetch_add_u32_impl

0.95% pg_atomic_fetch_add_u32

0.75% pg_atomic_fetch_sub_u32_impl

- 1.60% postgres postgres [.] hash_seq_search

- hash_seq_search

- 0.91% PreCommit_Portals

- 0.91% PreCommit_Portals

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#81Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#80)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-10 17:18:46 +0530, Amit Kapila wrote:

On Fri, Oct 10, 2014 at 1:27 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

Observations
----------------------
a. The patch performs really well (increase upto ~40%) incase all the
data fits in shared buffers (scale factor -100).
b. Incase data doesn't fit in shared buffers, but fits in RAM
(scale factor -3000), there is performance increase upto 16 client

count,

however after that it starts dipping (in above config unto ~4.4%).

Hm. Interesting. I don't see that dip on x86.

Is it possible that implementation of some atomic operation is costlier
for particular architecture?

Yes, sure. And IIRC POWER improved atomics performance considerably for
POWER8...

I have tried again for scale factor 3000 and could see the dip and this
time I have even tried with 175 client count and the dip is approximately
5% which is slightly more than 160 client count.

FWIW, the profile always looks like
-  48.61%      postgres  postgres              [.] s_lock
   - s_lock
      + 96.67% StrategyGetBuffer
      + 1.19% UnpinBuffer
      + 0.90% PinBuffer
      + 0.70% hash_search_with_hash_value
+   3.11%      postgres  postgres              [.] GetSnapshotData
+   2.47%      postgres  postgres              [.] StrategyGetBuffer
+   1.93%      postgres  [kernel.kallsyms]     [k] copy_user_generic_string
+   1.28%      postgres  postgres              [.] hash_search_with_hash_value
-   1.27%      postgres  postgres              [.] LWLockAttemptLock
   - LWLockAttemptLock
      - 97.78% LWLockAcquire
         + 38.76% ReadBuffer_common
         + 28.62% _bt_getbuf
         + 8.59% _bt_relandgetbuf
         + 6.25% GetSnapshotData
         + 5.93% VirtualXactLockTableInsert
         + 3.95% VirtualXactLockTableCleanup
         + 2.35% index_fetch_heap
         + 1.66% StartBufferIO
         + 1.56% LockReleaseAll
         + 1.55% _bt_next
         + 0.78% LockAcquireExtended
      + 1.47% _bt_next
      + 0.75% _bt_relandgetbuf

to me. Now that's with the client count 496, but it's similar with lower
counts.

BTW, that profile *clearly* indicates we should make StrategyGetBuffer()
smarter.

Patch_ver/Client_count 175 HEAD 248374 PATCH 235669

Now probably these shouldn't matter much in case backend needs to
wait for other Exclusive locker, but I am not sure what else could be
the reason for dip in case we need to have Exclusive LWLocks.

Any chance to get a profile?

Here it goes..

Lwlock_contention patches - client_count=128
----------------------------------------------------------------------

+   7.95%      postgres  postgres               [.] GetSnapshotData
+   3.58%      postgres  postgres               [.] AllocSetAlloc
+   2.51%      postgres  postgres               [.] _bt_compare
+   2.44%      postgres  postgres               [.]
hash_search_with_hash_value
+   2.33%      postgres  [kernel.kallsyms]      [k] .__copy_tofrom_user
+   2.24%      postgres  postgres               [.] AllocSetFreeIndex
+   1.75%      postgres  postgres               [.]
pg_atomic_fetch_add_u32_impl

Uh. Huh? Normally that'll be inline. That's compiled with gcc? What were
the compiler settings you used?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#82Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#81)
1 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-10 16:41:39 +0200, Andres Freund wrote:

FWIW, the profile always looks like
-  48.61%      postgres  postgres              [.] s_lock
- s_lock
+ 96.67% StrategyGetBuffer
+ 1.19% UnpinBuffer
+ 0.90% PinBuffer
+ 0.70% hash_search_with_hash_value
+   3.11%      postgres  postgres              [.] GetSnapshotData
+   2.47%      postgres  postgres              [.] StrategyGetBuffer
+   1.93%      postgres  [kernel.kallsyms]     [k] copy_user_generic_string
+   1.28%      postgres  postgres              [.] hash_search_with_hash_value
-   1.27%      postgres  postgres              [.] LWLockAttemptLock
- LWLockAttemptLock
- 97.78% LWLockAcquire
+ 38.76% ReadBuffer_common
+ 28.62% _bt_getbuf
+ 8.59% _bt_relandgetbuf
+ 6.25% GetSnapshotData
+ 5.93% VirtualXactLockTableInsert
+ 3.95% VirtualXactLockTableCleanup
+ 2.35% index_fetch_heap
+ 1.66% StartBufferIO
+ 1.56% LockReleaseAll
+ 1.55% _bt_next
+ 0.78% LockAcquireExtended
+ 1.47% _bt_next
+ 0.75% _bt_relandgetbuf

to me. Now that's with the client count 496, but it's similar with lower
counts.

BTW, that profile *clearly* indicates we should make StrategyGetBuffer()
smarter.

Which is nearly trivial now that atomics are in. Check out the attached
WIP patch which eliminates the spinlock from StrategyGetBuffer() unless
there's buffers on the freelist.

Test:
pgbench -M prepared -P 5 -S -c 496 -j 496 -T 5000
on a scale=1000 database, with 4GB of shared buffers.

Before:
progress: 40.0 s, 136252.3 tps, lat 3.628 ms stddev 4.547
progress: 45.0 s, 135049.0 tps, lat 3.660 ms stddev 4.515
progress: 50.0 s, 135788.9 tps, lat 3.640 ms stddev 4.398
progress: 55.0 s, 135268.4 tps, lat 3.654 ms stddev 4.469
progress: 60.0 s, 134991.6 tps, lat 3.661 ms stddev 4.739

after:
progress: 40.0 s, 207701.1 tps, lat 2.382 ms stddev 3.018
progress: 45.0 s, 208022.4 tps, lat 2.377 ms stddev 2.902
progress: 50.0 s, 209187.1 tps, lat 2.364 ms stddev 2.970
progress: 55.0 s, 206462.7 tps, lat 2.396 ms stddev 2.871
progress: 60.0 s, 210263.8 tps, lat 2.351 ms stddev 2.914

Yes, no kidding.

The results are similar, but less extreme, for smaller client counts
like 80 or 160.

Amit, since your test seems to be currently completely bottlenecked
within StrategyGetBuffer(), could you compare with that patch applied to
HEAD and the LW_SHARED patch for one client count? That'll may allow us
to see a more meaningful profile...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-WIP-lockless-StrategyGetBuffer-hotpath.patchtext/x-patch; charset=us-asciiDownload
>From 6b486e5b467e94ab9297d7656a5b39b816c5c55a Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Fri, 10 Oct 2014 17:36:46 +0200
Subject: [PATCH] WIP: lockless StrategyGetBuffer hotpath

---
 src/backend/storage/buffer/freelist.c | 154 ++++++++++++++++++++--------------
 1 file changed, 90 insertions(+), 64 deletions(-)

diff --git a/src/backend/storage/buffer/freelist.c b/src/backend/storage/buffer/freelist.c
index 5966beb..0c634a0 100644
--- a/src/backend/storage/buffer/freelist.c
+++ b/src/backend/storage/buffer/freelist.c
@@ -18,6 +18,12 @@
 #include "storage/buf_internals.h"
 #include "storage/bufmgr.h"
 
+#include "port/atomics.h"
+
+
+#define LATCHPTR_ACCESS_ONCE(var)	((Latch *)(*((volatile Latch **)&(var))))
+#define INT_ACCESS_ONCE(var)	((int)(*((volatile int *)&(var))))
+
 
 /*
  * The shared freelist control information.
@@ -27,8 +33,12 @@ typedef struct
 	/* Spinlock: protects the values below */
 	slock_t		buffer_strategy_lock;
 
-	/* Clock sweep hand: index of next buffer to consider grabbing */
-	int			nextVictimBuffer;
+	/*
+	 * Clock sweep hand: index of next buffer to consider grabbing. Note that
+	 * this isn't a concrete buffer - we only ever increase the value. So, to
+	 * get an actual buffer, it needs to be used modulo NBuffers.
+	 */
+	pg_atomic_uint32 nextVictimBuffer;
 
 	int			firstFreeBuffer;	/* Head of list of unused buffers */
 	int			lastFreeBuffer; /* Tail of list of unused buffers */
@@ -42,8 +52,8 @@ typedef struct
 	 * Statistics.  These counters should be wide enough that they can't
 	 * overflow during a single bgwriter cycle.
 	 */
-	uint32		completePasses; /* Complete cycles of the clock sweep */
-	uint32		numBufferAllocs;	/* Buffers allocated since last reset */
+	pg_atomic_uint32 completePasses; /* Complete cycles of the clock sweep */
+	pg_atomic_uint32 numBufferAllocs;	/* Buffers allocated since last reset */
 
 	/*
 	 * Notification latch, or NULL if none.  See StrategyNotifyBgWriter.
@@ -124,87 +134,107 @@ StrategyGetBuffer(BufferAccessStrategy strategy)
 			return buf;
 	}
 
-	/* Nope, so lock the freelist */
-	SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
-
 	/*
 	 * We count buffer allocation requests so that the bgwriter can estimate
 	 * the rate of buffer consumption.  Note that buffers recycled by a
 	 * strategy object are intentionally not counted here.
 	 */
-	StrategyControl->numBufferAllocs++;
+	pg_atomic_fetch_add_u32(&StrategyControl->numBufferAllocs, 1);
 
 	/*
-	 * If bgwriterLatch is set, we need to waken the bgwriter, but we should
-	 * not do so while holding buffer_strategy_lock; so release and re-grab.
-	 * This is annoyingly tedious, but it happens at most once per bgwriter
-	 * cycle, so the performance hit is minimal.
+	 * If bgwriterLatch is set, we need to waken the bgwriter. We don't want
+	 * to check this while holding the spinlock, so we the latch from memory
+	 * once to see whether it's set. There's no need to do so with a lock
+	 * present - we'll just set the latch next call if we missed it once.
+	 *
+	 * Since we're not guaranteed atomic 8 byte reads we need to acquire the
+	 * spinlock if not null to be sure we get a correct pointer. Because we
+	 * don't want to set the latch while holding the buffer_strategy_lock we
+	 * just grab the lock to read and reset the pointer.
 	 */
-	bgwriterLatch = StrategyControl->bgwriterLatch;
+	bgwriterLatch = LATCHPTR_ACCESS_ONCE(StrategyControl->bgwriterLatch);
 	if (bgwriterLatch)
 	{
+		/* we don't have guaranteed atomic 64bit reads */
+		SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
+		bgwriterLatch = LATCHPTR_ACCESS_ONCE(StrategyControl->bgwriterLatch);
 		StrategyControl->bgwriterLatch = NULL;
 		SpinLockRelease(&StrategyControl->buffer_strategy_lock);
-		SetLatch(bgwriterLatch);
-		SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
+
+		/* recheck */
+		if (bgwriterLatch)
+			SetLatch(bgwriterLatch);
 	}
 
 	/*
-	 * Try to get a buffer from the freelist.  Note that the freeNext fields
-	 * are considered to be protected by the buffer_strategy_lock not the
-	 * individual buffer spinlocks, so it's OK to manipulate them without
-	 * holding the spinlock.
+	 * First check, without acquiring the lock, wether there's buffers in the
+	 * freelist. Since we otherwise don't require the spinlock in every
+	 * StrategyGetBuffer() invocation, it'd be sad to acquire it here -
+	 * uselessly in mos tcases.
+	 *
+	 * If there's buffers on the freelist, acquire the spinlock and look into
+	 * them.
+	 *
+	 * Note that the freeNext fields are considered to be protected by
+	 * the buffer_strategy_lock not the individual buffer spinlocks, so it's
+	 * OK to manipulate them without holding the spinlock.
 	 */
-	while (StrategyControl->firstFreeBuffer >= 0)
+	if (INT_ACCESS_ONCE(StrategyControl->firstFreeBuffer) >= 0)
 	{
-		buf = &BufferDescriptors[StrategyControl->firstFreeBuffer];
-		Assert(buf->freeNext != FREENEXT_NOT_IN_LIST);
+		SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
 
-		/* Unconditionally remove buffer from freelist */
-		StrategyControl->firstFreeBuffer = buf->freeNext;
-		buf->freeNext = FREENEXT_NOT_IN_LIST;
+		while (StrategyControl->firstFreeBuffer >= 0)
+		{
+			buf = &BufferDescriptors[StrategyControl->firstFreeBuffer];
+			Assert(buf->freeNext != FREENEXT_NOT_IN_LIST);
 
-		/*
-		 * Release the lock so someone else can access the freelist (or run
-		 * the clocksweep) while we check out this buffer.
-		 */
-		SpinLockRelease(&StrategyControl->buffer_strategy_lock);
+			/* Unconditionally remove buffer from freelist */
+			StrategyControl->firstFreeBuffer = buf->freeNext;
+			buf->freeNext = FREENEXT_NOT_IN_LIST;
 
-		/*
-		 * If the buffer is pinned or has a nonzero usage_count, we cannot use
-		 * it; discard it and retry.  (This can only happen if VACUUM put a
-		 * valid buffer in the freelist and then someone else used it before
-		 * we got to it.  It's probably impossible altogether as of 8.3, but
-		 * we'd better check anyway.)
-		 */
-		LockBufHdr(buf);
-		if (buf->refcount == 0 && buf->usage_count == 0)
-		{
-			if (strategy != NULL)
-				AddBufferToRing(strategy, buf);
-			return buf;
-		}
-		UnlockBufHdr(buf);
+			/*
+			 * Release the lock so someone else can access the freelist (or run
+			 * the clocksweep) while we check out this buffer.
+			 */
+			SpinLockRelease(&StrategyControl->buffer_strategy_lock);
 
-		/* Reacquire the lock and go around for another pass. */
-		SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
+			/*
+			 * If the buffer is pinned or has a nonzero usage_count, we cannot use
+			 * it; discard it and retry.  (This can only happen if VACUUM put a
+			 * valid buffer in the freelist and then someone else used it before
+			 * we got to it.  It's probably impossible altogether as of 8.3, but
+			 * we'd better check anyway.)
+			 */
+			LockBufHdr(buf);
+			if (buf->refcount == 0 && buf->usage_count == 0)
+			{
+				if (strategy != NULL)
+					AddBufferToRing(strategy, buf);
+				return buf;
+			}
+			UnlockBufHdr(buf);
+
+			/* Reacquire the lock and go around for another pass. */
+			SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
+		}
+		SpinLockRelease(&StrategyControl->buffer_strategy_lock);
 	}
 
 	/* Nothing on the freelist, so run the "clock sweep" algorithm */
 	trycounter = NBuffers;
 	for (;;)
 	{
-		buf = &BufferDescriptors[StrategyControl->nextVictimBuffer];
+		int victim;
+
+		victim = pg_atomic_fetch_add_u32(&StrategyControl->nextVictimBuffer, 1);
 
-		if (++StrategyControl->nextVictimBuffer >= NBuffers)
+		buf = &BufferDescriptors[victim % NBuffers];
+
+		if (victim % NBuffers == 0)
 		{
-			StrategyControl->nextVictimBuffer = 0;
-			StrategyControl->completePasses++;
+			pg_atomic_add_fetch_u32(&StrategyControl->completePasses, 1);
 		}
 
-		/* Release the lock before manipulating the candidate buffer. */
-		SpinLockRelease(&StrategyControl->buffer_strategy_lock);
-
 		/*
 		 * If the buffer is pinned or has a nonzero usage_count, we cannot use
 		 * it; decrement the usage_count (unless pinned) and keep scanning.
@@ -238,9 +268,6 @@ StrategyGetBuffer(BufferAccessStrategy strategy)
 			elog(ERROR, "no unpinned buffers available");
 		}
 		UnlockBufHdr(buf);
-
-		/* Reacquire the lock and get a new candidate buffer. */
-		SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
 	}
 }
 
@@ -284,13 +311,12 @@ StrategySyncStart(uint32 *complete_passes, uint32 *num_buf_alloc)
 	int			result;
 
 	SpinLockAcquire(&StrategyControl->buffer_strategy_lock);
-	result = StrategyControl->nextVictimBuffer;
+	result = pg_atomic_read_u32(&StrategyControl->nextVictimBuffer) % NBuffers;
 	if (complete_passes)
-		*complete_passes = StrategyControl->completePasses;
+		*complete_passes = pg_atomic_read_u32(&StrategyControl->completePasses);
 	if (num_buf_alloc)
 	{
-		*num_buf_alloc = StrategyControl->numBufferAllocs;
-		StrategyControl->numBufferAllocs = 0;
+		*num_buf_alloc = pg_atomic_exchange_u32(&StrategyControl->numBufferAllocs, 0);
 	}
 	SpinLockRelease(&StrategyControl->buffer_strategy_lock);
 	return result;
@@ -389,11 +415,11 @@ StrategyInitialize(bool init)
 		StrategyControl->lastFreeBuffer = NBuffers - 1;
 
 		/* Initialize the clock sweep pointer */
-		StrategyControl->nextVictimBuffer = 0;
+		pg_atomic_init_u32(&StrategyControl->nextVictimBuffer, 0);
 
 		/* Clear statistics */
-		StrategyControl->completePasses = 0;
-		StrategyControl->numBufferAllocs = 0;
+		pg_atomic_init_u32(&StrategyControl->completePasses, 0);
+		pg_atomic_init_u32(&StrategyControl->numBufferAllocs, 0);
 
 		/* No pending notification */
 		StrategyControl->bgwriterLatch = NULL;
-- 
1.8.3.251.g1462b67

#83Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#81)
Re: Wait free LW_SHARED acquisition - v0.9

On Fri, Oct 10, 2014 at 8:11 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-10 17:18:46 +0530, Amit Kapila wrote:

On Fri, Oct 10, 2014 at 1:27 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

Observations
----------------------
a. The patch performs really well (increase upto ~40%) incase all

the

data fits in shared buffers (scale factor -100).
b. Incase data doesn't fit in shared buffers, but fits in RAM
(scale factor -3000), there is performance increase upto 16 client

count,

however after that it starts dipping (in above config unto ~4.4%).

Hm. Interesting. I don't see that dip on x86.

Is it possible that implementation of some atomic operation is costlier
for particular architecture?

Yes, sure. And IIRC POWER improved atomics performance considerably for
POWER8...

I have tried again for scale factor 3000 and could see the dip and this
time I have even tried with 175 client count and the dip is

approximately

5% which is slightly more than 160 client count.

FWIW, the profile always looks like:

For my tests on Power8, the profile looks somewhat similar to below
profile mentioned by you, please see this mail:
/messages/by-id/CAA4eK1Je9ZBLHsfiavHD18GDwXUx21zFqPJgq_Dz_ZoA35nLpQ@mail.gmail.com

However on Power7, the profile looks different which I have
posted above thread.

BTW, that profile *clearly* indicates we should make StrategyGetBuffer()
smarter.

Yeah, even bgreclaimer patch is able to achieve the same, however
after that the contention moves to somewhere else as you can see
in above link.

Here it goes..

Lwlock_contention patches - client_count=128
----------------------------------------------------------------------

+   7.95%      postgres  postgres               [.] GetSnapshotData
+   3.58%      postgres  postgres               [.] AllocSetAlloc
+   2.51%      postgres  postgres               [.] _bt_compare
+   2.44%      postgres  postgres               [.]
hash_search_with_hash_value
+   2.33%      postgres  [kernel.kallsyms]      [k] .__copy_tofrom_user
+   2.24%      postgres  postgres               [.] AllocSetFreeIndex
+   1.75%      postgres  postgres               [.]
pg_atomic_fetch_add_u32_impl

Uh. Huh? Normally that'll be inline. That's compiled with gcc? What were
the compiler settings you used?

Nothing specific, for performance tests where I have to take profiles
I use below:
./configure --prefix=<installation_path> CFLAGS="-fno-omit-frame-pointer"
make

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#84Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#83)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-11 06:18:11 +0530, Amit Kapila wrote:

On Fri, Oct 10, 2014 at 8:11 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-10 17:18:46 +0530, Amit Kapila wrote:

On Fri, Oct 10, 2014 at 1:27 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

Observations
----------------------
a. The patch performs really well (increase upto ~40%) incase all

the

data fits in shared buffers (scale factor -100).
b. Incase data doesn't fit in shared buffers, but fits in RAM
(scale factor -3000), there is performance increase upto 16 client

count,

however after that it starts dipping (in above config unto ~4.4%).

Hm. Interesting. I don't see that dip on x86.

Is it possible that implementation of some atomic operation is costlier
for particular architecture?

Yes, sure. And IIRC POWER improved atomics performance considerably for
POWER8...

I have tried again for scale factor 3000 and could see the dip and this
time I have even tried with 175 client count and the dip is

approximately

5% which is slightly more than 160 client count.

I've run some short tests on hydra:

scale 1000:

base:
4GB:
tps = 296273.004800 (including connections establishing)
tps = 296373.978100 (excluding connections establishing)

8GB:
tps = 338001.455970 (including connections establishing)
tps = 338177.439106 (excluding connections establishing)

base + freelist:
4GB:
tps = 297057.523528 (including connections establishing)
tps = 297156.987418 (excluding connections establishing)

8GB:
tps = 335123.867097 (including connections establishing)
tps = 335239.122472 (excluding connections establishing)

base + LW_SHARED:
4GB:
tps = 296262.164455 (including connections establishing)
tps = 296357.524819 (excluding connections establishing)
8GB:
tps = 336988.744742 (including connections establishing)
tps = 337097.836395 (excluding connections establishing)

base + LW_SHARED + freelist:
4GB:
tps = 296887.981743 (including connections establishing)
tps = 296980.231853 (excluding connections establishing)

8GB:
tps = 345049.062898 (including connections establishing)
tps = 345161.947055 (excluding connections establishing)

I've also run some preliminary tests using scale=3000 - and I couldn't
see a performance difference either.

Note that all these are noticeably faster than your results.

Lwlock_contention patches - client_count=128
----------------------------------------------------------------------

+   7.95%      postgres  postgres               [.] GetSnapshotData
+   3.58%      postgres  postgres               [.] AllocSetAlloc
+   2.51%      postgres  postgres               [.] _bt_compare
+   2.44%      postgres  postgres               [.]
hash_search_with_hash_value
+   2.33%      postgres  [kernel.kallsyms]      [k] .__copy_tofrom_user
+   2.24%      postgres  postgres               [.] AllocSetFreeIndex
+   1.75%      postgres  postgres               [.]
pg_atomic_fetch_add_u32_impl

Uh. Huh? Normally that'll be inline. That's compiled with gcc? What were
the compiler settings you used?

Nothing specific, for performance tests where I have to take profiles
I use below:
./configure --prefix=<installation_path> CFLAGS="-fno-omit-frame-pointer"
make

Hah. Doing so overwrites the CFLAGS configure normally sets. Check
# CFLAGS are selected so:
# If the user specifies something in the environment, that is used.
# else: If the template file set something, that is used.
# else: If coverage was enabled, don't set anything.
# else: If the compiler is GCC, then we use -O2.
# else: If the compiler is something else, then we use -O, unless debugging.

so, if you do like above, you're compiling without optimizations... So,
include at least -O2 as well.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#85Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#84)
Re: Wait free LW_SHARED acquisition - v0.9

On Sat, Oct 11, 2014 at 6:29 AM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-11 06:18:11 +0530, Amit Kapila wrote:
I've run some short tests on hydra:

scale 1000:

base:
4GB:
tps = 296273.004800 (including connections establishing)
tps = 296373.978100 (excluding connections establishing)

8GB:
tps = 338001.455970 (including connections establishing)
tps = 338177.439106 (excluding connections establishing)

base + freelist:
4GB:
tps = 297057.523528 (including connections establishing)
tps = 297156.987418 (excluding connections establishing)

8GB:
tps = 335123.867097 (including connections establishing)
tps = 335239.122472 (excluding connections establishing)

base + LW_SHARED:
4GB:
tps = 296262.164455 (including connections establishing)
tps = 296357.524819 (excluding connections establishing)
8GB:
tps = 336988.744742 (including connections establishing)
tps = 337097.836395 (excluding connections establishing)

base + LW_SHARED + freelist:
4GB:
tps = 296887.981743 (including connections establishing)
tps = 296980.231853 (excluding connections establishing)

8GB:
tps = 345049.062898 (including connections establishing)
tps = 345161.947055 (excluding connections establishing)

I've also run some preliminary tests using scale=3000 - and I couldn't
see a performance difference either.

Note that all these are noticeably faster than your results.

What is the client count?
Could you please post numbers you are getting for 3000 scale
factor for client count 128 and 175?

Nothing specific, for performance tests where I have to take profiles
I use below:
./configure --prefix=<installation_path>

CFLAGS="-fno-omit-frame-pointer"

make

Hah. Doing so overwrites the CFLAGS configure normally sets. Check
# CFLAGS are selected so:
# If the user specifies something in the environment, that is used.
# else: If the template file set something, that is used.
# else: If coverage was enabled, don't set anything.
# else: If the compiler is GCC, then we use -O2.
# else: If the compiler is something else, then we use -O, unless

debugging.

so, if you do like above, you're compiling without optimizations... So,
include at least -O2 as well.

Hmm. okay, but is this required when we do actual performance
tests, because for that currently I don't use CFLAGS.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#86Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#85)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-11 06:49:54 +0530, Amit Kapila wrote:

On Sat, Oct 11, 2014 at 6:29 AM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-11 06:18:11 +0530, Amit Kapila wrote:
I've run some short tests on hydra:

scale 1000:

base:
4GB:
tps = 296273.004800 (including connections establishing)
tps = 296373.978100 (excluding connections establishing)

8GB:
tps = 338001.455970 (including connections establishing)
tps = 338177.439106 (excluding connections establishing)

base + freelist:
4GB:
tps = 297057.523528 (including connections establishing)
tps = 297156.987418 (excluding connections establishing)

8GB:
tps = 335123.867097 (including connections establishing)
tps = 335239.122472 (excluding connections establishing)

base + LW_SHARED:
4GB:
tps = 296262.164455 (including connections establishing)
tps = 296357.524819 (excluding connections establishing)
8GB:
tps = 336988.744742 (including connections establishing)
tps = 337097.836395 (excluding connections establishing)

base + LW_SHARED + freelist:
4GB:
tps = 296887.981743 (including connections establishing)
tps = 296980.231853 (excluding connections establishing)

8GB:
tps = 345049.062898 (including connections establishing)
tps = 345161.947055 (excluding connections establishing)

I've also run some preliminary tests using scale=3000 - and I couldn't
see a performance difference either.

Note that all these are noticeably faster than your results.

What is the client count?

160, because that was the one you reported the biggest regression.

Could you please post numbers you are getting for 3000 scale
factor for client count 128 and 175?

Yes, although not tonight.... Also from hydra?

Nothing specific, for performance tests where I have to take profiles
I use below:
./configure --prefix=<installation_path>

CFLAGS="-fno-omit-frame-pointer"

make

Hah. Doing so overwrites the CFLAGS configure normally sets. Check
# CFLAGS are selected so:
# If the user specifies something in the environment, that is used.
# else: If the template file set something, that is used.
# else: If coverage was enabled, don't set anything.
# else: If the compiler is GCC, then we use -O2.
# else: If the compiler is something else, then we use -O, unless

debugging.

so, if you do like above, you're compiling without optimizations... So,
include at least -O2 as well.

Hmm. okay, but is this required when we do actual performance
tests, because for that currently I don't use CFLAGS.

I'm not sure what you mean? You need to include -O2 in CFLAGS whenever
you override it. Your profile was clearly without inlining... And since
your general performance numbers are a fair bit lower than what I see
with, hopefully, the same code on the same machine...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#87Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#86)
Re: Wait free LW_SHARED acquisition - v0.9

On Sat, Oct 11, 2014 at 7:00 AM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-11 06:49:54 +0530, Amit Kapila wrote:

On Sat, Oct 11, 2014 at 6:29 AM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-11 06:18:11 +0530, Amit Kapila wrote:
I've run some short tests on hydra:

Could you please post numbers you are getting for 3000 scale
factor for client count 128 and 175?

Yes, although not tonight....

No issues, whenever you get it.

Also from hydra?

Yes. One more thing I would like to share with you is that while doing
tests, there are some other settings change in postgresql.conf

maintenance_work_mem = 1GB
synchronous_commit = off
wal_writer_delay = 20ms
checkpoint_segments=256
checkpoint_timeout =35min

I don't think these parameters matter for the tests we are doing, but
still I thought it is good to share, because I forgot to send some of
these non-default settings in previous mail.

Nothing specific, for performance tests where I have to take

profiles

I use below:
./configure --prefix=<installation_path>

CFLAGS="-fno-omit-frame-pointer"

make

Hah. Doing so overwrites the CFLAGS configure normally sets. Check
# CFLAGS are selected so:
# If the user specifies something in the environment, that is used.
# else: If the template file set something, that is used.
# else: If coverage was enabled, don't set anything.
# else: If the compiler is GCC, then we use -O2.
# else: If the compiler is something else, then we use -O, unless

debugging.

so, if you do like above, you're compiling without optimizations...

So,

include at least -O2 as well.

Hmm. okay, but is this required when we do actual performance
tests, because for that currently I don't use CFLAGS.

I'm not sure what you mean? You need to include -O2 in CFLAGS whenever
you override it.

okay, thats what I wanted to ask you, so that we should not see different
numbers due to the way code is built.

When I do performance tests where I don't want to see profile,
I use below statement:
./configure --prefix=<installation_path>

And since
your general performance numbers are a fair bit lower than what I see
with, hopefully, the same code on the same machine...

You have reported numbers at 1000 scale factor and mine were
at 3000 scale factor, so I think the difference is expected.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#88Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#87)
Re: Wait free LW_SHARED acquisition - v0.9

Hi,

On 2014-10-11 07:26:57 +0530, Amit Kapila wrote:

On Sat, Oct 11, 2014 at 7:00 AM, Andres Freund <andres@2ndquadrant.com>

And since
your general performance numbers are a fair bit lower than what I see
with, hopefully, the same code on the same machine...

You have reported numbers at 1000 scale factor and mine were
at 3000 scale factor, so I think the difference is expected.

The numbers for 3000 show pretty much the same:

SCALE 128 160 175
HEAD 352113 339005 336491
LW_SHARED 365874 347931 342528

Hm. I wonder if you're using pgbench without -M prepared? That'd about
explain the difference.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#89Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#88)
Re: Wait free LW_SHARED acquisition - v0.9

On Sat, Oct 11, 2014 at 6:40 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-11 07:26:57 +0530, Amit Kapila wrote:

On Sat, Oct 11, 2014 at 7:00 AM, Andres Freund <andres@2ndquadrant.com>

And since
your general performance numbers are a fair bit lower than what I see
with, hopefully, the same code on the same machine...

You have reported numbers at 1000 scale factor and mine were
at 3000 scale factor, so I think the difference is expected.

The numbers for 3000 show pretty much the same:

SCALE 128 160 175
HEAD 352113 339005 336491
LW_SHARED 365874 347931 342528

Hm. I wonder if you're using pgbench without -M prepared?

No, I use below statement:
./pgbench -c 128 -j 128 -T 300 -S -M prepared postgres

That'd about
explain the difference.

Here I think first thing to clarify is why the numbers on HEAD are
different? Another thing is that I generally see difference in
numbers at 1000 and 3000 scale factor (although I have not run
lately), but in your case the numbers are almost same.

I will try once more by cleaning every thing(installation, data_dir, etc..)
but not today...

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#90Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#88)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-11 15:10:45 +0200, Andres Freund wrote:

Hi,

On 2014-10-11 07:26:57 +0530, Amit Kapila wrote:

On Sat, Oct 11, 2014 at 7:00 AM, Andres Freund <andres@2ndquadrant.com>

And since
your general performance numbers are a fair bit lower than what I see
with, hopefully, the same code on the same machine...

You have reported numbers at 1000 scale factor and mine were
at 3000 scale factor, so I think the difference is expected.

The numbers for 3000 show pretty much the same:

SCALE 128 160 175
HEAD 352113 339005 336491
LW_SHARED 365874 347931 342528

Hm. I wonder if you're using pgbench without -M prepared? That'd about
explain the difference.

Started a test for a run without -M prepared (i.e. -M simple):

SCALE 1 2 4 8 16 32 64 96 128 160 196
HEAD 7968 15132 31147 63395 123551 180436 242098 263625 249652 244240 232679
LW_SHARED 8268 15032 31267 63911 118943 180447 247067 269262 259165 247166 231292

This really doesn't look exciting to me.

scale 200, -M prepared:
SCALE 1 2 4 8 16 32 64 96 128 160 196
HEAD 13032 24712 50967 103801 201745 279226 380844 392623 379953 357206 361072
LW_SHARED 12997 25134 51119 102920 199597 282511 422424 460222 447662 436959 418519

My guess is that the primary benefit on systems with few sockets, like
this, is that with the new code there's far fewer problems with
processes sleeping (being scheduled out) while holding a spinlock.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#91Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#89)
Re: Wait free LW_SHARED acquisition - v0.9

On Sat, Oct 11, 2014 at 7:02 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Sat, Oct 11, 2014 at 6:40 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

On 2014-10-11 07:26:57 +0530, Amit Kapila wrote:

On Sat, Oct 11, 2014 at 7:00 AM, Andres Freund <andres@2ndquadrant.com

And since
your general performance numbers are a fair bit lower than what I

see

with, hopefully, the same code on the same machine...

You have reported numbers at 1000 scale factor and mine were
at 3000 scale factor, so I think the difference is expected.

The numbers for 3000 show pretty much the same:

SCALE 128 160 175
HEAD 352113 339005 336491
LW_SHARED 365874 347931 342528

Hm. I wonder if you're using pgbench without -M prepared?

No, I use below statement:
./pgbench -c 128 -j 128 -T 300 -S -M prepared postgres

That'd about
explain the difference.

Here I think first thing to clarify is why the numbers on HEAD are
different?

I have taken the latest code and recreated the database and tried
again on power-7 m/c (hydra) and below is the result:

Result with -M prepared:
Duration of each individual run - 5 mins

HEAD – commit 494affb

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 128 160 Run-1 258385 239908 Run-2 257835
238624 Run-3 255967 237905

Result without -M prepared:
Duration of each individual run - 5 mins

HEAD – commit 494affb

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 128 160 Run-1 228747 220961 Run-2 229817
214464 Run-3 227386 216619

I am not sure why we are seeing difference even though running
on same m/c with same configuration. I think there is some
difference in the way we are running tests, if you don't mind
could you please share the exact steps and non-default postgresql.conf
settings with me. The below list of things could be useful for
me to reproduce the numbers you are seeing:

a. build steps (any script you are using)
b. non-default postgresql.conf settings
c. Exact pgbench statements used

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#92Merlin Moncure
mmoncure@gmail.com
In reply to: Andres Freund (#82)
Re: Wait free LW_SHARED acquisition - v0.9

On Fri, Oct 10, 2014 at 11:00 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-10-10 16:41:39 +0200, Andres Freund wrote:

FWIW, the profile always looks like
-  48.61%      postgres  postgres              [.] s_lock
- s_lock
+ 96.67% StrategyGetBuffer
+ 1.19% UnpinBuffer
+ 0.90% PinBuffer
+ 0.70% hash_search_with_hash_value
+   3.11%      postgres  postgres              [.] GetSnapshotData
+   2.47%      postgres  postgres              [.] StrategyGetBuffer
+   1.93%      postgres  [kernel.kallsyms]     [k] copy_user_generic_string
+   1.28%      postgres  postgres              [.] hash_search_with_hash_value
-   1.27%      postgres  postgres              [.] LWLockAttemptLock
- LWLockAttemptLock
- 97.78% LWLockAcquire
+ 38.76% ReadBuffer_common
+ 28.62% _bt_getbuf
+ 8.59% _bt_relandgetbuf
+ 6.25% GetSnapshotData
+ 5.93% VirtualXactLockTableInsert
+ 3.95% VirtualXactLockTableCleanup
+ 2.35% index_fetch_heap
+ 1.66% StartBufferIO
+ 1.56% LockReleaseAll
+ 1.55% _bt_next
+ 0.78% LockAcquireExtended
+ 1.47% _bt_next
+ 0.75% _bt_relandgetbuf

to me. Now that's with the client count 496, but it's similar with lower
counts.

BTW, that profile *clearly* indicates we should make StrategyGetBuffer()
smarter.

Which is nearly trivial now that atomics are in. Check out the attached
WIP patch which eliminates the spinlock from StrategyGetBuffer() unless
there's buffers on the freelist.

Is this safe?

+ victim = pg_atomic_fetch_add_u32(&StrategyControl->nextVictimBuffer, 1);

- if (++StrategyControl->nextVictimBuffer >= NBuffers)
+ buf = &BufferDescriptors[victim % NBuffers];
+
+ if (victim % NBuffers == 0)
<snip>

I don't think there's any guarantee you could keep nextVictimBuffer
from wandering off the end. You could buy a little safety by CAS'ing
zero in instead, followed by atomic increment. However that's still
pretty dodgy IMO and requires two atomic ops which will underperform
the spin in some situations.

I like Robert's idea to keep the spinlock but preallocate a small
amount of buffers, say 8 or 16.

merlin

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#93Andres Freund
andres@2ndquadrant.com
In reply to: Merlin Moncure (#92)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-14 08:40:49 -0500, Merlin Moncure wrote:

On Fri, Oct 10, 2014 at 11:00 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-10-10 16:41:39 +0200, Andres Freund wrote:

FWIW, the profile always looks like
-  48.61%      postgres  postgres              [.] s_lock
- s_lock
+ 96.67% StrategyGetBuffer
+ 1.19% UnpinBuffer
+ 0.90% PinBuffer
+ 0.70% hash_search_with_hash_value
+   3.11%      postgres  postgres              [.] GetSnapshotData
+   2.47%      postgres  postgres              [.] StrategyGetBuffer
+   1.93%      postgres  [kernel.kallsyms]     [k] copy_user_generic_string
+   1.28%      postgres  postgres              [.] hash_search_with_hash_value
-   1.27%      postgres  postgres              [.] LWLockAttemptLock
- LWLockAttemptLock
- 97.78% LWLockAcquire
+ 38.76% ReadBuffer_common
+ 28.62% _bt_getbuf
+ 8.59% _bt_relandgetbuf
+ 6.25% GetSnapshotData
+ 5.93% VirtualXactLockTableInsert
+ 3.95% VirtualXactLockTableCleanup
+ 2.35% index_fetch_heap
+ 1.66% StartBufferIO
+ 1.56% LockReleaseAll
+ 1.55% _bt_next
+ 0.78% LockAcquireExtended
+ 1.47% _bt_next
+ 0.75% _bt_relandgetbuf

to me. Now that's with the client count 496, but it's similar with lower
counts.

BTW, that profile *clearly* indicates we should make StrategyGetBuffer()
smarter.

Which is nearly trivial now that atomics are in. Check out the attached
WIP patch which eliminates the spinlock from StrategyGetBuffer() unless
there's buffers on the freelist.

Is this safe?

+ victim = pg_atomic_fetch_add_u32(&StrategyControl->nextVictimBuffer, 1);

- if (++StrategyControl->nextVictimBuffer >= NBuffers)
+ buf = &BufferDescriptors[victim % NBuffers];
+
+ if (victim % NBuffers == 0)
<snip>

I don't think there's any guarantee you could keep nextVictimBuffer
from wandering off the end.

Not sure what you mean? It'll cycle around at 2^32. The code doesn't try
to avoid nextVictimBuffer from going above NBuffers. To avoid overrunning
&BufferDescriptors I'm doing % NBuffers.

Yes, that'll have the disadvantage of the first buffers being slightly
more likely to be hit, but for that to be relevant you'd need
unrealistically large amounts of shared_buffers.

You could buy a little safety by CAS'ing
zero in instead, followed by atomic increment. However that's still
pretty dodgy IMO and requires two atomic ops which will underperform
the spin in some situations.

I like Robert's idea to keep the spinlock but preallocate a small
amount of buffers, say 8 or 16.

I think that's pretty much orthogonal. I *do* think it makes sense to
increment nextVictimBuffer in bigger steps. But the above doesn't
prohibit doing so - and it'll still be be much better without the
spinlock. Same number of atomics, but no potential of spinning and no
potential of being put to sleep while holding the spinlock.

This callsite has a comparatively large likelihood of being put to sleep
while holding a spinlock because it can run for a very long time doing
nothing but spinlocking.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#94Merlin Moncure
mmoncure@gmail.com
In reply to: Andres Freund (#93)
Re: Wait free LW_SHARED acquisition - v0.9

On Tue, Oct 14, 2014 at 8:58 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-10-14 08:40:49 -0500, Merlin Moncure wrote:

On Fri, Oct 10, 2014 at 11:00 AM, Andres Freund <andres@2ndquadrant.com> wrote:

Which is nearly trivial now that atomics are in. Check out the attached
WIP patch which eliminates the spinlock from StrategyGetBuffer() unless
there's buffers on the freelist.

Is this safe?

+ victim = pg_atomic_fetch_add_u32(&StrategyControl->nextVictimBuffer, 1);

- if (++StrategyControl->nextVictimBuffer >= NBuffers)
+ buf = &BufferDescriptors[victim % NBuffers];
+
+ if (victim % NBuffers == 0)
<snip>

I don't think there's any guarantee you could keep nextVictimBuffer
from wandering off the end.

Not sure what you mean? It'll cycle around at 2^32. The code doesn't try
to avoid nextVictimBuffer from going above NBuffers. To avoid overrunning
&BufferDescriptors I'm doing % NBuffers.

Yes, that'll have the disadvantage of the first buffers being slightly
more likely to be hit, but for that to be relevant you'd need
unrealistically large amounts of shared_buffers.

Right -- my mistake. That's clever. I think that should work well.

I think that's pretty much orthogonal. I *do* think it makes sense to
increment nextVictimBuffer in bigger steps. But the above doesn't
prohibit doing so - and it'll still be be much better without the
spinlock. Same number of atomics, but no potential of spinning and no
potential of being put to sleep while holding the spinlock.

This callsite has a comparatively large likelihood of being put to sleep
while holding a spinlock because it can run for a very long time doing
nothing but spinlocking.

A while back, I submitted a minor tweak to the clock sweep so that,
instead of spinlocking every single buffer header as it swept it just
did a single TAS as a kind of a trylock and punted to the next buffer
if the test failed on the principle there's not good reason to hang
around. You only spin if you passed the first test; that should
reduce the likelihood of actual spinning to approximately zero. I
still maintain there's no reason not to do that (I couldn't show a
benefit but that was because mapping list locking was masking any
clock sweep contention at that time).

merlin

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#95Amit Kapila
amit.kapila16@gmail.com
In reply to: Merlin Moncure (#94)
Re: Wait free LW_SHARED acquisition - v0.9

On Wed, Oct 15, 2014 at 12:06 AM, Merlin Moncure <mmoncure@gmail.com> wrote:

A while back, I submitted a minor tweak to the clock sweep so that,
instead of spinlocking every single buffer header as it swept it just
did a single TAS as a kind of a trylock and punted to the next buffer
if the test failed on the principle there's not good reason to hang
around. You only spin if you passed the first test; that should
reduce the likelihood of actual spinning to approximately zero. I
still maintain there's no reason not to do that (I couldn't show a
benefit but that was because mapping list locking was masking any
clock sweep contention at that time).

If you feel that can now show the benefit, then I think you can rebase
it for the coming commit fest (which is going to start today).

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#96Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#91)
Re: Wait free LW_SHARED acquisition - v0.9

On Tue, Oct 14, 2014 at 11:34 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

I am not sure why we are seeing difference even though running
on same m/c with same configuration.

I have tried many times, but I could not get the numbers you have
posted above with HEAD, however now trying with the latest version
[1]: /messages/by-id/20141010111027.GC6670@alap3.anarazel.de
The data at higher client count is as below:

HEAD – commit 494affb

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 64 128 Run-1 271799 247777 Run-2 274341
245207 Run-3 275019 252258

HEAD – commit 494affb + wait free lw_shared_v2

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 64 128 Run-1 286209 274922 Run-2 289101
274495 Run-3 289639 273633

So I am planning to proceed further with the review/test of your
latest patch.

According to me, below things are left from myside:
a. do some basic tpc-b tests with patch
b. re-review latest version posted by you

I know that you have posted optimization into StrategyGetBuffer() in
this thread, however I feel we can evaluate it separately unless you
are of opinion that both the patches should go together.

[1]: /messages/by-id/20141010111027.GC6670@alap3.anarazel.de
/messages/by-id/20141010111027.GC6670@alap3.anarazel.de

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#97Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#96)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-17 17:14:16 +0530, Amit Kapila wrote:

On Tue, Oct 14, 2014 at 11:34 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

I am not sure why we are seeing difference even though running
on same m/c with same configuration.

I have tried many times, but I could not get the numbers you have
posted above with HEAD, however now trying with the latest version
[1] posted by you, everything seems to be fine at this workload.
The data at higher client count is as below:

I'll try to reproduce it next week. But I don't think it matters all
that much. Personally so far the performance numbers don't seem to
indicate much reason to wait any further. We sure improve further, but I
don't see much reason to wait because of that.

HEAD – commit 494affb

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 64 128 Run-1 271799 247777 Run-2 274341
245207 Run-3 275019 252258

HEAD – commit 494affb + wait free lw_shared_v2

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 64 128 Run-1 286209 274922 Run-2 289101
274495 Run-3 289639 273633

So here the results with LW_SHARED were consistently better, right? You
saw performance degradations here earlier?

So I am planning to proceed further with the review/test of your
latest patch.

According to me, below things are left from myside:
a. do some basic tpc-b tests with patch
b. re-review latest version posted by you

Cool!

I know that you have posted optimization into StrategyGetBuffer() in
this thread, however I feel we can evaluate it separately unless you
are of opinion that both the patches should go together.

[1]
/messages/by-id/20141010111027.GC6670@alap3.anarazel.de

No, I don't think they should go together - I wrote that patch because
it was the bottleneck in the possibly regressing test and I wanted to
see the full effect. Although I do think we should apply it ;)

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#98Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#97)
1 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.9

On Fri, Oct 17, 2014 at 11:41 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-17 17:14:16 +0530, Amit Kapila wrote:

On Tue, Oct 14, 2014 at 11:34 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
HEAD – commit 494affb + wait free lw_shared_v2

Shared_buffers=8GB; Scale Factor = 3000

Client Count/No. Of Runs (tps) 64 128 Run-1 286209 274922 Run-2

289101

274495 Run-3 289639 273633

So here the results with LW_SHARED were consistently better, right?

Yes.

You
saw performance degradations here earlier?

Yes.

So I am planning to proceed further with the review/test of your
latest patch.

According to me, below things are left from myside:
a. do some basic tpc-b tests with patch

I have done few tests, the results of which are below, the data indicates
that neither there is any noticeable gain nor any noticeable loss on tpc-b
tests which I think is what could have been expected of this patch.
There is slight variation at few client counts (for sync_commit =off,
at 32 and 128), however I feel that is just noise as I don't see any
general trend.

Performance Data
----------------------------
IBM POWER-8 24 cores, 192 hardware threads
RAM = 492GB
Database Locale =C
max_connections =300
checkpoint_segments=300
checkpoint_timeout =15min
maintenance_work_mem = 1GB
checkpoint_completion_target = 0.9
Client Count = number of concurrent sessions and threads (ex. -c 8 -j 8)
Duration of each individual run = 30mins
Test mode - tpc-b

Below data is median of 3 runs, detailed data is attached with this
mail.

Scale_factor =3000; shared_buffers=8GB;

Patch/Client_count 8 16 32 64 128 HEAD 3849 4889 3569 3845 4547
LW_SHARED 3844 4787 3532 3814 4408

Scale_factor =3000; shared_buffers=8GB; synchronous_commit=off;

Patch/Client_count 8 16 32 64 128 HEAD 5966 8297 10084 9348 8836
LW_SHARED 6070 8612 8839 9503 8584

While doing performance tests, I noticed a hang at higher client
counts with patch. I have tried to check call stack for few of
processes and it is as below:

#0 0x0000008010933e54 in .semop () from /lib64/libc.so.6
#1 0x0000000010286e48 in .PGSemaphoreLock ()
#2 0x00000000102f68bc in .LWLockAcquire ()
#3 0x00000000102d1ca0 in .ReadBuffer_common ()
#4 0x00000000102d2ae0 in .ReadBufferExtended ()
#5 0x00000000100a57d8 in ._bt_getbuf ()
#6 0x00000000100a6210 in ._bt_getroot ()
#7 0x00000000100aa910 in ._bt_search ()
#8 0x00000000100ab494 in ._bt_first ()
#9 0x00000000100a8e84 in .btgettuple ()
..

#0 0x0000008010933e54 in .semop () from /lib64/libc.so.6
#1 0x0000000010286e48 in .PGSemaphoreLock ()
#2 0x00000000102f68bc in .LWLockAcquire ()
#3 0x00000000102d1ca0 in .ReadBuffer_common ()
#4 0x00000000102d2ae0 in .ReadBufferExtended ()
#5 0x00000000100a57d8 in ._bt_getbuf ()
#6 0x00000000100a6210 in ._bt_getroot ()
#7 0x00000000100aa910 in ._bt_search ()
#8 0x00000000100ab494 in ._bt_first ()
...

The test configuration is as below:
Test env - Power - 7 (hydra)
scale_factor - 3000
shared_buffers - 8GB
test mode - pgbench read only

test execution -
./pgbench -c 128 -j 128 -T 1800 -S -M prepared postgres

I have ran it for half an hour, but it doesn't came out even after
~2 hours. It doesn't get reproduced every time, currently I am
able to reproduce it and the m/c is in same state, if you want any
info, let me know (unfortunately binaries are in release mode, so
might not get enough information).

b. re-review latest version posted by you

Cool!

I will post my feedback for code separately, once I am able to
completely review the new versions.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

Attachments:

perf_lwlock_contention_tpcb_data_v1.odsapplication/vnd.oasis.opendocument.spreadsheet; name=perf_lwlock_contention_tpcb_data_v1.odsDownload
#99Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#62)
Re: Wait free LW_SHARED acquisition - v0.2

On Wed, Oct 8, 2014 at 6:17 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

2.
LWLockWakeup()
{
..
#ifdef LWLOCK_STATS
lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
#else
SpinLockAcquire(&lock->mutex);
#endif
..
}

Earlier while releasing lock, we don't count it towards LWLock stats
spin_delay_count. I think if we see other places in lwlock.c, it only

gets

counted when we try to acquire it in a loop.

I think the previous situation was clearly suboptimal. I've now modified
things so all spinlock acquirations are counted.

Code has mainly 4 stats (sh_acquire_count, ex_acquire_count,
block_count, spin_delay_count) to track, if I try to see
all stats together to understand the contention situation,
the unpatched code makes sense. spin_delay_count gives
how much delay has happened to acquire spinlock which when
combined with other stats gives the clear situation about
the contention around aquisation of corresponding LWLock.
Now if we want to count the spin lock delay for Release call
as well, then the meaning of the stat is getting changed.
It might be that new meaning of spin_delay_count stat is more
useful in some situations, however the older one has its own
benefits, so I am not sure if changing this as part of this
patch is the best thing to do.

3.
LWLockRelease()
{
..
/* grant permission to run, even if a spurious share lock increases
lockcount */
else if (mode == LW_EXCLUSIVE && have_waiters)
check_waiters = true;
/* nobody has this locked anymore, potential exclusive lockers get a

chance

*/
else if (lockcount == 0 && have_waiters)
check_waiters = true;
..
}

It seems comments have been reversed in above code.

No, they look right. But I've expanded them in the version I'm going to
post in a couple minutes.

okay.

5.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &wakeup)
{
PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
LOG_LWDEBUG("LWLockRelease", l, mode, "release waiter");
dlist_delete(&waiter->lwWaitLink);
pg_write_barrier();
waiter->lwWaiting = false;
PGSemaphoreUnlock(&waiter->sem);
}
..
}

Why can't we decrement the nwaiters after waking up? I don't think
there is any major problem even if callers do that themselves, but
in some rare cases LWLockRelease() might spuriously assume that
there are some waiters and tries to call LWLockWakeup(). Although
this doesn't create any problem, keeping the value sane is good unless
there is some problem in doing so.

That won't work because then LWLockWakeup() wouldn't be called when
necessary - precisely because nwaiters is 0.

The reason I've done so is that it's otherwise much harder to debug
issues where there are backend that have been woken up already, but
haven't rerun yet. Without this there's simply no evidence of that
state. I can't see this being relevant for performance, so I'd rather
have it stay that way.

I am not sure what useful information we can get during debugging by not
doing this in LWLockWakeup() and w.r.t performance it can lead extra
function call, few checks and I think in some cases even can
acquire/release spinlock.

6.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
continue;
..
if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
{
..
wokeup_somebody = true;
}
..
}
..
}

a.
IIUC above logic, if the waiter queue is as follows:
(S-Shared; X-Exclusive) S X S S S X S S

it can skip the exclusive waiters and release shared waiter.

If my understanding is right, then I think instead of continue, there
should be *break* in above logic.

No, it looks correct to me. What happened is that the first S was woken
up. So there's no point in waking up an exclusive locker, but further
non-exclusive lockers can be woken up.

Okay, even then it makes the current logic of wakingup
different which I am not sure is what this patch is intended
for.

b.
Consider below sequence of waiters:
(S-Shared; X-Exclusive) S S X S S

I think as per un-patched code, it will wakeup waiters uptill

(including)

first Exclusive, but patch will wake up uptill (*excluding*) first
Exclusive.

I don't think the current code does that.

LWLockRelease()
{
..
/*
* If the front waiter wants exclusive lock, awaken him only.
*
Otherwise awaken as many waiters as want shared access.
*/
if (proc-

lwWaitMode != LW_EXCLUSIVE)

{
while (proc->lwWaitLink !=
NULL &&
proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
{
if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
releaseOK = false;
proc = proc->lwWaitLink;
}
}
/* proc is now the last PGPROC to be
released */
lock->head = proc->lwWaitLink;
proc->lwWaitLink = NULL;
..
}

In the above code, if the first waiter to be woken up is Exclusive waiter,
then it will woke that waiter, else shared waiters till it got
the first exclusive waiter and then first exlusive waiter.

And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time it'd
make sense for X to be woken up is when it gets run faster than the S
processes.

Do we get any major benefit by changing the logic of waking up waiters?

7.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
dlist_delete(&waiter->lwWaitLink);
dlist_push_tail(&wakeup, &waiter->lwWaitLink);
..
}
..
}

Use of dlist has simplified the code, but I think there might be a

slight

overhead of maintaining wakeup queue as compare to un-patched
mechanism especially when there is a long waiter queue.

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.

Okay, however I see Robert has also raised a point on this issue
which I am not sure is concluded.

And the code is *so* much more readable.

Code is more readable, but I don't understand why you
want to do refactoring as part of this patch which ideally
doesn't get any benefit from the same.

8.
LWLockConditionalAcquire()
{
..
/*
* We ran into an exclusive lock and might have blocked another
* exclusive lock from taking a shot because it took a time to back
* off. Retry till we are either sure we didn't block somebody (because
* somebody else certainly has the lock) or till we got it.
*
* We cannot rely on the two-step lock-acquisition protocol as in
* LWLockAcquire because we're not using it.
*/
if (potentially_spurious)
{
SPIN_DELAY();
goto retry;
}
..
}

Due to above logic, I think it can keep on retrying for long time before
it actually concludes whether it got lock or not incase other backend/'s
takes Exclusive lock after *double_check* and release before
unconditional increment of shared lock in function LWLockAttemptLock.
I understand that it might be difficult to have such a practical

scenario,

however still there is a theoratical possibility of same.

I'm not particularly concerned. We could optimize it a bit, but I really
don't think it's necessary.

No issues, I was slightly worried about cases where this
API wasn't suppose to take time earlier (like for contentlock in
BufferAlloc), but now it starts taking time. I am not able to see
any specific issue, so let's proceed with keeping the code
as you have in patch and will optimize later incase we face any problem.

Is there any advantage of retrying in LWLockConditionalAcquire()?

It's required for correctness. We only retry if we potentially blocked
an exclusive acquirer (by spuriously incrementing/decrementing lockcount
with 1). We need to be sure to either get the lock (in which case we can
wake up the waiter on release), or be sure that we didn't disturb
anyone.

Okay, got the point.

9.
LWLockAcquireOrWait()
{
..
/*
* NB: We're using nearly the same twice-in-a-row lock acquisition
* protocol as LWLockAcquire(). Check its comments for details.
*/
mustwait = LWLockAttemptLock(l, mode, false,

&potentially_spurious_first);

if (mustwait)
{
LWLockQueueSelf(l, LW_WAIT_UNTIL_FREE);

mustwait = LWLockAttemptLock(l, mode, false,

&potentially_spurious_second);

}

In this function, it doesn't seem to be required to use the return value
*mustwait* of second LWLockAttemptLock() call as a return value of
function, as per usage of this function if we don't get the lock at

first

attempt, then it needs to check if the corresponding WAL record is
flushed.

I don't think that's the appropriate comparison. Acquiring the lock in
the lock in the new implementation essentially consists out of these two
steps. We *DID* get the lock here. Without sleeping. So returning the
appropriate return code is correct.
In fact, returning false would break things, because the caller would
hold the lock without freeing it again?

Okay, I understand this point, but OTOH with this new logic
(attempt 2 times), in certain cases backends will get WALWriteLock,
when it is not really required which I am not sure can cause any problem,
so lets retain it as you have done in patch.

11.
LWLockAcquireOrWait()
{
..
Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
}

Isn't it better to use AssertArg() rather than Assert in above usage?

I've changed that. But I have to admit, I don't really see the point of
AssertArg(). If it'd output the value, then it'd be beneficial, but it
doesn't.

It seems you forgot to change in LWLockAcquireOrWait(), refer
below code.
@@ -773,14 +1191,14 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
  int extraWaits = 0;
 #ifdef LWLOCK_STATS
  lwlock_stats *lwstats;
-#endif
-
- PRINT_LWDEBUG("LWLockAcquireOrWait", lock);

-#ifdef LWLOCK_STATS
lwstats = get_lwlock_stats_entry(lock);
#endif

+ Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+

15.
/* must be greater than MAX_BACKENDS */
#define SHARED_LOCK_MASK (~EXCLUSIVE_LOCK)

a. how can we guarantee it to be greater than MaxBackends,
as MaxBackends is int (the max value for which will be
equal to SHARED_LOCK_MASK)?

MaxBackends luckily is limited to something lower. I've added a comment
to that regard.

b. This is used only for LWLOCK_STATS, so shouldn't we
define this under LWLOCK_STATS.

It's a general value, so I don't think that's appropriate.

No issues.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#100Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#99)
Re: Wait free LW_SHARED acquisition - v0.2

On Tue, Oct 21, 2014 at 7:56 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Wed, Oct 8, 2014 at 6:17 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

Today, I have verified all previous comments raised by
me and looked at new code and below are my findings:

4.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
..
}

Setting releaseOK in above context might not be required because if the
control comes in this part of code, it will not retry to acquire another
time.

Hm. You're probably right.

You have agreed to fix this comment, but it seems you have forgot
to change it.

11.
LWLockRelease()
{
..
PRINT_LWDEBUG("LWLockRelease", lock, mode);
}

Shouldn't this be in begining of LWLockRelease function rather than
after processing held_lwlocks array?

Ok.

You have agreed to fix this comment, but it seems you have forgot
to change it.

Below comment doesn't seem to be adressed?

LWLockAcquireOrWait()
{
..
LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "succeeded");
..
}

a. such a log is not there in any other LWLock.. variants,
if we want to introduce it, then shouldn't it be done at
other places as well.

Below point is yet to be resolved.

12.
#ifdef LWLOCK_DEBUG
lock->owner = MyProc;
#endif

Shouldn't it be reset in LWLockRelease?

That's actually intentional. It's quite useful to know the last owner
when debugging lwlock code.

Won't it cause any problem if the last owner process exits?

Can you explain how pg_read_barrier() in below code makes this
access safe?

LWLockWakeup()
{
..
+ pg_read_barrier(); /* pairs with nwaiters-- */
+ if (!BOOL_ACCESS_ONCE(lock->releaseOK))
..
}

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#101Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#100)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-22 13:32:07 +0530, Amit Kapila wrote:

On Tue, Oct 21, 2014 at 7:56 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Wed, Oct 8, 2014 at 6:17 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

Today, I have verified all previous comments raised by
me and looked at new code and below are my findings:

4.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
..
}

Setting releaseOK in above context might not be required because if the
control comes in this part of code, it will not retry to acquire another
time.

Hm. You're probably right.

You have agreed to fix this comment, but it seems you have forgot
to change it.

After I've thought more about it, it's is actually required. This
essentially *is* a retry. Someobdy woke us up, which is where releaseOK
is supposed to be set.

11.
LWLockRelease()
{
..
PRINT_LWDEBUG("LWLockRelease", lock, mode);
}

Shouldn't this be in begining of LWLockRelease function rather than
after processing held_lwlocks array?

Ok.

You have agreed to fix this comment, but it seems you have forgot
to change it.

Below comment doesn't seem to be adressed?

LWLockAcquireOrWait()
{
..
LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "succeeded");
..
}

a. such a log is not there in any other LWLock.. variants,
if we want to introduce it, then shouldn't it be done at
other places as well.

I think you're placing unneccessarily high consistency constraints on a
debugging feature here.

Below point is yet to be resolved.

12.
#ifdef LWLOCK_DEBUG
lock->owner = MyProc;
#endif

Shouldn't it be reset in LWLockRelease?

That's actually intentional. It's quite useful to know the last owner
when debugging lwlock code.

Won't it cause any problem if the last owner process exits?

No. PGPROCs aren't deallocated or anything. And it's a debugging only
variable.

Can you explain how pg_read_barrier() in below code makes this
access safe?

LWLockWakeup()
{
..
+ pg_read_barrier(); /* pairs with nwaiters-- */
+ if (!BOOL_ACCESS_ONCE(lock->releaseOK))
..
}

What's the concern you have? Full memory barriers (the atomic
nwaiters--) pair with read memory barriers.

Greetings,

Andres Freund

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#102Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#99)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-21 19:56:05 +0530, Amit Kapila wrote:

On Wed, Oct 8, 2014 at 6:17 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

2.
LWLockWakeup()
{
..
#ifdef LWLOCK_STATS
lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
#else
SpinLockAcquire(&lock->mutex);
#endif
..
}

Earlier while releasing lock, we don't count it towards LWLock stats
spin_delay_count. I think if we see other places in lwlock.c, it only

gets

counted when we try to acquire it in a loop.

I think the previous situation was clearly suboptimal. I've now modified
things so all spinlock acquirations are counted.

Code has mainly 4 stats (sh_acquire_count, ex_acquire_count,
block_count, spin_delay_count) to track, if I try to see
all stats together to understand the contention situation,
the unpatched code makes sense.

I don't think it does. It completely disregards that the contention may
actually be in LWLockRelease(). That contributes to to spinlock
contention just as much as LWLockAcquire().

spin_delay_count gives
how much delay has happened to acquire spinlock which when
combined with other stats gives the clear situation about
the contention around aquisation of corresponding LWLock.
Now if we want to count the spin lock delay for Release call
as well, then the meaning of the stat is getting changed.
It might be that new meaning of spin_delay_count stat is more
useful in some situations, however the older one has its own
benefits, so I am not sure if changing this as part of this
patch is the best thing to do.

In which case does the old definition make sense, where the new one
doesn't? I don't think it exists.

And changing it here seems to make sense because spinlock contention
fundamentally changes it meaning for lwlocks anyway as in most paths we
don't take a spinlock anymore.

5.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &wakeup)
{
PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
LOG_LWDEBUG("LWLockRelease", l, mode, "release waiter");
dlist_delete(&waiter->lwWaitLink);
pg_write_barrier();
waiter->lwWaiting = false;
PGSemaphoreUnlock(&waiter->sem);
}
..
}

Why can't we decrement the nwaiters after waking up? I don't think
there is any major problem even if callers do that themselves, but
in some rare cases LWLockRelease() might spuriously assume that
there are some waiters and tries to call LWLockWakeup(). Although
this doesn't create any problem, keeping the value sane is good unless
there is some problem in doing so.

That won't work because then LWLockWakeup() wouldn't be called when
necessary - precisely because nwaiters is 0.

The reason I've done so is that it's otherwise much harder to debug
issues where there are backend that have been woken up already, but
haven't rerun yet. Without this there's simply no evidence of that
state. I can't see this being relevant for performance, so I'd rather
have it stay that way.

I am not sure what useful information we can get during debugging by not
doing this in LWLockWakeup()

It's useful because you can detect backends that have been scheduled to
acquire the lock, but haven't yet. They're otherwise "invisible".

and w.r.t performance it can lead extra
function call, few checks and I think in some cases even can
acquire/release spinlock.

I fail to see how that could be the case. And again, this is code that's
only executed around a couple syscalls. And the cacheline will be
touched around there *anyway*.

6.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
continue;
..
if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
{
..
wokeup_somebody = true;
}
..
}
..
}

a.
IIUC above logic, if the waiter queue is as follows:
(S-Shared; X-Exclusive) S X S S S X S S

it can skip the exclusive waiters and release shared waiter.

If my understanding is right, then I think instead of continue, there
should be *break* in above logic.

No, it looks correct to me. What happened is that the first S was woken
up. So there's no point in waking up an exclusive locker, but further
non-exclusive lockers can be woken up.

Okay, even then it makes the current logic of wakingup
different which I am not sure is what this patch is intended
for.

It's already done in a separate patch...

b.
Consider below sequence of waiters:
(S-Shared; X-Exclusive) S S X S S

I think as per un-patched code, it will wakeup waiters uptill

(including)

first Exclusive, but patch will wake up uptill (*excluding*) first
Exclusive.

I don't think the current code does that.

LWLockRelease()
{
..
/*
* If the front waiter wants exclusive lock, awaken him only.
*
Otherwise awaken as many waiters as want shared access.
*/
if (proc-

lwWaitMode != LW_EXCLUSIVE)

{
while (proc->lwWaitLink !=
NULL &&
proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
{
if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
releaseOK = false;
proc = proc->lwWaitLink;
}
}
/* proc is now the last PGPROC to be
released */
lock->head = proc->lwWaitLink;
proc->lwWaitLink = NULL;
..
}

In the above code, if the first waiter to be woken up is Exclusive waiter,
then it will woke that waiter, else shared waiters till it got
the first exclusive waiter and then first exlusive waiter.

That's would be bug then. Per the comment you quoted "If the front
waiter wants exclusive lock, awaken him only. Otherwise awaken as many
waiters as want shared access.".

But I don't think it's what's happening. Note that 'proc =
proc->lwWaitLink;' is only executed if 'proc->lwWaitLink->lwWaitMode !=
LW_EXCLUSIVE'. Which is the next waiter...

And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time it'd
make sense for X to be woken up is when it gets run faster than the S
processes.

Do we get any major benefit by changing the logic of waking up waiters?

Yes.

7.
LWLockWakeup()
{
..
dlist_foreach_modify(iter, (dlist_head *) &l->waiters)
{
..
dlist_delete(&waiter->lwWaitLink);
dlist_push_tail(&wakeup, &waiter->lwWaitLink);
..
}
..
}

Use of dlist has simplified the code, but I think there might be a

slight

overhead of maintaining wakeup queue as compare to un-patched
mechanism especially when there is a long waiter queue.

I don't see that as being relevant. The difference is an instruction or
two - in the slow path we'll enter the kernel and sleep. This doesn't
matter in comparison.

Okay, however I see Robert has also raised a point on this issue
which I am not sure is concluded.

And the code is *so* much more readable.

Code is more readable, but I don't understand why you
want to do refactoring as part of this patch which ideally
doesn't get any benefit from the same.

I did it first without. But there's required stuff like
LWLockDequeueSelf(). And I had several bugs because of the list stuff.

And I did separate the conversion into a separate patch?

8.
LWLockConditionalAcquire()
{
..
/*
* We ran into an exclusive lock and might have blocked another
* exclusive lock from taking a shot because it took a time to back
* off. Retry till we are either sure we didn't block somebody (because
* somebody else certainly has the lock) or till we got it.
*
* We cannot rely on the two-step lock-acquisition protocol as in
* LWLockAcquire because we're not using it.
*/
if (potentially_spurious)
{
SPIN_DELAY();
goto retry;
}
..
}

Due to above logic, I think it can keep on retrying for long time before
it actually concludes whether it got lock or not incase other backend/'s
takes Exclusive lock after *double_check* and release before
unconditional increment of shared lock in function LWLockAttemptLock.
I understand that it might be difficult to have such a practical

scenario,

however still there is a theoratical possibility of same.

I'm not particularly concerned. We could optimize it a bit, but I really
don't think it's necessary.

No issues, I was slightly worried about cases where this
API wasn't suppose to take time earlier (like for contentlock in
BufferAlloc), but now it starts taking time.

The API previously acquired a spinlock. That also took time...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#103Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#101)
Re: Wait free LW_SHARED acquisition - v0.2

On Wed, Oct 22, 2014 at 7:12 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-22 13:32:07 +0530, Amit Kapila wrote:

On Tue, Oct 21, 2014 at 7:56 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:

On Wed, Oct 8, 2014 at 6:17 PM, Andres Freund <andres@2ndquadrant.com>

wrote:

On 2014-06-25 19:06:32 +0530, Amit Kapila wrote:

Today, I have verified all previous comments raised by
me and looked at new code and below are my findings:

4.
LWLockAcquireCommon()
{
..
if (!LWLockDequeueSelf(l))
{
for (;;)
{
PGSemaphoreLock(&proc->sem, false);
if (!proc->lwWaiting)
break;
extraWaits++;
}
lock->releaseOK = true;
..
}

Setting releaseOK in above context might not be required because if

the

control comes in this part of code, it will not retry to acquire

another

time.

Hm. You're probably right.

You have agreed to fix this comment, but it seems you have forgot
to change it.

After I've thought more about it, it's is actually required. This
essentially *is* a retry.

Won't it needs to be set before retry? Whats the use of setting it
when we have got the lock and we are not going to retry.

Someobdy woke us up, which is where releaseOK is supposed to be set.

I think that is true only in case when we are again going to retry or
atleast that seems to be the mechanism used currently in
LWLockAcquireCommon.

11.
LWLockRelease()
{
..
PRINT_LWDEBUG("LWLockRelease", lock, mode);
}

Shouldn't this be in begining of LWLockRelease function rather than
after processing held_lwlocks array?

Ok.

You have agreed to fix this comment, but it seems you have forgot
to change it.

Below comment doesn't seem to be adressed?

LWLockAcquireOrWait()
{
..
LOG_LWDEBUG("LWLockAcquireOrWait", lockid, mode, "succeeded");
..
}

a. such a log is not there in any other LWLock.. variants,
if we want to introduce it, then shouldn't it be done at
other places as well.

I think you're placing unneccessarily high consistency constraints on a
debugging feature here.

This was just a very minor suggestion to keep code consistent,
which if you want to ignore is okay. I understand that having
or not having code consistent for this doesn't matter.

Below point is yet to be resolved.

12.
#ifdef LWLOCK_DEBUG
lock->owner = MyProc;
#endif

Shouldn't it be reset in LWLockRelease?

That's actually intentional. It's quite useful to know the last owner
when debugging lwlock code.

Won't it cause any problem if the last owner process exits?

No. PGPROCs aren't deallocated or anything. And it's a debugging only
variable.

Thats right, the problem I was thinking is of wrong information.
Ex. if process holding Exclusive locker has exited and then
lot of other processes took shared locks and one new Exclusive
locker waits on getting the lock, at that moment during debugging
we can get wrong information about lock owner.

However I think you are mainly worried about situtions when many
backends are waiting for Exclusive locker which is probably the
most common scenario.

Can you explain how pg_read_barrier() in below code makes this
access safe?

LWLockWakeup()
{
..
+ pg_read_barrier(); /* pairs with nwaiters-- */
+ if (!BOOL_ACCESS_ONCE(lock->releaseOK))
..
}

What's the concern you have? Full memory barriers (the atomic
nwaiters--) pair with read memory barriers.

IIUC, then pairing with nwaiters in LWLockAcquireCommon() ensures
that releaseOK is set before again attemting for lock as atomic
operation provides the necessary barrier. The point I am not
getting is what kind of guarantee pg_read_barrier() provides us
or why is it required?

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#104Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#102)
Re: Wait free LW_SHARED acquisition - v0.2

On Wed, Oct 22, 2014 at 8:04 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-21 19:56:05 +0530, Amit Kapila wrote:

On Wed, Oct 8, 2014 at 6:17 PM, Andres Freund <andres@2ndquadrant.com>
wrote:
spin_delay_count gives
how much delay has happened to acquire spinlock which when
combined with other stats gives the clear situation about
the contention around aquisation of corresponding LWLock.
Now if we want to count the spin lock delay for Release call
as well, then the meaning of the stat is getting changed.
It might be that new meaning of spin_delay_count stat is more
useful in some situations, however the older one has its own
benefits, so I am not sure if changing this as part of this
patch is the best thing to do.

In which case does the old definition make sense, where the new one
doesn't? I don't think it exists.

And changing it here seems to make sense because spinlock contention
fundamentally changes it meaning for lwlocks anyway as in most paths we
don't take a spinlock anymore.

On second thought, I think probably you are right here.

Why can't we decrement the nwaiters after waking up? I don't think
there is any major problem even if callers do that themselves, but
in some rare cases LWLockRelease() might spuriously assume that
there are some waiters and tries to call LWLockWakeup(). Although
this doesn't create any problem, keeping the value sane is good

unless

there is some problem in doing so.

That won't work because then LWLockWakeup() wouldn't be called when
necessary - precisely because nwaiters is 0.

The reason I've done so is that it's otherwise much harder to debug
issues where there are backend that have been woken up already, but
haven't rerun yet. Without this there's simply no evidence of that
state. I can't see this being relevant for performance, so I'd rather
have it stay that way.

I am not sure what useful information we can get during debugging by not
doing this in LWLockWakeup()

It's useful because you can detect backends that have been scheduled to
acquire the lock, but haven't yet. They're otherwise "invisible".

and w.r.t performance it can lead extra
function call, few checks and I think in some cases even can
acquire/release spinlock.

I fail to see how that could be the case.

Won't it happen incase first backend sets releaseOK to true and another
backend which tries to wakeup waiters on lock will acquire spinlock
and tries to release the waiters.

And again, this is code that's
only executed around a couple syscalls. And the cacheline will be
touched around there *anyway*.

Sure, but I think syscalls are required in case we need to wake any
waiter.

In the above code, if the first waiter to be woken up is Exclusive

waiter,

then it will woke that waiter, else shared waiters till it got
the first exclusive waiter and then first exlusive waiter.

That's would be bug then.

I am not sure of it, but I think it's more important to validate the
new waking startegy as you see benefits by doing so.

Per the comment you quoted "If the front
waiter wants exclusive lock, awaken him only. Otherwise awaken as many
waiters as want shared access.".

But I don't think it's what's happening. Note that 'proc =
proc->lwWaitLink;' is only executed if 'proc->lwWaitLink->lwWaitMode !=
LW_EXCLUSIVE'. Which is the next waiter...

And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time it'd
make sense for X to be woken up is when it gets run faster than the S
processes.

Do we get any major benefit by changing the logic of waking up waiters?

Yes.

I think one downside I could see of new strategy is that the chance of
Exclusive waiter to take more time before getting woked up is increased
as now it will by pass Exclusive waiters in queue. I don't have any
concrete proof that it can do any harm to performance, so may be it's
okay to have this new mechanism, however I think it might be helpful
if you could add a comment in code to explain the benefit by skipping
Exclusive lockers.

Code is more readable, but I don't understand why you
want to do refactoring as part of this patch which ideally
doesn't get any benefit from the same.

I did it first without. But there's required stuff like
LWLockDequeueSelf(). And I had several bugs because of the list stuff.

And I did separate the conversion into a separate patch?

Yeah, but the main patch for wait free LW_SHARED also uses
it.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#105Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#104)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-10-24 15:59:30 +0530, Amit Kapila wrote:

and w.r.t performance it can lead extra
function call, few checks and I think in some cases even can
acquire/release spinlock.

I fail to see how that could be the case.

Won't it happen incase first backend sets releaseOK to true and another
backend which tries to wakeup waiters on lock will acquire spinlock
and tries to release the waiters.

Sure, that can happen.

And again, this is code that's
only executed around a couple syscalls. And the cacheline will be
touched around there *anyway*.

Sure, but I think syscalls are required in case we need to wake any
waiter.

It won't wake up a waiter if there's none on the list.

And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time it'd
make sense for X to be woken up is when it gets run faster than the S
processes.

Do we get any major benefit by changing the logic of waking up waiters?

Yes.

I think one downside I could see of new strategy is that the chance of
Exclusive waiter to take more time before getting woked up is increased
as now it will by pass Exclusive waiters in queue.

Note that that *already* happens for any *new* shared locker that comes
in. It doesn't really make sense to have share lockers queued behind the
exclusive locker if others just go in front of it anyway.

Code is more readable, but I don't understand why you
want to do refactoring as part of this patch which ideally
doesn't get any benefit from the same.

I did it first without. But there's required stuff like
LWLockDequeueSelf(). And I had several bugs because of the list stuff.

And I did separate the conversion into a separate patch?

Yeah, but the main patch for wait free LW_SHARED also uses
it.

Well, the only thing that it could have done given that the other patch
is a preqrequisite is reverting the behaviour?

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#106Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#105)
Re: Wait free LW_SHARED acquisition - v0.2

On Fri, Oct 24, 2014 at 4:05 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-24 15:59:30 +0530, Amit Kapila wrote:

and w.r.t performance it can lead extra
function call, few checks and I think in some cases even can
acquire/release spinlock.

I fail to see how that could be the case.

Won't it happen incase first backend sets releaseOK to true and another
backend which tries to wakeup waiters on lock will acquire spinlock
and tries to release the waiters.

Sure, that can happen.

And again, this is code that's
only executed around a couple syscalls. And the cacheline will be
touched around there *anyway*.

Sure, but I think syscalls are required in case we need to wake any
waiter.

It won't wake up a waiter if there's none on the list.

Yeap, but still it will acquire/release spinlock.

And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time

it'd

make sense for X to be woken up is when it gets run faster than

the S

processes.

Do we get any major benefit by changing the logic of waking up

waiters?

Yes.

I think one downside I could see of new strategy is that the chance of
Exclusive waiter to take more time before getting woked up is increased
as now it will by pass Exclusive waiters in queue.

Note that that *already* happens for any *new* shared locker that comes
in. It doesn't really make sense to have share lockers queued behind the
exclusive locker if others just go in front of it anyway.

Yeah, but I think it is difficult to avoid that behaviour as even when it
wakes
Exclusive locker, some new shared locker can comes in and acquire the
lock before Exclusive locker.

I think it is difficult to say what is the best waking strategy, as
priority for
Exclusive lockers is not clearly defined incase of LWLocks.

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#107Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#98)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-21 12:40:56 +0530, Amit Kapila wrote:

While doing performance tests, I noticed a hang at higher client
counts with patch. I have tried to check call stack for few of
processes and it is as below:

#0 0x0000008010933e54 in .semop () from /lib64/libc.so.6
#1 0x0000000010286e48 in .PGSemaphoreLock ()
#2 0x00000000102f68bc in .LWLockAcquire ()
#3 0x00000000102d1ca0 in .ReadBuffer_common ()
#4 0x00000000102d2ae0 in .ReadBufferExtended ()
#5 0x00000000100a57d8 in ._bt_getbuf ()
#6 0x00000000100a6210 in ._bt_getroot ()
#7 0x00000000100aa910 in ._bt_search ()
#8 0x00000000100ab494 in ._bt_first ()
#9 0x00000000100a8e84 in .btgettuple ()
..

#0 0x0000008010933e54 in .semop () from /lib64/libc.so.6
#1 0x0000000010286e48 in .PGSemaphoreLock ()
#2 0x00000000102f68bc in .LWLockAcquire ()
#3 0x00000000102d1ca0 in .ReadBuffer_common ()
#4 0x00000000102d2ae0 in .ReadBufferExtended ()
#5 0x00000000100a57d8 in ._bt_getbuf ()
#6 0x00000000100a6210 in ._bt_getroot ()
#7 0x00000000100aa910 in ._bt_search ()
#8 0x00000000100ab494 in ._bt_first ()
...

The test configuration is as below:
Test env - Power - 7 (hydra)
scale_factor - 3000
shared_buffers - 8GB
test mode - pgbench read only

test execution -
./pgbench -c 128 -j 128 -T 1800 -S -M prepared postgres

I have ran it for half an hour, but it doesn't came out even after
~2 hours. It doesn't get reproduced every time, currently I am
able to reproduce it and the m/c is in same state, if you want any
info, let me know (unfortunately binaries are in release mode, so
might not get enough information).

Hm. What commit did you apply the series ontop? I managed to reproduce a
hang, but it was just something that heikki had already fixed...

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#108Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#107)
Re: Wait free LW_SHARED acquisition - v0.9

On Thu, Oct 30, 2014 at 5:52 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-21 12:40:56 +0530, Amit Kapila wrote:

While doing performance tests, I noticed a hang at higher client
counts with patch. I have tried to check call stack for few of
processes and it is as below:

#0 0x0000008010933e54 in .semop () from /lib64/libc.so.6
#1 0x0000000010286e48 in .PGSemaphoreLock ()
#2 0x00000000102f68bc in .LWLockAcquire ()
#3 0x00000000102d1ca0 in .ReadBuffer_common ()
#4 0x00000000102d2ae0 in .ReadBufferExtended ()
#5 0x00000000100a57d8 in ._bt_getbuf ()
#6 0x00000000100a6210 in ._bt_getroot ()
#7 0x00000000100aa910 in ._bt_search ()
#8 0x00000000100ab494 in ._bt_first ()
#9 0x00000000100a8e84 in .btgettuple ()
..

#0 0x0000008010933e54 in .semop () from /lib64/libc.so.6
#1 0x0000000010286e48 in .PGSemaphoreLock ()
#2 0x00000000102f68bc in .LWLockAcquire ()
#3 0x00000000102d1ca0 in .ReadBuffer_common ()
#4 0x00000000102d2ae0 in .ReadBufferExtended ()
#5 0x00000000100a57d8 in ._bt_getbuf ()
#6 0x00000000100a6210 in ._bt_getroot ()
#7 0x00000000100aa910 in ._bt_search ()
#8 0x00000000100ab494 in ._bt_first ()
...

The test configuration is as below:
Test env - Power - 7 (hydra)
scale_factor - 3000
shared_buffers - 8GB
test mode - pgbench read only

test execution -
./pgbench -c 128 -j 128 -T 1800 -S -M prepared postgres

I have ran it for half an hour, but it doesn't came out even after
~2 hours. It doesn't get reproduced every time, currently I am
able to reproduce it and the m/c is in same state, if you want any
info, let me know (unfortunately binaries are in release mode, so
might not get enough information).

Hm. What commit did you apply the series ontop? I managed to reproduce a
hang, but it was just something that heikki had already fixed...

commit 494affbd900d1c90de17414a575af1a085c3e37a
Author: Noah Misch <noah@leadboat.com>
Date: Sun Oct 12 23:33:37 2014 -0400

And, I think you are saying that heikki's commit e0d97d has fixed
this issue, in that case I will check once by including that fix?

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#109Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#108)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-30 18:54:57 +0530, Amit Kapila wrote:

On Thu, Oct 30, 2014 at 5:52 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-21 12:40:56 +0530, Amit Kapila wrote:

I have ran it for half an hour, but it doesn't came out even after
~2 hours. It doesn't get reproduced every time, currently I am
able to reproduce it and the m/c is in same state, if you want any
info, let me know (unfortunately binaries are in release mode, so
might not get enough information).

Hm. What commit did you apply the series ontop? I managed to reproduce a
hang, but it was just something that heikki had already fixed...

commit 494affbd900d1c90de17414a575af1a085c3e37a
Author: Noah Misch <noah@leadboat.com>
Date: Sun Oct 12 23:33:37 2014 -0400

And, I think you are saying that heikki's commit e0d97d has fixed
this issue, in that case I will check once by including that fix?

Well, the hang I was able to reproduce was originally hanging because of
that. I saw lot of content locks waiting as well, but the "origin" seems
to have a backend waiting for a xloginsert.

The way I could trigger it quite fast was by first running a read/write
pgbench and then switch to a readonly one.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#110Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#109)
Re: Wait free LW_SHARED acquisition - v0.9

On Thu, Oct 30, 2014 at 6:58 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-30 18:54:57 +0530, Amit Kapila wrote:

On Thu, Oct 30, 2014 at 5:52 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

Hm. What commit did you apply the series ontop? I managed to

reproduce a

hang, but it was just something that heikki had already fixed...

commit 494affbd900d1c90de17414a575af1a085c3e37a
Author: Noah Misch <noah@leadboat.com>
Date: Sun Oct 12 23:33:37 2014 -0400

And, I think you are saying that heikki's commit e0d97d has fixed
this issue, in that case I will check once by including that fix?

Well, the hang I was able to reproduce was originally hanging because of
that. I saw lot of content locks waiting as well, but the "origin" seems
to have a backend waiting for a xloginsert.

The way I could trigger it quite fast was by first running a read/write
pgbench and then switch to a readonly one.

So what exactly you mean by 'switch to'?
Is it that both read-write and readonly pgbench were running together
or after read-write got finished and then by running read-only pgbench,
you are able to reproduce it?

With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

#111Andres Freund
andres@2ndquadrant.com
In reply to: Amit Kapila (#110)
Re: Wait free LW_SHARED acquisition - v0.9

On 2014-10-30 19:05:06 +0530, Amit Kapila wrote:

On Thu, Oct 30, 2014 at 6:58 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-30 18:54:57 +0530, Amit Kapila wrote:

On Thu, Oct 30, 2014 at 5:52 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

Hm. What commit did you apply the series ontop? I managed to

reproduce a

hang, but it was just something that heikki had already fixed...

commit 494affbd900d1c90de17414a575af1a085c3e37a
Author: Noah Misch <noah@leadboat.com>
Date: Sun Oct 12 23:33:37 2014 -0400

And, I think you are saying that heikki's commit e0d97d has fixed
this issue, in that case I will check once by including that fix?

Well, the hang I was able to reproduce was originally hanging because of
that. I saw lot of content locks waiting as well, but the "origin" seems
to have a backend waiting for a xloginsert.

The way I could trigger it quite fast was by first running a read/write
pgbench and then switch to a readonly one.

So what exactly you mean by 'switch to'?
Is it that both read-write and readonly pgbench were running together
or after read-write got finished and then by running read-only pgbench,
you are able to reproduce it?

I don't think that matters all that much. In this case I first had a
read-write one (accidentally, by leaving of -S), and then aborted and
ran a readonly pgbench. That turned out to trigger it relatively fast.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#112Robert Haas
robertmhaas@gmail.com
In reply to: Amit Kapila (#106)
Re: Wait free LW_SHARED acquisition - v0.2

On Sat, Oct 25, 2014 at 1:50 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Oct 24, 2014 at 4:05 PM, Andres Freund <andres@2ndquadrant.com>
wrote:

On 2014-10-24 15:59:30 +0530, Amit Kapila wrote:

and w.r.t performance it can lead extra
function call, few checks and I think in some cases even can
acquire/release spinlock.

I fail to see how that could be the case.

Won't it happen incase first backend sets releaseOK to true and another
backend which tries to wakeup waiters on lock will acquire spinlock
and tries to release the waiters.

Sure, that can happen.

And again, this is code that's
only executed around a couple syscalls. And the cacheline will be
touched around there *anyway*.

Sure, but I think syscalls are required in case we need to wake any
waiter.

It won't wake up a waiter if there's none on the list.

Yeap, but still it will acquire/release spinlock.

And it'd be a pretty pointless
behaviour, leading to useless increased contention. The only time
it'd
make sense for X to be woken up is when it gets run faster than
the S
processes.

Do we get any major benefit by changing the logic of waking up
waiters?

Yes.

I think one downside I could see of new strategy is that the chance of
Exclusive waiter to take more time before getting woked up is increased
as now it will by pass Exclusive waiters in queue.

Note that that *already* happens for any *new* shared locker that comes
in. It doesn't really make sense to have share lockers queued behind the
exclusive locker if others just go in front of it anyway.

Yeah, but I think it is difficult to avoid that behaviour as even when it
wakes
Exclusive locker, some new shared locker can comes in and acquire the
lock before Exclusive locker.

I think it is difficult to say what is the best waking strategy, as priority
for
Exclusive lockers is not clearly defined incase of LWLocks.

Andres, where are we with this patch?

1. You're going to commit it, but haven't gotten around to it yet.

2. You're going to modify it some more and repost, but haven't gotten
around to it yet.

3. You're willing to see it modified if somebody else does the work,
but are out of time to spend on it yourself.

4. Something else?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#113Andres Freund
andres@2ndquadrant.com
In reply to: Robert Haas (#112)
Re: Wait free LW_SHARED acquisition - v0.2

On 2014-11-17 10:21:04 -0500, Robert Haas wrote:

Andres, where are we with this patch?

1. You're going to commit it, but haven't gotten around to it yet.

2. You're going to modify it some more and repost, but haven't gotten
around to it yet.

3. You're willing to see it modified if somebody else does the work,
but are out of time to spend on it yourself.

4. Something else?

I'm working on it. Amit had found a hang on PPC that I couldn't
reproduce on x86. Since then I've reproduced it and I think yesterday I
found the problem. Unfortunately it always took a couple hours to
trigger...

I've also made some, in my opinion, cleanups to the patch since
then. Those have the nice side effect of making the size of struct
LWLock smaller, but that wasn't actually the indended effect.

I'll repost once I've verified the problem is fixed and I've updated all
commentary.

The current problem is that I seem to have found a problem that's also
reproducible with master :(. After a couple of hours a
pgbench -h /tmp -p 5440 scale3000 -M prepared -P 5 -c 180 -j 60 -T 20000 -S
against a
-c max_connections=200 -c shared_buffers=4GB
cluster seems to hang on PPC. With all the backends waiting in buffer
mapping locks. I'm now making sure it's really master and not my patch
causing the problem - it's just not trivial with 180 processes involved.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#114Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#113)
Re: Wait free LW_SHARED acquisition - v0.2

On Mon, Nov 17, 2014 at 10:31 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-11-17 10:21:04 -0500, Robert Haas wrote:

Andres, where are we with this patch?

1. You're going to commit it, but haven't gotten around to it yet.

2. You're going to modify it some more and repost, but haven't gotten
around to it yet.

3. You're willing to see it modified if somebody else does the work,
but are out of time to spend on it yourself.

4. Something else?

I'm working on it. Amit had found a hang on PPC that I couldn't
reproduce on x86. Since then I've reproduced it and I think yesterday I
found the problem. Unfortunately it always took a couple hours to
trigger...

I've also made some, in my opinion, cleanups to the patch since
then. Those have the nice side effect of making the size of struct
LWLock smaller, but that wasn't actually the indended effect.

I'll repost once I've verified the problem is fixed and I've updated all
commentary.

The current problem is that I seem to have found a problem that's also
reproducible with master :(. After a couple of hours a
pgbench -h /tmp -p 5440 scale3000 -M prepared -P 5 -c 180 -j 60 -T 20000 -S
against a
-c max_connections=200 -c shared_buffers=4GB
cluster seems to hang on PPC. With all the backends waiting in buffer
mapping locks. I'm now making sure it's really master and not my patch
causing the problem - it's just not trivial with 180 processes involved.

Ah, OK. Thanks for the update.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#115Michael Paquier
michael.paquier@gmail.com
In reply to: Robert Haas (#114)
Re: Wait free LW_SHARED acquisition - v0.2

On Tue, Nov 18, 2014 at 12:33 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Nov 17, 2014 at 10:31 AM, Andres Freund <andres@2ndquadrant.com> wrote:

On 2014-11-17 10:21:04 -0500, Robert Haas wrote:

Andres, where are we with this patch?

1. You're going to commit it, but haven't gotten around to it yet.

2. You're going to modify it some more and repost, but haven't gotten
around to it yet.

3. You're willing to see it modified if somebody else does the work,
but are out of time to spend on it yourself.

4. Something else?

I'm working on it. Amit had found a hang on PPC that I couldn't
reproduce on x86. Since then I've reproduced it and I think yesterday I
found the problem. Unfortunately it always took a couple hours to
trigger...

I've also made some, in my opinion, cleanups to the patch since
then. Those have the nice side effect of making the size of struct
LWLock smaller, but that wasn't actually the indended effect.

I'll repost once I've verified the problem is fixed and I've updated all
commentary.

The current problem is that I seem to have found a problem that's also
reproducible with master :(. After a couple of hours a
pgbench -h /tmp -p 5440 scale3000 -M prepared -P 5 -c 180 -j 60 -T 20000 -S
against a
-c max_connections=200 -c shared_buffers=4GB
cluster seems to hang on PPC. With all the backends waiting in buffer
mapping locks. I'm now making sure it's really master and not my patch
causing the problem - it's just not trivial with 180 processes involved.

Ah, OK. Thanks for the update.

Ping? This patch is in a stale state for a couple of weeks and still
marked as waiting on author for this CF.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#116Michael Paquier
michael.paquier@gmail.com
In reply to: Michael Paquier (#115)
Re: Wait free LW_SHARED acquisition - v0.2

On Wed, Dec 3, 2014 at 4:03 PM, Michael Paquier
<michael.paquier@gmail.com> wrote:

Ping? This patch is in a stale state for a couple of weeks and still
marked as waiting on author for this CF.

Marked as returned with feedback.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#117Andres Freund
andres@2ndquadrant.com
In reply to: Andres Freund (#1)
3 attachment(s)
Re: Wait free LW_SHARED acquisition - v0.10

Hi,

Attached is a new version of the patchset which I intend to commit soon.

Stuff changed since 0.9:

* Greatly simplified locking logic - the whole concept that a lock could
be spuriously acquired is gone. That cost a small bit of performance
(0.5%, I thought it'd be much bigger) on x86, but is a noticeable
performance *benefit* on PPC.

* releaseOK (and other internal flags) are rolled into the former
'lockcount' variable which is now named state. By having it inside the
same atomic reasoning about the state gets easier as there's no skew
between observing the lockcount and other variables.

* The number of queued waiters isn't required anymore, it's only a
debugging aid (#ifdef LOCK_DEBUG) at this point.

Patches:
0001: errhidecontext() patch
0002: dlist()ify lwWaitLink
0003: LW_SHARED scalability

I've done a fair amount of benchmarking and on bigger system the new
code seems to be a win pretty much generally.

Greetings,

Andres Freund

--
Andres Freund http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-Add-capability-to-suppress-CONTEXT-messages-to-elog-.patchtext/x-patch; charset=us-asciiDownload
>From 5e6af912377a1b43ea0168951238cb6a5e0b233e Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Sun, 21 Dec 2014 15:45:55 +0100
Subject: [PATCH 1/4] Add capability to suppress CONTEXT: messages to elog
 machinery.

Hiding context messages usually is not a good idea - except for rather
verbose debugging/development utensils like LOG_DEBUG. There the
amount of repeated context messages just bloat the log without adding
information.
---
 src/backend/utils/error/elog.c | 24 ++++++++++++++++++++++--
 src/include/utils/elog.h       |  2 ++
 2 files changed, 24 insertions(+), 2 deletions(-)

diff --git a/src/backend/utils/error/elog.c b/src/backend/utils/error/elog.c
index 2316464..8f04b19 100644
--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -1081,6 +1081,25 @@ errhidestmt(bool hide_stmt)
 	return 0;					/* return value does not matter */
 }
 
+/*
+ * errhidestmt --- optionally suppress CONTEXT: field of log entry
+ *
+ * This should only be used for verbose debugging messages where the repeated
+ * inclusion of CONTEXT: bloats the log volume too much.
+ */
+int
+errhidecontext(bool hide_ctx)
+{
+	ErrorData  *edata = &errordata[errordata_stack_depth];
+
+	/* we don't bother incrementing recursion_depth */
+	CHECK_STACK_DEPTH();
+
+	edata->hide_ctx = hide_ctx;
+
+	return 0;					/* return value does not matter */
+}
+
 
 /*
  * errfunction --- add reporting function name to the current error
@@ -2724,7 +2743,8 @@ write_csvlog(ErrorData *edata)
 	appendStringInfoChar(&buf, ',');
 
 	/* errcontext */
-	appendCSVLiteral(&buf, edata->context);
+	if (!edata->hide_ctx)
+		appendCSVLiteral(&buf, edata->context);
 	appendStringInfoChar(&buf, ',');
 
 	/* user query --- only reported if not disabled by the caller */
@@ -2856,7 +2876,7 @@ send_message_to_server_log(ErrorData *edata)
 			append_with_tabs(&buf, edata->internalquery);
 			appendStringInfoChar(&buf, '\n');
 		}
-		if (edata->context)
+		if (edata->context && !edata->hide_ctx)
 		{
 			log_line_prefix(&buf, edata);
 			appendStringInfoString(&buf, _("CONTEXT:  "));
diff --git a/src/include/utils/elog.h b/src/include/utils/elog.h
index 87438b8..ec7ed22 100644
--- a/src/include/utils/elog.h
+++ b/src/include/utils/elog.h
@@ -221,6 +221,7 @@ errcontext_msg(const char *fmt,...)
 __attribute__((format(PG_PRINTF_ATTRIBUTE, 1, 2)));
 
 extern int	errhidestmt(bool hide_stmt);
+extern int	errhidecontext(bool hide_ctx);
 
 extern int	errfunction(const char *funcname);
 extern int	errposition(int cursorpos);
@@ -385,6 +386,7 @@ typedef struct ErrorData
 	bool		output_to_client;		/* will report to client? */
 	bool		show_funcname;	/* true to force funcname inclusion */
 	bool		hide_stmt;		/* true to prevent STATEMENT: inclusion */
+	bool		hide_ctx;		/* true to prevent CONTEXT: inclusion */
 	const char *filename;		/* __FILE__ of ereport() call */
 	int			lineno;			/* __LINE__ of ereport() call */
 	const char *funcname;		/* __func__ of ereport() call */
-- 
2.2.0.rc0.18.ga1ad247

0002-Convert-the-PGPROC-lwWaitLink-list-into-a-dlist-inst.patchtext/x-patch; charset=us-asciiDownload
>From 0ef51c826faa53620fc7d8d39d5df6206be729f3 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Tue, 7 Oct 2014 15:32:34 +0200
Subject: [PATCH 2/4] Convert the PGPROC->lwWaitLink list into a dlist instead
 of open coding it.

Besides being shorter and much easier to read it changes the logic in
LWLockRelease() to release all shared lockers when waking up any. This
can yield some significant performance improvements - and the fairness
isn't really much worse than before, as we always allowed new shared
lockers to jump the queue.
---
 src/backend/access/transam/twophase.c |   1 -
 src/backend/storage/lmgr/lwlock.c     | 146 ++++++++++++----------------------
 src/backend/storage/lmgr/proc.c       |   2 -
 src/include/storage/lwlock.h          |   5 +-
 src/include/storage/proc.h            |   3 +-
 5 files changed, 57 insertions(+), 100 deletions(-)

diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 40de84e..ad3e872 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -390,7 +390,6 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 	proc->roleId = owner;
 	proc->lwWaiting = false;
 	proc->lwWaitMode = 0;
-	proc->lwWaitLink = NULL;
 	proc->waitLock = NULL;
 	proc->waitProcLock = NULL;
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 43f4d6b..d9d7044 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -117,9 +117,9 @@ inline static void
 PRINT_LWDEBUG(const char *where, const LWLock *lock)
 {
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): excl %d shared %d head %p rOK %d",
+		elog(LOG, "%s(%s %d): excl %d shared %d rOK %d",
 			 where, T_NAME(lock), T_ID(lock),
-			 (int) lock->exclusive, lock->shared, lock->head,
+			 (int) lock->exclusive, lock->shared,
 			 (int) lock->releaseOK);
 }
 
@@ -479,8 +479,7 @@ LWLockInitialize(LWLock *lock, int tranche_id)
 	lock->exclusive = 0;
 	lock->shared = 0;
 	lock->tranche = tranche_id;
-	lock->head = NULL;
-	lock->tail = NULL;
+	dlist_init(&lock->waiters);
 }
 
 
@@ -619,12 +618,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = mode;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -840,12 +834,7 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		proc->lwWaitLink = NULL;
-		if (lock->head == NULL)
-			lock->head = proc;
-		else
-			lock->tail->lwWaitLink = proc;
-		lock->tail = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/* Can release the mutex now */
 		SpinLockRelease(&lock->mutex);
@@ -1002,10 +991,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		proc->lwWaiting = true;
 		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
 		/* waiters are added to the front of the queue */
-		proc->lwWaitLink = lock->head;
-		if (lock->head == NULL)
-			lock->tail = proc;
-		lock->head = proc;
+		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
 
 		/*
 		 * Set releaseOK, to make sure we get woken up as soon as the lock is
@@ -1087,9 +1073,10 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 void
 LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 {
-	PGPROC	   *head;
-	PGPROC	   *proc;
-	PGPROC	   *next;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+
+	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
 	SpinLockAcquire(&lock->mutex);
@@ -1104,24 +1091,16 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	 * See if there are any LW_WAIT_UNTIL_FREE waiters that need to be woken
 	 * up. They are always in the front of the queue.
 	 */
-	head = lock->head;
-
-	if (head != NULL && head->lwWaitMode == LW_WAIT_UNTIL_FREE)
+	dlist_foreach_modify(iter, &lock->waiters)
 	{
-		proc = head;
-		next = proc->lwWaitLink;
-		while (next && next->lwWaitMode == LW_WAIT_UNTIL_FREE)
-		{
-			proc = next;
-			next = next->lwWaitLink;
-		}
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+			break;
 
-		/* proc is now the last PGPROC to be released */
-		lock->head = next;
-		proc->lwWaitLink = NULL;
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 	}
-	else
-		head = NULL;
 
 	/* We are done updating shared state of the lock itself. */
 	SpinLockRelease(&lock->mutex);
@@ -1129,15 +1108,14 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, &wakeup)
 	{
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		dlist_delete(&waiter->lwWaitLink);
 		/* check comment in LWLockRelease() about this barrier */
 		pg_write_barrier();
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 }
 
@@ -1148,10 +1126,12 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 void
 LWLockRelease(LWLock *lock)
 {
-	PGPROC	   *head;
-	PGPROC	   *proc;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
 	int			i;
 
+	dlist_init(&wakeup);
+
 	PRINT_LWDEBUG("LWLockRelease", lock);
 
 	/*
@@ -1187,58 +1167,39 @@ LWLockRelease(LWLock *lock)
 	 * if someone has already awakened waiters that haven't yet acquired the
 	 * lock.
 	 */
-	head = lock->head;
-	if (head != NULL)
+	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
 	{
-		if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
+		/*
+		 * Remove the to-be-awakened PGPROCs from the queue.
+		 */
+		bool		releaseOK = true;
+		bool		wokeup_somebody = false;
+
+		dlist_foreach_modify(iter, &lock->waiters)
 		{
-			/*
-			 * Remove the to-be-awakened PGPROCs from the queue.
-			 */
-			bool		releaseOK = true;
+			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 
-			proc = head;
+			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+				continue;
 
-			/*
-			 * First wake up any backends that want to be woken up without
-			 * acquiring the lock.
-			 */
-			while (proc->lwWaitMode == LW_WAIT_UNTIL_FREE && proc->lwWaitLink)
-				proc = proc->lwWaitLink;
+			dlist_delete(&waiter->lwWaitLink);
+			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
 
 			/*
-			 * If the front waiter wants exclusive lock, awaken him only.
-			 * Otherwise awaken as many waiters as want shared access.
+			 * Prevent additional wakeups until retryer gets to
+			 * run. Backends that are just waiting for the lock to become
+			 * free don't retry automatically.
 			 */
-			if (proc->lwWaitMode != LW_EXCLUSIVE)
+			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
 			{
-				while (proc->lwWaitLink != NULL &&
-					   proc->lwWaitLink->lwWaitMode != LW_EXCLUSIVE)
-				{
-					if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
-						releaseOK = false;
-					proc = proc->lwWaitLink;
-				}
-			}
-			/* proc is now the last PGPROC to be released */
-			lock->head = proc->lwWaitLink;
-			proc->lwWaitLink = NULL;
-
-			/*
-			 * Prevent additional wakeups until retryer gets to run. Backends
-			 * that are just waiting for the lock to become free don't retry
-			 * automatically.
-			 */
-			if (proc->lwWaitMode != LW_WAIT_UNTIL_FREE)
 				releaseOK = false;
+				wokeup_somebody = true;
+			}
 
-			lock->releaseOK = releaseOK;
-		}
-		else
-		{
-			/* lock is still held, can't awaken anything */
-			head = NULL;
+			if(waiter->lwWaitMode == LW_EXCLUSIVE)
+				break;
 		}
+		lock->releaseOK = releaseOK;
 	}
 
 	/* We are done updating shared state of the lock itself. */
@@ -1249,13 +1210,12 @@ LWLockRelease(LWLock *lock)
 	/*
 	 * Awaken any waiters I removed from the queue.
 	 */
-	while (head != NULL)
+	dlist_foreach_modify(iter, &wakeup)
 	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 		LOG_LWDEBUG("LWLockRelease", T_NAME(lock), T_ID(lock),
 					"release waiter");
-		proc = head;
-		head = proc->lwWaitLink;
-		proc->lwWaitLink = NULL;
+		dlist_delete(&waiter->lwWaitLink);
 		/*
 		 * Guarantee that lwWaiting being unset only becomes visible once the
 		 * unlink from the link has completed. Otherwise the target backend
@@ -1267,8 +1227,8 @@ LWLockRelease(LWLock *lock)
 		 * another lock.
 		 */
 		pg_write_barrier();
-		proc->lwWaiting = false;
-		PGSemaphoreUnlock(&proc->sem);
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
 	}
 
 	/*
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index ea88a24..a4789fc 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -372,7 +372,6 @@ InitProcess(void)
 		MyPgXact->vacuumFlags |= PROC_IS_AUTOVACUUM;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
@@ -535,7 +534,6 @@ InitAuxiliaryProcess(void)
 	MyPgXact->vacuumFlags = 0;
 	MyProc->lwWaiting = false;
 	MyProc->lwWaitMode = 0;
-	MyProc->lwWaitLink = NULL;
 	MyProc->waitLock = NULL;
 	MyProc->waitProcLock = NULL;
 #ifdef USE_ASSERT_CHECKING
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 09654a8..c84970a 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -14,6 +14,7 @@
 #ifndef LWLOCK_H
 #define LWLOCK_H
 
+#include "lib/ilist.h"
 #include "storage/s_lock.h"
 
 struct PGPROC;
@@ -50,9 +51,7 @@ typedef struct LWLock
 	char		exclusive;		/* # of exclusive holders (0 or 1) */
 	int			shared;			/* # of shared holders (0..MaxBackends) */
 	int			tranche;		/* tranche ID */
-	struct PGPROC *head;		/* head of list of waiting PGPROCs */
-	struct PGPROC *tail;		/* tail of list of waiting PGPROCs */
-	/* tail is undefined when head is NULL */
+	dlist_head	waiters;		/* list of waiting PGPROCs */
 } LWLock;
 
 /*
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 4ad4164..e868f0c 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -15,6 +15,7 @@
 #define _PROC_H_
 
 #include "access/xlogdefs.h"
+#include "lib/ilist.h"
 #include "storage/latch.h"
 #include "storage/lock.h"
 #include "storage/pg_sema.h"
@@ -104,7 +105,7 @@ struct PGPROC
 	/* Info about LWLock the process is currently waiting for, if any. */
 	bool		lwWaiting;		/* true if waiting for an LW lock */
 	uint8		lwWaitMode;		/* lwlock mode being waited for */
-	struct PGPROC *lwWaitLink;	/* next waiter for same LW lock */
+	dlist_node	lwWaitLink;		/* position in LW lock wait list */
 
 	/* Info about lock the process is currently waiting for, if any. */
 	/* waitLock and waitProcLock are NULL if not currently waiting. */
-- 
2.2.0.rc0.18.ga1ad247

0003-Wait-free-LW_SHARED-LWLock-acquisition.patchtext/x-patch; charset=us-asciiDownload
>From 5e78af23020185b1a870537e9f107d89cea36bae Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Thu, 18 Sep 2014 16:14:16 +0200
Subject: [PATCH 3/4] Wait free LW_SHARED LWLock acquisition.

The old LWLock implementation had the problem that concurrent shared
lock acquisitions required exclusively acquiring a spinlock. Often
that could lead to acquirers waiting behind the spinlock, even if the
actual LWLock was free.

The new implementation doesn't acquire the spinlock when acquiring the
lock itself. Instead the new atomic operations are used to atomically
manipulate the state. Only the waitqueue, used only in the slow path,
is still protected by the spinlock. Check lwlock.c's header for an
explanation about the used algorithm.

For some common workloads on larger machines this can yield
significant performance improvements. Particularly in read mostly
workloads.

Reviewed-By: Amit Kapila and Robert Haas
Author: Andres Freund

Discussion: 20130926225545.GB26663@awork2.anarazel.de
---
 src/backend/storage/lmgr/lwlock.c | 931 ++++++++++++++++++++++++++------------
 src/include/storage/lwlock.h      |  24 +-
 2 files changed, 653 insertions(+), 302 deletions(-)

diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index d9d7044..f0db0a6 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -24,7 +24,53 @@
  * IDENTIFICATION
  *	  src/backend/storage/lmgr/lwlock.c
  *
- *-------------------------------------------------------------------------
+ * NOTES:
+ *
+ * This used to be a pretty straight forward reader-writer lock
+ * implementation, in which the internal state was protected by a
+ * spinlock. Unfortunately the overhead of taking the spinlock proved to be
+ * too high for workloads/locks that were taken in shared mode very
+ * frequently. Often we were spinning in the (obviously exclusive) spinlock,
+ * while trying to acquire a shared lock that was actually free.
+ *
+ * Thus a new implementation was devised that provides wait-free shared lock
+ * acquisition for locks that aren't exclusively locked.
+ *
+ * The basic idea is to have a single atomic variable 'lockcount' instead of
+ * the formerly separate shared and exclusive counters and to use atomic
+ * operations to acquire the lock. That's fairly easy to do for plain
+ * rw-spinlocks, but a lot harder for something like LWLocks that want to wait
+ * in the OS.
+ *
+ * For lock acquisition we use an atomic compare-and-exchange on the lockcount
+ * variable. For exclusive lock we swap in a sentinel value
+ * (LW_VAL_EXCLUSIVE), for shared locks we count the number of holders.
+ *
+ * To release the lock we use an atomic decrement to release the lock. If the
+ * new value is zero (we get that atomically), we know we can/have to release
+ * waiters.
+ *
+ * Obviously it is important that the sentinel value for exclusive locks
+ * doesn't conflict with the maximum number of possible share lockers -
+ * luckily MAX_BACKENDS makes that easily possible.
+ *
+ *
+ * The attentive reader might have noticed that naively doing the above has a
+ * glaring race condition: We try to lock using the atomic operations and
+ * notice that we have to wait. Unfortunately by the time we have finished
+ * queuing, the former locker very well might have already finished it's
+ * work. That's problematic because we're now stuck waiting inside the OS.
+
+ * To mitigate those races we use a two phased attempt at locking:
+ *   Phase 1: Try to do it atomically, if we succeed, nice
+ *   Phase 2: Add ourselves to the waitqueue of the lock
+ *   Phase 3: Try to grab the lock again, if we succeed, remove ourselves from
+ *            the queue
+ *   Phase 4: Sleep till wake-up, goto Phase 1
+ *
+ * This protects us against the problem from above as nobody can release too
+ *    quick, before we're queued, since after Phase 2 we're already queued.
+ * -------------------------------------------------------------------------
  */
 #include "postgres.h"
 
@@ -35,8 +81,8 @@
 #include "commands/async.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
+#include "postmaster/postmaster.h"
 #include "replication/slot.h"
-#include "storage/barrier.h"
 #include "storage/ipc.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
@@ -51,6 +97,16 @@
 /* We use the ShmemLock spinlock to protect LWLockAssign */
 extern slock_t *ShmemLock;
 
+#define LW_FLAG_HAS_WAITERS			((uint32) 1 << 30)
+#define LW_FLAG_RELEASE_OK			((uint32) 1 << 29)
+
+#define LW_VAL_EXCLUSIVE			((uint32) 1 << 24)
+#define LW_VAL_SHARED				1
+
+#define LW_LOCK_MASK				((uint32) ((1 << 25)-1))
+/* Must be greater than MAX_BACKENDS - which is 2^23-1, so we're fine. */
+#define LW_SHARED_MASK				((uint32)(1 << 23))
+
 /*
  * This is indexed by tranche ID and stores metadata for all tranches known
  * to the current backend.
@@ -81,8 +137,15 @@ static LWLockTranche MainLWLockTranche;
  */
 #define MAX_SIMUL_LWLOCKS	200
 
+/* struct representing the LWLocks we're holding */
+typedef struct LWLockHandle
+{
+	LWLock *lock;
+	LWLockMode	mode;
+} LWLockHandle;
+
 static int	num_held_lwlocks = 0;
-static LWLock *held_lwlocks[MAX_SIMUL_LWLOCKS];
+static LWLockHandle held_lwlocks[MAX_SIMUL_LWLOCKS];
 
 static int	lock_addin_request = 0;
 static bool lock_addin_request_allowed = true;
@@ -103,6 +166,7 @@ typedef struct lwlock_stats
 	int			sh_acquire_count;
 	int			ex_acquire_count;
 	int			block_count;
+	int			dequeue_self_count;
 	int			spin_delay_count;
 }	lwlock_stats;
 
@@ -114,24 +178,42 @@ static lwlock_stats lwlock_stats_dummy;
 bool		Trace_lwlocks = false;
 
 inline static void
-PRINT_LWDEBUG(const char *where, const LWLock *lock)
+PRINT_LWDEBUG(const char *where, LWLock *lock, LWLockMode mode)
 {
+	/* hide statement & context here, otherwise the log is just too verbose */
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): excl %d shared %d rOK %d",
-			 where, T_NAME(lock), T_ID(lock),
-			 (int) lock->exclusive, lock->shared,
-			 (int) lock->releaseOK);
+	{
+		uint32 state = pg_atomic_read_u32(&lock->state);
+		ereport(LOG,
+				(errhidestmt(true),
+				 errhidecontext(true),
+				 errmsg("%d: %s(%s %d): excl %u shared %u haswaiters %u waiters %u rOK %d",
+						MyProcPid,
+						where, T_NAME(lock), T_ID(lock),
+						!!(state & LW_VAL_EXCLUSIVE),
+						state & LW_SHARED_MASK,
+						!!(state & LW_FLAG_HAS_WAITERS),
+						pg_atomic_read_u32(&lock->nwaiters),
+						!!(state & LW_FLAG_RELEASE_OK))));
+	}
 }
 
 inline static void
-LOG_LWDEBUG(const char *where, const char *name, int index, const char *msg)
+LOG_LWDEBUG(const char *where, LWLock *lock, const char *msg)
 {
+	/* hide statement & context here, otherwise the log is just too verbose */
 	if (Trace_lwlocks)
-		elog(LOG, "%s(%s %d): %s", where, name, index, msg);
+	{
+		ereport(LOG,
+				(errhidestmt(true),
+				 errhidecontext(true),
+				 errmsg("%s(%s %d): %s", where, T_NAME(lock), T_ID(lock), msg)));
+	}
 }
+
 #else							/* not LOCK_DEBUG */
-#define PRINT_LWDEBUG(a,b)
-#define LOG_LWDEBUG(a,b,c,d)
+#define PRINT_LWDEBUG(a,b,c) ((void)0)
+#define LOG_LWDEBUG(a,b,c) ((void)0)
 #endif   /* LOCK_DEBUG */
 
 #ifdef LWLOCK_STATS
@@ -192,11 +274,11 @@ print_lwlock_stats(int code, Datum arg)
 	while ((lwstats = (lwlock_stats *) hash_seq_search(&scan)) != NULL)
 	{
 		fprintf(stderr,
-			  "PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u\n",
+				"PID %d lwlock %s %d: shacq %u exacq %u blk %u spindelay %u dequeue self %u\n",
 				MyProcPid, LWLockTrancheArray[lwstats->key.tranche]->name,
 				lwstats->key.instance, lwstats->sh_acquire_count,
 				lwstats->ex_acquire_count, lwstats->block_count,
-				lwstats->spin_delay_count);
+				lwstats->spin_delay_count, lwstats->dequeue_self_count);
 	}
 
 	LWLockRelease(&MainLWLockArray[0].lock);
@@ -226,6 +308,7 @@ get_lwlock_stats_entry(LWLock *lock)
 		lwstats->sh_acquire_count = 0;
 		lwstats->ex_acquire_count = 0;
 		lwstats->block_count = 0;
+		lwstats->dequeue_self_count = 0;
 		lwstats->spin_delay_count = 0;
 	}
 	return lwstats;
@@ -336,6 +419,9 @@ LWLockShmemSize(void)
 void
 CreateLWLocks(void)
 {
+	StaticAssertExpr(LW_VAL_EXCLUSIVE > (uint32) MAX_BACKENDS,
+					 "MAX_BACKENDS too big for lwlock.c");
+
 	if (!IsUnderPostmaster)
 	{
 		int			numLocks = NumLWLocks();
@@ -475,13 +561,330 @@ void
 LWLockInitialize(LWLock *lock, int tranche_id)
 {
 	SpinLockInit(&lock->mutex);
-	lock->releaseOK = true;
-	lock->exclusive = 0;
-	lock->shared = 0;
+	pg_atomic_init_u32(&lock->state, LW_FLAG_RELEASE_OK);
+#ifdef LOCK_DEBUG
+	pg_atomic_init_u32(&lock->nwaiters, 0);
+#endif
 	lock->tranche = tranche_id;
 	dlist_init(&lock->waiters);
 }
 
+/*
+ * Internal function that tries to atomically acquire the lwlock in the passed
+ * in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * callers job.
+ *
+ * Returns true if the lock isn't free and we need to wait.
+ */
+static bool
+LWLockAttemptLock(LWLock* lock, LWLockMode mode)
+{
+	AssertArg(mode == LW_EXCLUSIVE || mode == LW_SHARED);
+
+	/* loop until we've determined whether we could acquire the lock or not */
+	while (true)
+	{
+		uint32 old_state;
+		uint32 expected_state;
+		uint32 desired_state;
+		bool lock_free;
+
+		old_state = pg_atomic_read_u32(&lock->state);
+		expected_state = old_state;
+		desired_state = expected_state;
+
+		if (mode == LW_EXCLUSIVE)
+		{
+			lock_free = (expected_state & LW_LOCK_MASK) == 0;
+			if (lock_free)
+				desired_state += LW_VAL_EXCLUSIVE;
+		}
+		else
+		{
+			lock_free = (expected_state & LW_VAL_EXCLUSIVE) == 0;
+			if (lock_free)
+				desired_state += LW_VAL_SHARED;
+		}
+
+		/*
+		 * Attempt to swap in the state we are expecting. If we didn't see
+		 * lock to be free, that's just the old value. If we saw it as free,
+		 * we'll attempt to mark it acquired. The reason that we always swap
+		 * in the value is that this doubles as a memory barrier. We could try
+		 * to be smarter and only swap in values if we saw the lock as free,
+		 * but benchmark haven't shown it as beneficial so far.
+		 *
+		 * Retry if the value changed since we last looked at it.
+		 */
+		if (pg_atomic_compare_exchange_u32(&lock->state,
+										   &expected_state, desired_state))
+		{
+			if (lock_free)
+			{
+				/* Great! Got the lock. */
+#ifdef LOCK_DEBUG
+				if (mode == LW_EXCLUSIVE)
+					lock->owner = MyProc;
+#endif
+				return false;
+			}
+			else
+				return true; /* someobdy else has the lock */
+		}
+	}
+	pg_unreachable();
+}
+
+/*
+ * Wakeup all the lockers that currently have a chance to acquire the lock.
+ */
+static void
+LWLockWakeup(LWLock *lock)
+{
+	bool		new_release_ok;
+	bool		wokeup_somebody = false;
+	dlist_head	wakeup;
+	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	dlist_init(&wakeup);
+
+	new_release_ok = true;
+
+	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	dlist_foreach_modify(iter, &lock->waiters)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
+			continue;
+
+		dlist_delete(&waiter->lwWaitLink);
+		dlist_push_tail(&wakeup, &waiter->lwWaitLink);
+
+		if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
+		{
+			/*
+			 * Prevent additional wakeups until retryer gets to run. Backends
+			 * that are just waiting for the lock to become free don't retry
+			 * automatically.
+			 */
+			new_release_ok = false;
+			/*
+			 * Don't wakeup (further) exclusive locks.
+			 */
+			wokeup_somebody = true;
+		}
+
+		/*
+		 * Once we've woken up an exclusive lock, there's no point in waking
+		 * up anybody else.
+		 */
+		if(waiter->lwWaitMode == LW_EXCLUSIVE)
+			break;
+	}
+
+	Assert(dlist_is_empty(&wakeup) || pg_atomic_read_u32(&lock->state) & LW_FLAG_HAS_WAITERS);
+
+	/* Unset both flags at once if required */
+	if (!new_release_ok && dlist_is_empty(&wakeup))
+		pg_atomic_fetch_and_u32(&lock->state,
+								~(LW_FLAG_RELEASE_OK | LW_FLAG_HAS_WAITERS));
+	else if (!new_release_ok)
+		pg_atomic_fetch_and_u32(&lock->state, ~LW_FLAG_RELEASE_OK);
+	else if (dlist_is_empty(&wakeup))
+		pg_atomic_fetch_and_u32(&lock->state, ~LW_FLAG_HAS_WAITERS);
+	else if (new_release_ok)
+		pg_atomic_fetch_or_u32(&lock->state, LW_FLAG_RELEASE_OK);
+
+	/* We are done updating the shared state of the lock queue. */
+	SpinLockRelease(&lock->mutex);
+
+	/* Awaken any waiters I removed from the queue. */
+	dlist_foreach_modify(iter, &wakeup)
+	{
+		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
+
+		LOG_LWDEBUG("LWLockRelease", lock, "release waiter");
+		dlist_delete(&waiter->lwWaitLink);
+		/*
+		 * Guarantee that lwWaiting being unset only becomes visible once the
+		 * unlink from the link has completed. Otherwise the target backend
+		 * could be woken up for other reason and enqueue for a new lock - if
+		 * that happens before the list unlink happens, the list would end up
+		 * being corrupted.
+		 *
+		 * The barrier pairs with the SpinLockAcquire() when enqueing for
+		 * another lock.
+		 */
+		pg_write_barrier();
+		waiter->lwWaiting = false;
+		PGSemaphoreUnlock(&waiter->sem);
+	}
+}
+
+/*
+ * Add ourselves to the end of the queue.
+ *
+ * NB: Mode can be LW_WAIT_UNTIL_FREE here!
+ */
+static void
+LWLockQueueSelf(LWLock *lock, LWLockMode mode)
+{
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	/*
+	 * If we don't have a PGPROC structure, there's no way to wait. This
+	 * should never occur, since MyProc should only be null during shared
+	 * memory initialization.
+	 */
+	if (MyProc == NULL)
+		elog(PANIC, "cannot wait without a PGPROC structure");
+
+	if (MyProc->lwWaiting)
+		elog(PANIC, "queueing for lock while waiting on another one");
+
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/* setting the flag is protected by the spinlock */
+	pg_atomic_fetch_or_u32(&lock->state, LW_FLAG_HAS_WAITERS);
+
+	MyProc->lwWaiting = true;
+	MyProc->lwWaitMode = mode;
+
+	/* LW_WAIT_UNTIL_FREE waiters are always at the front of the queue */
+	if (mode == LW_WAIT_UNTIL_FREE)
+		dlist_push_head(&lock->waiters, &MyProc->lwWaitLink);
+	else
+		dlist_push_tail(&lock->waiters, &MyProc->lwWaitLink);
+
+	/* Can release the mutex now */
+	SpinLockRelease(&lock->mutex);
+
+#ifdef LOCK_DEBUG
+	pg_atomic_fetch_add_u32(&lock->nwaiters, 1);
+#endif
+
+}
+
+/*
+ * Remove ourselves from the waitlist.
+ *
+ * This is used if we queued ourselves because we thought we needed to sleep
+ * but, after further checking, we discovered that we don't actually need to
+ * do so. Returns false if somebody else already has woken us up, otherwise
+ * returns true.
+ */
+static void
+LWLockDequeueSelf(LWLock *lock)
+{
+	bool	found = false;
+	dlist_mutable_iter iter;
+
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+
+	lwstats->dequeue_self_count++;
+#endif
+
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
+	SpinLockAcquire(&lock->mutex);
+#endif
+
+	/*
+	 * Can't just remove ourselves from the list, but we need to iterate over
+	 * all entries as somebody else could have unqueued us.
+	 */
+	dlist_foreach_modify(iter, &lock->waiters)
+	{
+		PGPROC *proc = dlist_container(PGPROC, lwWaitLink, iter.cur);
+		if (proc == MyProc)
+		{
+			found = true;
+			dlist_delete(&proc->lwWaitLink);
+			break;
+		}
+	}
+
+	if (dlist_is_empty(&lock->waiters) &&
+		(pg_atomic_read_u32(&lock->state) & LW_FLAG_HAS_WAITERS) != 0)
+	{
+		pg_atomic_fetch_and_u32(&lock->state, ~LW_FLAG_HAS_WAITERS);
+	}
+
+	SpinLockRelease(&lock->mutex);
+
+	/* clear waiting state again, nice for debugging */
+	if (found)
+		MyProc->lwWaiting = false;
+	else
+	{
+		int		extraWaits = 0;
+
+		/*
+		 * Somebody else dequeued us and has or will wake us up. Deal with the
+		 * superflous absorption of a wakeup.
+		 */
+
+		/*
+		 * Reset releaseOk if somebody woke us before we removed ourselves -
+		 * they'll have set it to false.
+		 */
+		pg_atomic_fetch_or_u32(&lock->state, LW_FLAG_RELEASE_OK);
+
+		/*
+		 * Now wait for the scheduled wakeup, otherwise our ->lwWaiting would
+		 * get reset at some inconvenient point later. Most of the time this
+		 * will immediately return.
+		 */
+		for (;;)
+		{
+			/* "false" means cannot accept cancel/die interrupt here. */
+			PGSemaphoreLock(&MyProc->sem, false);
+			if (!MyProc->lwWaiting)
+				break;
+			extraWaits++;
+		}
+
+		/*
+		 * Fix the process wait semaphore's count for any absorbed wakeups.
+		 */
+		while (extraWaits-- > 0)
+			PGSemaphoreUnlock(&MyProc->sem);
+	}
+
+#ifdef LOCK_DEBUG
+	{
+		/* not waiting anymore */
+		uint32 nwaiters = pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+		Assert(nwaiters < MAX_BACKENDS);
+	}
+#endif
+}
 
 /*
  * LWLockAcquire - acquire a lightweight lock in the specified mode
@@ -513,18 +916,19 @@ static inline bool
 LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 {
 	PGPROC	   *proc = MyProc;
-	bool		retry = false;
 	bool		result = true;
 	int			extraWaits = 0;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
 #endif
 
-	PRINT_LWDEBUG("LWLockAcquire", lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
 
-#ifdef LWLOCK_STATS
-	lwstats = get_lwlock_stats_entry(lock);
+	PRINT_LWDEBUG("LWLockAcquire", lock, mode);
 
+#ifdef LWLOCK_STATS
 	/* Count lock acquisition attempts */
 	if (mode == LW_EXCLUSIVE)
 		lwstats->ex_acquire_count++;
@@ -570,58 +974,44 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 	{
 		bool		mustwait;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
-#ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
-#else
-		SpinLockAcquire(&lock->mutex);
-#endif
-
-		/* If retrying, allow LWLockRelease to release waiters again */
-		if (retry)
-			lock->releaseOK = true;
-
-		/* If I can get the lock, do so quickly. */
-		if (mode == LW_EXCLUSIVE)
-		{
-			if (lock->exclusive == 0 && lock->shared == 0)
-			{
-				lock->exclusive++;
-				mustwait = false;
-			}
-			else
-				mustwait = true;
-		}
-		else
-		{
-			if (lock->exclusive == 0)
-			{
-				lock->shared++;
-				mustwait = false;
-			}
-			else
-				mustwait = true;
-		}
+		/*
+		 * Try to grab the lock the first time, we're not in the waitqueue
+		 * yet/anymore.
+		 */
+		mustwait = LWLockAttemptLock(lock, mode);
 
 		if (!mustwait)
+		{
+			/* XXX: remove before commit? */
+			LOG_LWDEBUG("LWLockAcquire", lock, "immediately acquired lock");
 			break;				/* got the lock */
+		}
 
 		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait. This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
+		 * Ok, at this point we couldn't grab the lock on the first try. We
+		 * cannot simply queue ourselves to the end of the list and wait to be
+		 * woken up because by now the lock could long have been released.
+		 * Instead add us to the queue and try to grab the lock again. If we
+		 * succeed we need to revert the queuing and be happy, otherwise we
+		 * recheck the lock. If we still couldn't grab it, we know that the
+		 * other lock will see our queue entries when releasing since they
+		 * existed before we checked for the lock.
 		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
 
-		proc->lwWaiting = true;
-		proc->lwWaitMode = mode;
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		/* add to the queue */
+		LWLockQueueSelf(lock, mode);
+
+		/* we're now guaranteed to be woken up if necessary */
+		mustwait = LWLockAttemptLock(lock, mode);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
+		{
+			LOG_LWDEBUG("LWLockAcquire", lock, "acquired, undoing queue");
+
+			LWLockDequeueSelf(lock);
+			break;
+		}
 
 		/*
 		 * Wait until awakened.
@@ -635,7 +1025,7 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockAcquire", T_NAME(lock), T_ID(lock), "waiting");
+		LOG_LWDEBUG("LWLockAcquire", lock, "waiting");
 
 #ifdef LWLOCK_STATS
 		lwstats->block_count++;
@@ -652,12 +1042,22 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 			extraWaits++;
 		}
 
+		/* Retrying, allow LWLockRelease to release waiters again. */
+		pg_atomic_fetch_or_u32(&lock->state, LW_FLAG_RELEASE_OK);
+
+#ifdef LOCK_DEBUG
+		{
+			/* not waiting anymore */
+			uint32 nwaiters = pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+			Assert(nwaiters < MAX_BACKENDS);
+		}
+#endif
+
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
-		LOG_LWDEBUG("LWLockAcquire", T_NAME(lock), T_ID(lock), "awakened");
+		LOG_LWDEBUG("LWLockAcquire", lock, "awakened");
 
 		/* Now loop back and try to acquire lock again. */
-		retry = true;
 		result = false;
 	}
 
@@ -665,13 +1065,11 @@ LWLockAcquireCommon(LWLock *lock, LWLockMode mode, uint64 *valptr, uint64 val)
 	if (valptr)
 		*valptr = val;
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), mode);
 
 	/* Add lock to list of locks held by this backend */
-	held_lwlocks[num_held_lwlocks++] = lock;
+	held_lwlocks[num_held_lwlocks].lock = lock;
+	held_lwlocks[num_held_lwlocks++].mode = mode;
 
 	/*
 	 * Fix the process wait semaphore's count for any absorbed wakeups.
@@ -694,7 +1092,9 @@ LWLockConditionalAcquire(LWLock *lock, LWLockMode mode)
 {
 	bool		mustwait;
 
-	PRINT_LWDEBUG("LWLockConditionalAcquire", lock);
+	AssertArg(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockConditionalAcquire", lock, mode);
 
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
@@ -707,50 +1107,24 @@ LWLockConditionalAcquire(LWLock *lock, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
+	/* Check for the lock */
+	mustwait = LWLockAttemptLock(lock, mode);
 
 	if (mustwait)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockConditionalAcquire",
-					T_NAME(lock), T_ID(lock), "failed");
-		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock),
-												 T_ID(lock), mode);
+
+		LOG_LWDEBUG("LWLockConditionalAcquire", lock, "failed");
+		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE_FAIL(T_NAME(lock), T_ID(lock), mode);
 	}
 	else
 	{
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lock;
+		held_lwlocks[num_held_lwlocks].lock = lock;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
 		TRACE_POSTGRESQL_LWLOCK_CONDACQUIRE(T_NAME(lock), T_ID(lock), mode);
 	}
-
 	return !mustwait;
 }
 
@@ -776,14 +1150,14 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	int			extraWaits = 0;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
-#endif
-
-	PRINT_LWDEBUG("LWLockAcquireOrWait", lock);
 
-#ifdef LWLOCK_STATS
 	lwstats = get_lwlock_stats_entry(lock);
 #endif
 
+	Assert(mode == LW_SHARED || mode == LW_EXCLUSIVE);
+
+	PRINT_LWDEBUG("LWLockAcquireOrWait", lock, mode);
+
 	/* Ensure we will have room to remember the lock */
 	if (num_held_lwlocks >= MAX_SIMUL_LWLOCKS)
 		elog(ERROR, "too many LWLocks taken");
@@ -795,81 +1169,63 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	 */
 	HOLD_INTERRUPTS();
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* If I can get the lock, do so quickly. */
-	if (mode == LW_EXCLUSIVE)
-	{
-		if (lock->exclusive == 0 && lock->shared == 0)
-		{
-			lock->exclusive++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
-	else
-	{
-		if (lock->exclusive == 0)
-		{
-			lock->shared++;
-			mustwait = false;
-		}
-		else
-			mustwait = true;
-	}
+	/*
+	 * NB: We're using nearly the same twice-in-a-row lock acquisition
+	 * protocol as LWLockAcquire(). Check its comments for details.
+	 */
+	mustwait = LWLockAttemptLock(lock, mode);
 
 	if (mustwait)
 	{
-		/*
-		 * Add myself to wait queue.
-		 *
-		 * If we don't have a PGPROC structure, there's no way to wait.  This
-		 * should never occur, since MyProc should only be null during shared
-		 * memory initialization.
-		 */
-		if (proc == NULL)
-			elog(PANIC, "cannot wait without a PGPROC structure");
-
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
+		mustwait = LWLockAttemptLock(lock, mode);
 
-		/*
-		 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
-		 * wakups, because we share the semaphore with ProcWaitForSignal.
-		 */
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock),
-					"waiting");
+		if (mustwait)
+		{
+			/*
+			 * Wait until awakened.  Like in LWLockAcquire, be prepared for bogus
+			 * wakups, because we share the semaphore with ProcWaitForSignal.
+			 */
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "waiting");
 
 #ifdef LWLOCK_STATS
-		lwstats->block_count++;
+			lwstats->block_count++;
 #endif
+			TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(l), T_ID(l), mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_START(T_NAME(lock), T_ID(lock), mode);
+			for (;;)
+			{
+				/* "false" means cannot accept cancel/die interrupt here. */
+				PGSemaphoreLock(&proc->sem, false);
+				if (!proc->lwWaiting)
+					break;
+				extraWaits++;
+			}
 
-		for (;;)
-		{
-			/* "false" means cannot accept cancel/die interrupt here. */
-			PGSemaphoreLock(&proc->sem, false);
-			if (!proc->lwWaiting)
-				break;
-			extraWaits++;
-		}
+#ifdef LOCK_DEBUG
+			{
+				/* not waiting anymore */
+				uint32 nwaiters = pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+				Assert(nwaiters < MAX_BACKENDS);
+			}
+#endif
+			TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
 
-		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock), mode);
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "awakened");
+		}
+		else
+		{
+			LOG_LWDEBUG("LWLockAcquireOrWait", lock, "acquired, undoing queue");
 
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock),
-					"awakened");
-	}
-	else
-	{
-		/* We are done updating shared state of the lock itself. */
-		SpinLockRelease(&lock->mutex);
+			/*
+			  * Got lock in the second attempt, undo queueing. We need to
+			  * treat this as having successfully acquired the lock, otherwise
+			  * we'd not necessarily wake up people we've prevented from
+			  * acquiring the lock.
+			  */
+			LWLockDequeueSelf(lock);
+		}
 	}
 
 	/*
@@ -882,16 +1238,17 @@ LWLockAcquireOrWait(LWLock *lock, LWLockMode mode)
 	{
 		/* Failed to get lock, so release interrupt holdoff */
 		RESUME_INTERRUPTS();
-		LOG_LWDEBUG("LWLockAcquireOrWait", T_NAME(lock), T_ID(lock), "failed");
+		LOG_LWDEBUG("LWLockAcquireOrWait", lock, "failed");
 		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT_FAIL(T_NAME(lock), T_ID(lock),
 													 mode);
 	}
 	else
 	{
+		LOG_LWDEBUG("LWLockAcquireOrWait", lock, "succeeded");
 		/* Add lock to list of locks held by this backend */
-		held_lwlocks[num_held_lwlocks++] = lock;
-		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock),
-												mode);
+		held_lwlocks[num_held_lwlocks].lock = lock;
+		held_lwlocks[num_held_lwlocks++].mode = mode;
+		TRACE_POSTGRESQL_LWLOCK_ACQUIRE_OR_WAIT(T_NAME(lock), T_ID(lock), mode);
 	}
 
 	return !mustwait;
@@ -923,13 +1280,11 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 	bool		result = false;
 #ifdef LWLOCK_STATS
 	lwlock_stats *lwstats;
-#endif
-
-	PRINT_LWDEBUG("LWLockWaitForVar", lock);
 
-#ifdef LWLOCK_STATS
 	lwstats = get_lwlock_stats_entry(lock);
-#endif   /* LWLOCK_STATS */
+#endif
+
+	PRINT_LWDEBUG("LWLockWaitForVar", lock, LW_WAIT_UNTIL_FREE);
 
 	/*
 	 * Quick test first to see if it the slot is free right now.
@@ -938,7 +1293,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 	 * barrier here as far as the current usage is concerned.  But that might
 	 * not be safe in general.
 	 */
-	if (lock->exclusive == 0)
+	if ((pg_atomic_read_u32(&lock->state) & LW_VAL_EXCLUSIVE) == 0)
 		return true;
 
 	/*
@@ -956,21 +1311,24 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		bool		mustwait;
 		uint64		value;
 
-		/* Acquire mutex.  Time spent holding mutex should be short! */
+		mustwait = (pg_atomic_read_u32(&lock->state) & LW_VAL_EXCLUSIVE) != 0;
+
+		if (mustwait)
+		{
+			/*
+			 * Perform comparison using spinlock as we can't rely on atomic 64
+			 * bit reads/stores.
+			 */
 #ifdef LWLOCK_STATS
-		lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+			lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
 #else
-		SpinLockAcquire(&lock->mutex);
+			SpinLockAcquire(&lock->mutex);
 #endif
 
-		/* Is the lock now free, and if not, does the value match? */
-		if (lock->exclusive == 0)
-		{
-			result = true;
-			mustwait = false;
-		}
-		else
-		{
+			/*
+			 * XXX: We can significantly optimize this on platforms with 64bit
+			 * atomics.
+			 */
 			value = *valptr;
 			if (value != oldval)
 			{
@@ -980,27 +1338,42 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 			}
 			else
 				mustwait = true;
+			SpinLockRelease(&lock->mutex);
 		}
+		else
+			mustwait = false;
 
 		if (!mustwait)
 			break;				/* the lock was free or value didn't match */
 
 		/*
-		 * Add myself to wait queue.
+		 * Add myself to wait queue. Note that this is racy, somebody else
+		 * could wakeup before we're finished queuing.
+		 * NB: We're using nearly the same twice-in-a-row lock acquisition
+		 * protocol as LWLockAcquire(). Check its comments for details.
+		 */
+		LWLockQueueSelf(lock, LW_WAIT_UNTIL_FREE);
+
+		/*
+		 * Set RELEASE_OK flag, to make sure we get woken up as soon as the
+		 * lock is released.
 		 */
-		proc->lwWaiting = true;
-		proc->lwWaitMode = LW_WAIT_UNTIL_FREE;
-		/* waiters are added to the front of the queue */
-		dlist_push_head(&lock->waiters, &proc->lwWaitLink);
+		pg_atomic_fetch_or_u32(&lock->state, LW_FLAG_RELEASE_OK);
 
 		/*
-		 * Set releaseOK, to make sure we get woken up as soon as the lock is
-		 * released.
+		 * We're now guaranteed to be woken up if necessary. Recheck the
+		 * lock's state.
 		 */
-		lock->releaseOK = true;
+		mustwait = (pg_atomic_read_u32(&lock->state) & LW_VAL_EXCLUSIVE) != 0;
+
+		/* Ok, lock is free after we queued ourselves. Undo queueing. */
+		if (!mustwait)
+		{
+			LOG_LWDEBUG("LWLockWaitForVar", lock, "free, undoing queue");
 
-		/* Can release the mutex now */
-		SpinLockRelease(&lock->mutex);
+			LWLockDequeueSelf(lock);
+			break;
+		}
 
 		/*
 		 * Wait until awakened.
@@ -1014,7 +1387,7 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 		 * so that the lock manager or signal manager will see the received
 		 * signal when it next waits.
 		 */
-		LOG_LWDEBUG("LWLockWaitForVar", T_NAME(lock), T_ID(lock), "waiting");
+		LOG_LWDEBUG("LWLockWaitForVar", lock, "waiting");
 
 #ifdef LWLOCK_STATS
 		lwstats->block_count++;
@@ -1032,17 +1405,22 @@ LWLockWaitForVar(LWLock *lock, uint64 *valptr, uint64 oldval, uint64 *newval)
 			extraWaits++;
 		}
 
+#ifdef LOCK_DEBUG
+		{
+			/* not waiting anymore */
+			uint32 nwaiters = pg_atomic_fetch_sub_u32(&lock->nwaiters, 1);
+			Assert(nwaiters < MAX_BACKENDS);
+		}
+#endif
+
 		TRACE_POSTGRESQL_LWLOCK_WAIT_DONE(T_NAME(lock), T_ID(lock),
 										  LW_EXCLUSIVE);
 
-		LOG_LWDEBUG("LWLockWaitForVar", T_NAME(lock), T_ID(lock), "awakened");
+		LOG_LWDEBUG("LWLockWaitForVar", lock, "awakened");
 
 		/* Now loop back and check the status of the lock again. */
 	}
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
-
 	TRACE_POSTGRESQL_LWLOCK_ACQUIRE(T_NAME(lock), T_ID(lock), LW_EXCLUSIVE);
 
 	/*
@@ -1075,14 +1453,24 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 {
 	dlist_head	wakeup;
 	dlist_mutable_iter iter;
+#ifdef LWLOCK_STATS
+	lwlock_stats *lwstats;
+
+	lwstats = get_lwlock_stats_entry(lock);
+#endif
+
+	PRINT_LWDEBUG("LWLockUpdateVar", lock, LW_EXCLUSIVE);
 
 	dlist_init(&wakeup);
 
 	/* Acquire mutex.  Time spent holding mutex should be short! */
+#ifdef LWLOCK_STATS
+	lwstats->spin_delay_count += SpinLockAcquire(&lock->mutex);
+#else
 	SpinLockAcquire(&lock->mutex);
+#endif
 
-	/* we should hold the lock */
-	Assert(lock->exclusive == 1);
+	Assert(pg_atomic_read_u32(&lock->state) & LW_VAL_EXCLUSIVE);
 
 	/* Update the lock's value */
 	*valptr = val;
@@ -1112,7 +1500,7 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 	{
 		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
 		dlist_delete(&waiter->lwWaitLink);
-		/* check comment in LWLockRelease() about this barrier */
+		/* check comment in LWLockWakeup() about this barrier */
 		pg_write_barrier();
 		waiter->lwWaiting = false;
 		PGSemaphoreUnlock(&waiter->sem);
@@ -1126,22 +1514,22 @@ LWLockUpdateVar(LWLock *lock, uint64 *valptr, uint64 val)
 void
 LWLockRelease(LWLock *lock)
 {
-	dlist_head	wakeup;
-	dlist_mutable_iter iter;
+	LWLockMode	mode;
+	uint32		oldstate;
+	bool		check_waiters;
 	int			i;
 
-	dlist_init(&wakeup);
-
-	PRINT_LWDEBUG("LWLockRelease", lock);
-
 	/*
 	 * Remove lock from list of locks held.  Usually, but not always, it will
 	 * be the latest-acquired lock; so search array backwards.
 	 */
 	for (i = num_held_lwlocks; --i >= 0;)
 	{
-		if (lock == held_lwlocks[i])
+		if (lock == held_lwlocks[i].lock)
+		{
+			mode = held_lwlocks[i].mode;
 			break;
+		}
 	}
 	if (i < 0)
 		elog(ERROR, "lock %s %d is not held", T_NAME(lock), T_ID(lock));
@@ -1149,88 +1537,45 @@ LWLockRelease(LWLock *lock)
 	for (; i < num_held_lwlocks; i++)
 		held_lwlocks[i] = held_lwlocks[i + 1];
 
-	/* Acquire mutex.  Time spent holding mutex should be short! */
-	SpinLockAcquire(&lock->mutex);
-
-	/* Release my hold on lock */
-	if (lock->exclusive > 0)
-		lock->exclusive--;
-	else
-	{
-		Assert(lock->shared > 0);
-		lock->shared--;
-	}
+	PRINT_LWDEBUG("LWLockRelease", lock, mode);
 
 	/*
-	 * See if I need to awaken any waiters.  If I released a non-last shared
-	 * hold, there cannot be anything to do.  Also, do not awaken any waiters
-	 * if someone has already awakened waiters that haven't yet acquired the
-	 * lock.
+	 * Release my hold on lock, after that it can immediately be acquired by
+	 * others, even if we still have to wakeup other waiters.
 	 */
-	if (lock->exclusive == 0 && lock->shared == 0 && lock->releaseOK)
-	{
-		/*
-		 * Remove the to-be-awakened PGPROCs from the queue.
-		 */
-		bool		releaseOK = true;
-		bool		wokeup_somebody = false;
-
-		dlist_foreach_modify(iter, &lock->waiters)
-		{
-			PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-
-			if (wokeup_somebody && waiter->lwWaitMode == LW_EXCLUSIVE)
-				continue;
-
-			dlist_delete(&waiter->lwWaitLink);
-			dlist_push_tail(&wakeup, &waiter->lwWaitLink);
-
-			/*
-			 * Prevent additional wakeups until retryer gets to
-			 * run. Backends that are just waiting for the lock to become
-			 * free don't retry automatically.
-			 */
-			if (waiter->lwWaitMode != LW_WAIT_UNTIL_FREE)
-			{
-				releaseOK = false;
-				wokeup_somebody = true;
-			}
+	if (mode == LW_EXCLUSIVE)
+		oldstate = pg_atomic_sub_fetch_u32(&lock->state, LW_VAL_EXCLUSIVE);
+	else
+		oldstate = pg_atomic_sub_fetch_u32(&lock->state, LW_VAL_SHARED);
 
-			if(waiter->lwWaitMode == LW_EXCLUSIVE)
-				break;
-		}
-		lock->releaseOK = releaseOK;
-	}
+	/* nobody else can have that kind of lock */
+	Assert(!(oldstate & LW_VAL_EXCLUSIVE));
 
-	/* We are done updating shared state of the lock itself. */
-	SpinLockRelease(&lock->mutex);
 
-	TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+	/*
+	 * We're still waiting for backends to get scheduled, don't wake them up
+	 * again.
+	 */
+	if ((oldstate & (LW_FLAG_HAS_WAITERS | LW_FLAG_RELEASE_OK)) ==
+		(LW_FLAG_HAS_WAITERS | LW_FLAG_RELEASE_OK) &&
+		(oldstate & LW_LOCK_MASK) == 0)
+		check_waiters = true;
+	else
+		check_waiters = false;
 
 	/*
-	 * Awaken any waiters I removed from the queue.
+	 * As waking up waiters requires the spinlock to be acquired, only do so
+	 * if necessary.
 	 */
-	dlist_foreach_modify(iter, &wakeup)
+	if (check_waiters)
 	{
-		PGPROC *waiter = dlist_container(PGPROC, lwWaitLink, iter.cur);
-		LOG_LWDEBUG("LWLockRelease", T_NAME(lock), T_ID(lock),
-					"release waiter");
-		dlist_delete(&waiter->lwWaitLink);
-		/*
-		 * Guarantee that lwWaiting being unset only becomes visible once the
-		 * unlink from the link has completed. Otherwise the target backend
-		 * could be woken up for other reason and enqueue for a new lock - if
-		 * that happens before the list unlink happens, the list would end up
-		 * being corrupted.
-		 *
-		 * The barrier pairs with the SpinLockAcquire() when enqueing for
-		 * another lock.
-		 */
-		pg_write_barrier();
-		waiter->lwWaiting = false;
-		PGSemaphoreUnlock(&waiter->sem);
+		/* XXX: remove before commit? */
+		LOG_LWDEBUG("LWLockRelease", lock, "releasing waiters");
+		LWLockWakeup(lock);
 	}
 
+	TRACE_POSTGRESQL_LWLOCK_RELEASE(T_NAME(lock), T_ID(lock));
+
 	/*
 	 * Now okay to allow cancel/die interrupts.
 	 */
@@ -1254,7 +1599,7 @@ LWLockReleaseAll(void)
 	{
 		HOLD_INTERRUPTS();		/* match the upcoming RESUME_INTERRUPTS */
 
-		LWLockRelease(held_lwlocks[num_held_lwlocks - 1]);
+		LWLockRelease(held_lwlocks[num_held_lwlocks - 1].lock);
 	}
 }
 
@@ -1262,8 +1607,8 @@ LWLockReleaseAll(void)
 /*
  * LWLockHeldByMe - test whether my process currently holds a lock
  *
- * This is meant as debug support only.  We do not distinguish whether the
- * lock is held shared or exclusive.
+ * This is meant as debug support only.  We currently do not distinguish
+ * whether the lock is held shared or exclusive.
  */
 bool
 LWLockHeldByMe(LWLock *l)
@@ -1272,7 +1617,7 @@ LWLockHeldByMe(LWLock *l)
 
 	for (i = 0; i < num_held_lwlocks; i++)
 	{
-		if (held_lwlocks[i] == l)
+		if (held_lwlocks[i].lock == l)
 			return true;
 	}
 	return false;
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index c84970a..f15a951 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -16,6 +16,7 @@
 
 #include "lib/ilist.h"
 #include "storage/s_lock.h"
+#include "port/atomics.h"
 
 struct PGPROC;
 
@@ -47,11 +48,16 @@ typedef struct LWLockTranche
 typedef struct LWLock
 {
 	slock_t		mutex;			/* Protects LWLock and queue of PGPROCs */
-	bool		releaseOK;		/* T if ok to release waiters */
-	char		exclusive;		/* # of exclusive holders (0 or 1) */
-	int			shared;			/* # of shared holders (0..MaxBackends) */
-	int			tranche;		/* tranche ID */
+	uint16		tranche;		/* tranche ID */
+
+	pg_atomic_uint32 state;		/* state of exlusive/nonexclusive lockers */
+#ifdef LOCK_DEBUG
+	pg_atomic_uint32 nwaiters;	/* number of waiters */
+#endif
 	dlist_head	waiters;		/* list of waiting PGPROCs */
+#ifdef LOCK_DEBUG
+	struct PGPROC *owner;		/* last exlusive owner of the lock */
+#endif
 } LWLock;
 
 /*
@@ -66,11 +72,11 @@ typedef struct LWLock
  * (Of course, we have to also ensure that the array start address is suitably
  * aligned.)
  *
- * Even on a 32-bit platform, an lwlock will be more than 16 bytes, because
- * it contains 2 integers and 2 pointers, plus other stuff.  It should fit
- * into 32 bytes, though, unless slock_t is really big.  On a 64-bit platform,
- * it should fit into 32 bytes unless slock_t is larger than 4 bytes.  We
- * allow for that just in case.
+ * On a 32-bit platforms a LWLock will these days fit into 16 bytes, but since
+ * that didn't use to be the case and cramming more lwlocks into a cacheline
+ * might be detrimental performancewise we still use 32 byte alignment
+ * there. So, both on 32 and 64 bit platforms, it should fit into 32 bytes
+ * unless slock_t is really big.  We allow for that just in case.
  */
 #define LWLOCK_PADDED_SIZE	(sizeof(LWLock) <= 32 ? 32 : 64)
 
-- 
2.2.0.rc0.18.ga1ad247