From 905c67a8387286bbd39ebca49c2b127403de4e80 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Wed, 19 Nov 2025 16:37:26 -0500
Subject: [PATCH v6 08/14] bufmgr: Implement buffer content locks independently
 of lwlocks

Until now buffer content locks were implemented using lwlocks. That has the
obvious advantage of not needing a separate efficient implementation of
locks. However, the time for a dedicated buffer content lock implementation
has come:

1) Hint bits are currently set while holding only a share lock. This leads to
   having to copy pages while they are being written out if checksums are
   enabled, which is not cheap. We would like to add AIO writes, however once
   many buffers can be written out at the same time, it gets a lot more
   expensive to copy them, particularly because that copy needs to reside in
   shared buffers (for worker mode to have access to the buffer).

   In addition, modifying buffers while they are being written out can cause
   issues with unbuffered/direct-IO, as some filesystems (like btrfs) do not
   like that, due to filesystem internal checksums getting corrupted.

   The solution to this is to require a new share-exclusive lock-level to set
   hint bits and to write out buffers, making those operations mutually
   exclusive. We could introduce such a lock level into the generic lwlock
   implementation, however it does not look like there would be other users,
   and it does add some overhead into important codepaths.

2) For AIO writes we need to be able to race-freely check whether a buffer is
   undergoing IO and whether an exclusive lock on the page can be acquired. That
   is rather hard to do efficiently when the buffer state and the lock state
   are separate atomic variables. This is a major hindrance to allowing writes
   to be done asynchronously.

3) Buffer locks are by far the most frequently taken locks. Optimizing them
   specifically for their use case is worth the effort. E.g. by merging
   content locks into buffer locks we will be able to release a buffer lock
   and pin in one atomic operation.

4) There are more complicated optimizations, like long-lived "super pinned &
   locked" pages, that cannot realistically be implemented with the generic
   lwlock implementation.

Therefore implement content locks inside bufmgr.c. The lockstate is stored as
part of BufferDesc.state. The implementation of buffer content locks is fairly
similar to lwlocks, with a few important differences:

1) An additional lock-level share-exclusive has been added. This lock level
   conflicts with exclusive locks and itself, but not share locks.

2) Error recovery for content locks is implemented as part of the already
   existing private-refcount tracking mechanism in combination with resowners,
   instead of a bespoke mechanism as the case for lwlocks. This means we do
   not need to add dedicated error-recovery codepaths to release all content
   locks (like done with LWLockReleaseAll() for content locks).

3) The lock state is embedded in BufferDesc.state instead of having its own
   struct.

4) The wakeup logic is a tad more complicated due to needing to support the
   additional lock level

This commit unfortunately introduces some code that is very similar to the
code in lwlock.c, however the code is not equivalent enough to easily merge
it. The future wins that this commit makes possible seem worth the cost.

As of this commit nothing uses the new share-exclusive lock mode. It will be
documented and used in a future commit. It seemed too complicated to introduce
the lock-level in a separate commit.

TODO:
- Address FIXMEs

- Perhaps move the locking code into a buffer_locking.h or such? Needs to be
  inline functions for efficiency unfortunately.

- reflow some comments that I didn't reflow to make the diff more readable

Discussion: https://postgr.es/m/fvfmkr5kk4nyex56ejgxj3uzi63isfxovp2biecb4bspbjrze7@az2pljabhnff
---
 src/include/storage/buf_internals.h           |  55 +-
 src/include/storage/bufmgr.h                  |  20 +-
 src/include/storage/proc.h                    |   8 +-
 src/backend/postmaster/auxprocess.c           |   1 +
 src/backend/storage/buffer/buf_init.c         |   7 +-
 src/backend/storage/buffer/bufmgr.c           | 776 ++++++++++++++++--
 .../utils/activity/wait_event_names.txt       |   3 +
 7 files changed, 804 insertions(+), 66 deletions(-)

diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 28519ad2813..0a145d95024 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -23,6 +23,7 @@
 #include "storage/condition_variable.h"
 #include "storage/lwlock.h"
 #include "storage/procnumber.h"
+#include "storage/proclist_types.h"
 #include "storage/shmem.h"
 #include "storage/smgr.h"
 #include "storage/spin.h"
@@ -32,22 +33,29 @@
 /*
  * Buffer state is a single 64-bit variable where following data is combined.
  *
+ * State of the buffer itself:
  * - 18 bits refcount
  * - 4 bits usage count
  * - 10 bits of flags
  *
+ * State of the content lock:
+ * - 1 bit has_waiter
+ * - 1 bit release_ok
+ * - 1 bit lock state locked
+ * - 1 bit exclusively locked
+ * - 1 bit share exclusively locked
+ * - 18 bits share lock count
+ *
  * Combining these values allows to perform some operations without locking
  * the buffer header, by modifying them together with a CAS loop.
  *
- * NB: A future commit will use a significant portion of the remaining bits to
- * implement buffer locking as part of the state variable.
- *
  * The definition of buffer state components is below.
  */
 #define BUF_REFCOUNT_BITS 18
 #define BUF_USAGECOUNT_BITS 4
 #define BUF_FLAG_BITS 10
 
+/* FIXME: Also assert lock state size, just not yet sure how */
 StaticAssertDecl(BUF_REFCOUNT_BITS + BUF_USAGECOUNT_BITS + BUF_FLAG_BITS == 32,
 				 "parts of buffer state space need to equal 32");
 
@@ -69,7 +77,7 @@ StaticAssertDecl(BUF_REFCOUNT_BITS + BUF_USAGECOUNT_BITS + BUF_FLAG_BITS == 32,
 	((uint32)(((state) & BUF_USAGECOUNT_MASK) >> BUF_USAGECOUNT_SHIFT))
 
 /*
- * Flags for buffer descriptors
+ * Flags for buffer descriptor state
  *
  * Note: BM_TAG_VALID essentially means that there is a buffer hashtable
  * entry associated with the buffer's tag.
@@ -111,6 +119,20 @@ StaticAssertDecl(BM_MAX_USAGE_COUNT < (UINT64CONST(1) << BUF_USAGECOUNT_BITS),
 StaticAssertDecl(MAX_BACKENDS_BITS <= BUF_REFCOUNT_BITS,
 				 "MAX_BACKENDS_BITS needs to be <= BUF_REFCOUNT_BITS");
 
+
+/*
+ * Definitions related to buffer content locks
+ */
+#define BM_LOCK_HAS_WAITERS         (UINT64CONST(1) << 63)
+#define BM_LOCK_RELEASE_OK          (UINT64CONST(1) << 62)
+
+#define BM_LOCK_VAL_SHARED          (UINT64CONST(1) << 32)
+#define BM_LOCK_VAL_SHARE_EXCLUSIVE (UINT64CONST(1) << (32 + MAX_BACKENDS_BITS))
+#define BM_LOCK_VAL_EXCLUSIVE       (UINT64CONST(1) << (32 + 1 + MAX_BACKENDS_BITS))
+
+#define BM_LOCK_MASK                (((uint64)MAX_BACKENDS << 32) | BM_LOCK_VAL_SHARE_EXCLUSIVE | BM_LOCK_VAL_EXCLUSIVE)
+
+
 /*
  * Buffer tag identifies which disk block the buffer contains.
  *
@@ -253,9 +275,6 @@ BufMappingPartitionLockByIndex(uint32 index)
  * it is held.  However, existing buffer pins may be released while the buffer
  * header spinlock is held, using an atomic subtraction.
  *
- * The LWLock can take care of itself.  The buffer header lock is *not* used
- * to control access to the data in the buffer!
- *
  * If we have the buffer pinned, its tag can't change underneath us, so we can
  * examine the tag without locking the buffer header.  Also, in places we do
  * one-time reads of the flags without bothering to lock the buffer header;
@@ -268,6 +287,15 @@ BufMappingPartitionLockByIndex(uint32 index)
  * wait_backend_pgprocno and setting flag bit BM_PIN_COUNT_WAITER.  At present,
  * there can be only one such waiter per buffer.
  *
+ * The content of buffers is protected via the buffer content lock,
+ * implemented as part buffer state. Note that the buffer header lock is *not*
+ * used to control access to the data in the buffer! We used to use an LWLock
+ * to implement the content lock, but having a dedicated implementation of
+ * content locks allows to implement some otherwise hard things (e.g.
+ * race-freely checking if AIO is in progress before locking a buffer
+ * exclusively) and makes otherwise impossible optimizations possible
+ * (e.g. unlocking and unpinning a buffer in one atomic operation).
+ *
  * We use this same struct for local buffer headers, but the locks are not
  * used and not all of the flag bits are useful either. To avoid unnecessary
  * overhead, manipulations of the state field should be done without actual
@@ -309,7 +337,12 @@ typedef struct BufferDesc
 	int			wait_backend_pgprocno;
 
 	PgAioWaitRef io_wref;		/* set iff AIO is in progress */
-	LWLock		content_lock;	/* to lock access to buffer contents */
+
+	/*
+	 * List of PGPROCs waiting for the buffer content lock. Protected by the
+	 * buffer header spinlock.
+	 */
+	proclist_head lock_waiters;
 } BufferDesc;
 
 /*
@@ -396,12 +429,6 @@ BufferDescriptorGetIOCV(const BufferDesc *bdesc)
 	return &(BufferIOCVArray[bdesc->buf_id]).cv;
 }
 
-static inline LWLock *
-BufferDescriptorGetContentLock(const BufferDesc *bdesc)
-{
-	return (LWLock *) (&bdesc->content_lock);
-}
-
 /*
  * Functions for acquiring/releasing a shared buffer header's spinlock.  Do
  * not apply these to local buffers!
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 5fc3de20abc..8e442492d4d 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -204,6 +204,7 @@ typedef enum BufferLockMode
 {
 	BUFFER_LOCK_UNLOCK,
 	BUFFER_LOCK_SHARE,
+	BUFFER_LOCK_SHARE_EXCLUSIVE,
 	BUFFER_LOCK_EXCLUSIVE,
 } BufferLockMode;
 
@@ -302,7 +303,24 @@ extern void BufferGetTag(Buffer buffer, RelFileLocator *rlocator,
 extern void MarkBufferDirtyHint(Buffer buffer, bool buffer_std);
 
 extern void UnlockBuffers(void);
-extern void LockBuffer(Buffer buffer, BufferLockMode mode);
+extern void UnlockBuffer(Buffer buffer);
+extern void LockBufferInternal(Buffer buffer, BufferLockMode mode);
+
+/*
+ * Handling BUFFER_LOCK_UNLOCK in bufmgr.c leads to sufficiently worse branch
+ * prediction to impact performance. Therefore handle that switch here, were
+ * most of the time `mode` will be a constant and thus can be optimized out by
+ * the compiler.
+ */
+static inline void
+LockBuffer(Buffer buffer, BufferLockMode mode)
+{
+	if (mode == BUFFER_LOCK_UNLOCK)
+		UnlockBuffer(buffer);
+	else
+		LockBufferInternal(buffer, mode);
+}
+
 extern bool ConditionalLockBuffer(Buffer buffer);
 extern void LockBufferForCleanup(Buffer buffer);
 extern bool ConditionalLockBufferForCleanup(Buffer buffer);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index c6f5ebceefd..d1f6d314d57 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -242,7 +242,13 @@ struct PGPROC
 	 */
 	bool		recoveryConflictPending;
 
-	/* Info about LWLock the process is currently waiting for, if any. */
+	/*
+	 * Info about LWLock the process is currently waiting for, if any.
+	 *
+	 * This is currently used both for lwlocks and buffer content locks, which
+	 * is acceptable, although not pretty, because a backend can't wait for
+	 * both types of locks at the same time.
+	 */
 	uint8		lwWaiting;		/* see LWLockWaitState */
 	uint8		lwWaitMode;		/* lwlock mode being waited for */
 	proclist_node lwWaitLink;	/* position in LW lock wait list */
diff --git a/src/backend/postmaster/auxprocess.c b/src/backend/postmaster/auxprocess.c
index a6d3630398f..2012f959fd9 100644
--- a/src/backend/postmaster/auxprocess.c
+++ b/src/backend/postmaster/auxprocess.c
@@ -18,6 +18,7 @@
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/auxprocess.h"
+#include "storage/bufmgr.h"
 #include "storage/condition_variable.h"
 #include "storage/ipc.h"
 #include "storage/proc.h"
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index 25f71191ec3..f3224c793c4 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -17,6 +17,7 @@
 #include "storage/aio.h"
 #include "storage/buf_internals.h"
 #include "storage/bufmgr.h"
+#include "storage/proclist.h"
 
 BufferDescPadded *BufferDescriptors;
 char	   *BufferBlocks;
@@ -121,16 +122,14 @@ BufferManagerShmemInit(void)
 
 			ClearBufferTag(&buf->tag);
 
-			pg_atomic_init_u64(&buf->state, 0);
+			pg_atomic_init_u64(&buf->state, BM_LOCK_RELEASE_OK);
 			buf->wait_backend_pgprocno = INVALID_PROC_NUMBER;
 
 			buf->buf_id = i;
 
 			pgaio_wref_clear(&buf->io_wref);
 
-			LWLockInitialize(BufferDescriptorGetContentLock(buf),
-							 LWTRANCHE_BUFFER_CONTENT);
-
+			proclist_init(&buf->lock_waiters);
 			ConditionVariableInit(BufferDescriptorGetIOCV(buf));
 		}
 	}
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index b7ce4bafdea..da83b775d0b 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -58,6 +58,7 @@
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "storage/proc.h"
+#include "storage/proclist.h"
 #include "storage/read_stream.h"
 #include "storage/smgr.h"
 #include "storage/standby.h"
@@ -96,6 +97,12 @@ typedef struct PrivateRefCountData
 	 * How many times has the buffer been pinned by this backend.
 	 */
 	int32		refcount;
+
+	/*
+	 * Is the buffer locked by this backend? BUFFER_LOCK_UNLOCK indicates that
+	 * the buffer is not locked.
+	 */
+	BufferLockMode lockmode;
 } PrivateRefCountData;
 
 typedef struct PrivateRefCountEntry
@@ -334,6 +341,7 @@ ReservePrivateRefCountEntry(void)
 		victim_entry->buffer = InvalidBuffer;
 		memset(&victim_entry->data, 0, sizeof(victim_entry->data));
 		victim_entry->data.refcount = 0;
+		victim_entry->data.lockmode = BUFFER_LOCK_UNLOCK;
 
 		PrivateRefCountOverflowed++;
 	}
@@ -357,6 +365,7 @@ NewPrivateRefCountEntry(Buffer buffer)
 	PrivateRefCountArrayKeys[ReservedRefCountSlot] = buffer;
 	res->buffer = buffer;
 	res->data.refcount = 0;
+	res->data.lockmode = BUFFER_LOCK_UNLOCK;
 
 	PrivateRefcountEntryLast = ReservedRefCountSlot;
 
@@ -495,6 +504,7 @@ static void
 ForgetPrivateRefCountEntry(PrivateRefCountEntry *ref)
 {
 	Assert(ref->data.refcount == 0);
+	Assert(ref->data.lockmode == BUFFER_LOCK_UNLOCK);
 
 	if (ref >= &PrivateRefCountArray[0] &&
 		ref < &PrivateRefCountArray[REFCOUNT_ARRAY_ENTRIES])
@@ -604,6 +614,20 @@ static inline int buffertag_comparator(const BufferTag *ba, const BufferTag *bb)
 static inline int ckpt_buforder_comparator(const CkptSortItem *a, const CkptSortItem *b);
 static int	ts_ckpt_progress_comparator(Datum a, Datum b, void *arg);
 
+static void BufferLockAcquire(Buffer buffer, BufferDesc *buf_hdr, BufferLockMode mode);
+static void BufferLockUnlock(Buffer buffer, BufferDesc *buf_hdr);
+static bool BufferLockConditional(Buffer buffer, BufferDesc *buf_hdr, BufferLockMode mode);
+static bool BufferLockHeldByMeInMode(BufferDesc *buf_hdr, BufferLockMode mode);
+static bool BufferLockHeldByMe(BufferDesc *buf_hdr);
+static inline void BufferLockDisown(Buffer buffer, BufferDesc *buf_hdr);
+static inline int BufferLockDisownInternal(Buffer buffer, BufferDesc *buf_hdr);
+static inline bool BufferLockAttempt(BufferDesc *buf_hdr, BufferLockMode mode);
+static void BufferLockQueueSelf(BufferDesc *buf_hdr, BufferLockMode mode);
+static void BufferLockDequeueSelf(BufferDesc *buf_hdr);
+static void BufferLockWakeup(BufferDesc *buf_hdr, bool unlocked);
+static void BufferLockProcessRelease(BufferDesc *buf_hdr, BufferLockMode mode, uint64 lockstate);
+static inline uint64 BufferLockReleaseSub(BufferLockMode mode);
+
 
 /*
  * Implementation of PrefetchBuffer() for shared buffers.
@@ -2404,8 +2428,6 @@ again:
 	 */
 	if (buf_state & BM_DIRTY)
 	{
-		LWLock	   *content_lock;
-
 		Assert(buf_state & BM_TAG_VALID);
 		Assert(buf_state & BM_VALID);
 
@@ -2423,8 +2445,7 @@ again:
 		 * one just happens to be trying to split the page the first one got
 		 * from StrategyGetBuffer.)
 		 */
-		content_lock = BufferDescriptorGetContentLock(buf_hdr);
-		if (!LWLockConditionalAcquire(content_lock, LW_SHARED))
+		if (!BufferLockConditional(buf, buf_hdr, BUFFER_LOCK_SHARE))
 		{
 			/*
 			 * Someone else has locked the buffer, so give it up and loop back
@@ -2453,7 +2474,7 @@ again:
 			if (XLogNeedsFlush(lsn)
 				&& StrategyRejectBuffer(strategy, buf_hdr, from_ring))
 			{
-				LWLockRelease(content_lock);
+				LockBuffer(buf, BUFFER_LOCK_UNLOCK);
 				UnpinBuffer(buf_hdr);
 				goto again;
 			}
@@ -2461,7 +2482,7 @@ again:
 
 		/* OK, do the I/O */
 		FlushBuffer(buf_hdr, NULL, IOOBJECT_RELATION, io_context);
-		LWLockRelease(content_lock);
+		LockBuffer(buf, BUFFER_LOCK_UNLOCK);
 
 		ScheduleBufferTagForWriteback(&BackendWritebackContext, io_context,
 									  &buf_hdr->tag);
@@ -2903,7 +2924,7 @@ BufferIsLockedByMe(Buffer buffer)
 	else
 	{
 		bufHdr = GetBufferDescriptor(buffer - 1);
-		return LWLockHeldByMe(BufferDescriptorGetContentLock(bufHdr));
+		return BufferLockHeldByMe(bufHdr);
 	}
 }
 
@@ -2928,23 +2949,8 @@ BufferIsLockedByMeInMode(Buffer buffer, BufferLockMode mode)
 	}
 	else
 	{
-		LWLockMode	lw_mode;
-
-		switch (mode)
-		{
-			case BUFFER_LOCK_EXCLUSIVE:
-				lw_mode = LW_EXCLUSIVE;
-				break;
-			case BUFFER_LOCK_SHARE:
-				lw_mode = LW_SHARED;
-				break;
-			default:
-				pg_unreachable();
-		}
-
 		bufHdr = GetBufferDescriptor(buffer - 1);
-		return LWLockHeldByMeInMode(BufferDescriptorGetContentLock(bufHdr),
-									lw_mode);
+		return BufferLockHeldByMeInMode(bufHdr, mode);
 	}
 }
 
@@ -3331,7 +3337,7 @@ UnpinBufferNoOwner(BufferDesc *buf)
 		 * I'd better not still hold the buffer content lock. Can't use
 		 * BufferIsLockedByMe(), as that asserts the buffer is pinned.
 		 */
-		Assert(!LWLockHeldByMe(BufferDescriptorGetContentLock(buf)));
+		Assert(!BufferLockHeldByMe(buf));
 
 		/* decrement the shared reference count */
 		old_buf_state = pg_atomic_fetch_sub_u64(&buf->state, BUF_REFCOUNT_ONE);
@@ -4176,6 +4182,8 @@ static void
 AssertNotCatalogBufferLock(LWLock *lock, LWLockMode mode,
 						   void *unused_context)
 {
+	/* FIXME */
+#ifdef NOT_YET
 	BufferDesc *bufHdr;
 	BufferTag	tag;
 	Oid			relid;
@@ -4205,6 +4213,7 @@ AssertNotCatalogBufferLock(LWLock *lock, LWLockMode mode,
 		return;
 
 	Assert(!IsCatalogRelationOid(relid));
+#endif
 }
 #endif
 
@@ -4470,9 +4479,11 @@ static void
 FlushUnlockedBuffer(BufferDesc *buf, SMgrRelation reln,
 					IOObject io_object, IOContext io_context)
 {
-	LWLockAcquire(BufferDescriptorGetContentLock(buf), LW_SHARED);
+	Buffer		buffer = BufferDescriptorGetBuffer(buf);
+
+	BufferLockAcquire(buffer, buf, BUFFER_LOCK_SHARE);
 	FlushBuffer(buf, reln, IOOBJECT_RELATION, IOCONTEXT_NORMAL);
-	LWLockRelease(BufferDescriptorGetContentLock(buf));
+	BufferLockUnlock(buffer, buf);
 }
 
 /*
@@ -5615,9 +5626,10 @@ MarkBufferDirtyHint(Buffer buffer, bool buffer_std)
  *
  * Used to clean up after errors.
  *
- * Currently, we can expect that lwlock.c's LWLockReleaseAll() took care
- * of releasing buffer content locks per se; the only thing we need to deal
- * with here is clearing any PIN_COUNT request that was in progress.
+ * Currently, we can expect that resource owner cleanup, via
+ * ResOwnerReleaseBufferPin(), took care releasing buffer content locks per
+ * se; the only thing we need to deal with here is clearing any PIN_COUNT
+ * request that was in progress.
  */
 void
 UnlockBuffers(void)
@@ -5648,25 +5660,684 @@ UnlockBuffers(void)
 }
 
 /*
- * Acquire or release the content_lock for the buffer.
+ * Acquire the buffer content lock in the specified mode
+ *
+ * If the lock is not available, sleep until it is.
+ *
+ * Side effect: cancel/die interrupts are held off until lock release.
+ *
+ * This uses almost the same locking approach as lwlock.c's
+ * LWLockAcquire(). See documentation atop of lwlock.c for a more detailed
+ * discussion.
+ *
+ * The reason that this, and most of the other BufferLock* functions, get both
+ * the Buffer and BufferDesc* as parameters, is that looking up one from the
+ * other repeatedly shows up noticeably in profiles.
+ */
+static inline void
+BufferLockAcquire(Buffer buffer, BufferDesc *buf_hdr, BufferLockMode mode)
+{
+	PrivateRefCountEntry *entry;
+	int			extraWaits = 0;
+
+	/*
+	 * Get reference to the refcount entry before we hold the lock, it seems
+	 * better to do before holding the lock.
+	 */
+	entry = GetPrivateRefCountEntry(buffer, true);
+
+	/*
+	 * Lock out cancel/die interrupts until we exit the code section protected
+	 * by the content lock.  This ensures that interrupts will not interfere
+	 * with manipulations of data structures in shared memory.
+	 */
+	HOLD_INTERRUPTS();
+
+	for (;;)
+	{
+		bool		mustwait;
+		uint32		wait_event;
+
+		/*
+		 * Try to grab the lock the first time, we're not in the waitqueue
+		 * yet/anymore.
+		 */
+		mustwait = BufferLockAttempt(buf_hdr, mode);
+
+		if (likely(!mustwait))
+		{
+			break;
+		}
+
+		/*
+		 * 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 locker will see our queue entries when releasing since they
+		 * existed before we checked for the lock.
+		 */
+
+		/* add to the queue */
+		BufferLockQueueSelf(buf_hdr, mode);
+
+		/* we're now guaranteed to be woken up if necessary */
+		mustwait = BufferLockAttempt(buf_hdr, mode);
+
+		/* ok, grabbed the lock the second time round, need to undo queueing */
+		if (!mustwait)
+		{
+			BufferLockDequeueSelf(buf_hdr);
+			break;
+		}
+
+		switch (mode)
+		{
+			case BUFFER_LOCK_EXCLUSIVE:
+				wait_event = WAIT_EVENT_BUFFER_EXCLUSIVE;
+				break;
+			case BUFFER_LOCK_SHARE_EXCLUSIVE:
+				wait_event = WAIT_EVENT_BUFFER_SHARE_EXCLUSIVE;
+				break;
+			case BUFFER_LOCK_SHARE:
+				wait_event = WAIT_EVENT_BUFFER_SHARED;
+				break;
+			case BUFFER_LOCK_UNLOCK:
+				pg_unreachable();
+
+		}
+		pgstat_report_wait_start(wait_event);
+
+		/*
+		 * Wait until awakened.
+		 *
+		 * It is possible that we get awakened for a reason other than being
+		 * signaled by LWLockRelease.  If so, loop back and wait again.  Once
+		 * we've gotten the LWLock, re-increment the sema by the number of
+		 * additional signals received.
+		 */
+		for (;;)
+		{
+			PGSemaphoreLock(MyProc->sem);
+			if (MyProc->lwWaiting == LW_WS_NOT_WAITING)
+				break;
+			extraWaits++;
+		}
+
+		pgstat_report_wait_end();
+
+		/* Retrying, allow BufferLockRelease to release waiters again. */
+		pg_atomic_fetch_or_u64(&buf_hdr->state, BM_LOCK_RELEASE_OK);
+	}
+
+	/* Remember that we now hold this lock */
+	entry->data.lockmode = mode;
+
+	/*
+	 * Fix the process wait semaphore's count for any absorbed wakeups.
+	 */
+	while (unlikely(extraWaits-- > 0))
+		PGSemaphoreUnlock(MyProc->sem);
+}
+
+/*
+ * Release a previously acquired buffer content lock.
+ */
+static void
+BufferLockUnlock(Buffer buffer, BufferDesc *buf_hdr)
+{
+	BufferLockMode mode;
+	uint64		oldstate;
+	uint64		sub;
+
+	mode = BufferLockDisownInternal(buffer, buf_hdr);
+
+	/*
+	 * Release my hold on lock, after that it can immediately be acquired by
+	 * others, even if we still have to wakeup other waiters.
+	 */
+	sub = BufferLockReleaseSub(mode);
+
+	oldstate = pg_atomic_sub_fetch_u64(&buf_hdr->state, sub);
+
+	BufferLockProcessRelease(buf_hdr, mode, oldstate);
+
+	/*
+	 * Now okay to allow cancel/die interrupts.
+	 */
+	RESUME_INTERRUPTS();
+}
+
+
+/*
+ * Acquire the content lock for the buffer, but only if we don't have to wait.
+ */
+static bool
+BufferLockConditional(Buffer buffer, BufferDesc *buf_hdr, BufferLockMode mode)
+{
+	bool		mustwait;
+
+	/*
+	 * Lock out cancel/die interrupts until we exit the code section protected
+	 * by the content lock.  This ensures that interrupts will not interfere
+	 * with manipulations of data structures in shared memory.
+	 */
+	HOLD_INTERRUPTS();
+
+	/* Check for the lock */
+	mustwait = BufferLockAttempt(buf_hdr, mode);
+
+	if (mustwait)
+	{
+		/* Failed to get lock, so release interrupt holdoff */
+		RESUME_INTERRUPTS();
+	}
+	else
+	{
+		PrivateRefCountEntry *entry =
+			GetPrivateRefCountEntry(buffer, true);
+
+		entry->data.lockmode = mode;
+	}
+
+	return !mustwait;
+}
+
+/*
+ * Internal function that tries to atomically acquire the content lock in the
+ * passed in mode.
+ *
+ * This function will not block waiting for a lock to become free - that's the
+ * caller's job.
+ *
+ * Similar to LWLockAttemptLock().
+ */
+static inline bool
+BufferLockAttempt(BufferDesc *buf_hdr, BufferLockMode mode)
+{
+	uint64		old_state;
+
+	/*
+	 * Read once outside the loop, later iterations will get the newer value
+	 * via compare & exchange.
+	 */
+	old_state = pg_atomic_read_u64(&buf_hdr->state);
+
+	/* loop until we've determined whether we could acquire the lock or not */
+	while (true)
+	{
+		uint64		desired_state;
+		bool		lock_free;
+
+		desired_state = old_state;
+
+		if (mode == BUFFER_LOCK_EXCLUSIVE)
+		{
+			lock_free = (old_state & BM_LOCK_MASK) == 0;
+			if (lock_free)
+				desired_state += BM_LOCK_VAL_EXCLUSIVE;
+		}
+		else if (mode == BUFFER_LOCK_SHARE_EXCLUSIVE)
+		{
+			lock_free = (old_state & (BM_LOCK_VAL_EXCLUSIVE | BM_LOCK_VAL_SHARE_EXCLUSIVE)) == 0;
+			if (lock_free)
+				desired_state += BM_LOCK_VAL_SHARE_EXCLUSIVE;
+		}
+		else
+		{
+			lock_free = (old_state & BM_LOCK_VAL_EXCLUSIVE) == 0;
+			if (lock_free)
+				desired_state += BM_LOCK_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 (likely(pg_atomic_compare_exchange_u64(&buf_hdr->state,
+												  &old_state, desired_state)))
+		{
+			if (lock_free)
+			{
+				/* Great! Got the lock. */
+				return false;
+			}
+			else
+				return true;	/* somebody else has the lock */
+		}
+	}
+
+	pg_unreachable();
+}
+
+/*
+ * Add ourselves to the end of the content lock's wait queue.
+ */
+static void
+BufferLockQueueSelf(BufferDesc *buf_hdr, BufferLockMode 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");
+
+	if (MyProc->lwWaiting != LW_WS_NOT_WAITING)
+		elog(PANIC, "queueing for lock while waiting on another one");
+
+	LockBufHdr(buf_hdr);
+
+	/* setting the flag is protected by the spinlock */
+	pg_atomic_fetch_or_u64(&buf_hdr->state, BM_LOCK_HAS_WAITERS);
+
+	/*
+	 * FIXME: This is reusing the lwlock fields. That's not a correctness
+	 * issue, a backend can't wait for both an lwlock and a buffer content
+	 * lock at the same time. However, it seems pretty ugly, particularly
+	 * given that the field names have an lw* prefix. But duplicating the
+	 * fields also seems somewhat superfluous.
+	 */
+	MyProc->lwWaiting = LW_WS_WAITING;
+	MyProc->lwWaitMode = mode;
+
+	proclist_push_tail(&buf_hdr->lock_waiters, MyProcNumber, lwWaitLink);
+
+	/* Can release the mutex now */
+	UnlockBufHdr(buf_hdr);
+}
+
+/*
+ * 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.
+ */
+static void
+BufferLockDequeueSelf(BufferDesc *buf_hdr)
+{
+	bool		on_waitlist;
+
+	LockBufHdr(buf_hdr);
+
+	on_waitlist = MyProc->lwWaiting == LW_WS_WAITING;
+	if (on_waitlist)
+		proclist_delete(&buf_hdr->lock_waiters, MyProcNumber, lwWaitLink);
+
+	if (proclist_is_empty(&buf_hdr->lock_waiters) &&
+		(pg_atomic_read_u64(&buf_hdr->state) & BM_LOCK_HAS_WAITERS) != 0)
+	{
+		pg_atomic_fetch_and_u64(&buf_hdr->state, ~BM_LOCK_HAS_WAITERS);
+	}
+
+	/* XXX: combine with fetch_and above? */
+	UnlockBufHdr(buf_hdr);
+
+	/* clear waiting state again, nice for debugging */
+	if (on_waitlist)
+		MyProc->lwWaiting = LW_WS_NOT_WAITING;
+	else
+	{
+		int			extraWaits = 0;
+
+
+		/*
+		 * Somebody else dequeued us and has or will wake us up. Deal with the
+		 * superfluous absorption of a wakeup.
+		 */
+
+		/*
+		 * Reset RELEASE_OK flag if somebody woke us before we removed
+		 * ourselves - they'll have set it to false.
+		 */
+		pg_atomic_fetch_or_u64(&buf_hdr->state, BM_LOCK_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 (;;)
+		{
+			PGSemaphoreLock(MyProc->sem);
+			if (MyProc->lwWaiting == LW_WS_NOT_WAITING)
+				break;
+			extraWaits++;
+		}
+
+		/*
+		 * Fix the process wait semaphore's count for any absorbed wakeups.
+		 */
+		while (extraWaits-- > 0)
+			PGSemaphoreUnlock(MyProc->sem);
+	}
+}
+
+/*
+ * Stop treating lock as held by current backend.
+ *
+ * After calling this function it's the callers responsibility to ensure that
+ * the lock gets released, even in case of an error. This only is desirable if
+ * the lock is going to be released in a different process than the process
+ * that acquired it.
+ */
+static inline void
+BufferLockDisown(Buffer buffer, BufferDesc *buf_hdr)
+{
+	BufferLockDisownInternal(buffer, buf_hdr);
+	RESUME_INTERRUPTS();
+}
+
+/*
+ * Stop treating lock as held by current backend.
+ *
+ * This is the code that can be shared between actually releasing a lock
+ * (BufferLockUnlock()) and just not tracking ownership of the lock anymore
+ * without releasing the lock (BufferLockDisown()).
+ */
+static inline int
+BufferLockDisownInternal(Buffer buffer, BufferDesc *buf_hdr)
+{
+	BufferLockMode mode;
+	PrivateRefCountEntry *ref;
+
+	ref = GetPrivateRefCountEntry(buffer, false);
+	if (ref == NULL)
+		elog(ERROR, "lock %d is not held", buffer);
+	mode = ref->data.lockmode;
+	ref->data.lockmode = BUFFER_LOCK_UNLOCK;
+
+	return mode;
+}
+
+/*
+ * Wakeup all the lockers that currently have a chance to acquire the lock.
+ */
+static void
+BufferLockWakeup(BufferDesc *buf_hdr, bool unlocked)
+{
+	bool		new_release_ok;
+	bool		wake_exclusive = unlocked;
+	bool		wake_share_exclusive = true;
+	proclist_head wakeup;
+	proclist_mutable_iter iter;
+
+	proclist_init(&wakeup);
+
+	new_release_ok = true;
+
+	/* lock wait list while collecting backends to wake up */
+	LockBufHdr(buf_hdr);
+
+	proclist_foreach_modify(iter, &buf_hdr->lock_waiters, lwWaitLink)
+	{
+		PGPROC	   *waiter = GetPGProcByNumber(iter.cur);
+
+		if (!wake_exclusive && waiter->lwWaitMode == BUFFER_LOCK_EXCLUSIVE)
+			continue;
+
+		if (!wake_share_exclusive && waiter->lwWaitMode == BUFFER_LOCK_SHARE_EXCLUSIVE)
+			continue;
+
+		proclist_delete(&buf_hdr->lock_waiters, iter.cur, lwWaitLink);
+		proclist_push_tail(&wakeup, iter.cur, lwWaitLink);
+
+		/*
+		 * 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 share-exclusive/exclusive lock waiters after
+		 * waking a conflicting waiter.
+		 */
+		if (waiter->lwWaitMode == BUFFER_LOCK_EXCLUSIVE)
+		{
+			wake_exclusive = false;
+			wake_share_exclusive = false;
+		}
+		else if (waiter->lwWaitMode == BUFFER_LOCK_SHARE_EXCLUSIVE)
+			wake_share_exclusive = false;
+
+		/*
+		 * Signal that the process isn't on the wait list anymore. This allows
+		 * BufferLockDequeueSelf() to remove itself of the waitlist with a
+		 * proclist_delete(), rather than having to check if it has been
+		 * removed from the list.
+		 */
+		Assert(waiter->lwWaiting == LW_WS_WAITING);
+		waiter->lwWaiting = LW_WS_PENDING_WAKEUP;
+
+		/*
+		 * Once we've woken up an exclusive lock, there's no point in waking
+		 * up anybody else.
+		 */
+		if (waiter->lwWaitMode == BUFFER_LOCK_EXCLUSIVE)
+			break;
+	}
+
+	Assert(proclist_is_empty(&wakeup) || pg_atomic_read_u64(&buf_hdr->state) & BM_LOCK_HAS_WAITERS);
+
+	/* unset required flags, and release lock, in one fell swoop */
+	{
+		uint64		old_state;
+		uint64		desired_state;
+
+		old_state = pg_atomic_read_u64(&buf_hdr->state);
+		while (true)
+		{
+			desired_state = old_state;
+
+			/* compute desired flags */
+
+			if (new_release_ok)
+				desired_state |= BM_LOCK_RELEASE_OK;
+			else
+				desired_state &= ~BM_LOCK_RELEASE_OK;
+
+			if (proclist_is_empty(&buf_hdr->lock_waiters))
+				desired_state &= ~BM_LOCK_HAS_WAITERS;
+
+			desired_state &= ~BM_LOCKED;	/* release lock */
+
+			if (pg_atomic_compare_exchange_u64(&buf_hdr->state, &old_state,
+											   desired_state))
+				break;
+		}
+	}
+
+	/* Awaken any waiters I removed from the queue. */
+	proclist_foreach_modify(iter, &wakeup, lwWaitLink)
+	{
+		PGPROC	   *waiter = GetPGProcByNumber(iter.cur);
+
+		proclist_delete(&wakeup, iter.cur, 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 LWLockWaitListLock() when enqueuing for
+		 * another lock.
+		 */
+		pg_write_barrier();
+		waiter->lwWaiting = LW_WS_NOT_WAITING;
+		PGSemaphoreUnlock(waiter->sem);
+	}
+}
+
+/*
+ * Compute subtraction from buffer state for a release of a held lock in
+ * `mode`.
+ *
+ * This is separated from BufferLockUnlock() as we want to combine the lock
+ * release with other atomic operations when possible, leading to the lock
+ * release being done in multiple places.
+ */
+static inline uint64
+BufferLockReleaseSub(BufferLockMode mode)
+{
+
+	/*
+	 * Turns out that a switch() leads gcc to generate sufficiently worse code
+	 * for this to show up in profiles...
+	 */
+	if (mode == BUFFER_LOCK_EXCLUSIVE)
+		return BM_LOCK_VAL_EXCLUSIVE;
+	else if (mode == BUFFER_LOCK_SHARE_EXCLUSIVE)
+		return BM_LOCK_VAL_SHARE_EXCLUSIVE;
+	else
+	{
+		Assert(mode == BUFFER_LOCK_SHARE);
+		return BM_LOCK_VAL_SHARED;
+	}
+
+	return 0;
+}
+
+/*
+ * Handle work that needs to be done after releasing a lock that was held in
+ * `mode`, where `lockstate` is the result of the atomic operation modifying
+ * the state variable.
+ *
+ * This is separated from BufferLockUnlock() as we want to combine the lock
+ * release with other atomic operations when possible, leading to the lock
+ * release being done in multiple places.
+ */
+static void
+BufferLockProcessRelease(BufferDesc *buf_hdr, BufferLockMode mode, uint64 lockstate)
+{
+	bool		check_waiters = false;
+	bool		unlocked = false;
+
+	/* nobody else can have that kind of lock */
+	Assert(!(lockstate & BM_LOCK_VAL_EXCLUSIVE));
+
+	/*
+	 * We're still waiting for backends to get scheduled, don't wake them up
+	 * again.
+	 */
+	if ((lockstate & (BM_LOCK_HAS_WAITERS | BM_LOCK_RELEASE_OK)) ==
+		(BM_LOCK_HAS_WAITERS | BM_LOCK_RELEASE_OK))
+	{
+		if ((lockstate & BM_LOCK_MASK) == 0)
+		{
+			check_waiters = true;
+			unlocked = true;
+		}
+		else if (mode == BUFFER_LOCK_SHARE_EXCLUSIVE)
+		{
+			check_waiters = true;
+			unlocked = false;
+		}
+	}
+
+	/*
+	 * As waking up waiters requires the spinlock to be acquired, only do so
+	 * if necessary.
+	 */
+	if (check_waiters)
+		BufferLockWakeup(buf_hdr, unlocked);
+}
+
+/*
+ * BufferLockHeldByMeInMode - test whether my process holds the content lock
+ * in the specified mode
+ *
+ * This is meant as debug support only.
+ */
+static bool
+BufferLockHeldByMeInMode(BufferDesc *buf_hdr, BufferLockMode mode)
+{
+	PrivateRefCountEntry *entry =
+		GetPrivateRefCountEntry(BufferDescriptorGetBuffer(buf_hdr), false);
+
+	if (!entry)
+		return false;
+	else
+		return entry->data.lockmode == mode;
+
+}
+
+/*
+ * BufferLockHeldByMe - test whether my process holds the content lock in any
+ * mode
+ *
+ * This is meant as debug support only.
+ */
+static bool
+BufferLockHeldByMe(BufferDesc *buf_hdr)
+{
+	PrivateRefCountEntry *entry =
+		GetPrivateRefCountEntry(BufferDescriptorGetBuffer(buf_hdr), false);
+
+	if (!entry)
+		return false;
+	else
+		return entry->data.lockmode != BUFFER_LOCK_UNLOCK;
+}
+
+/*
+ * Release the content lock for the buffer.
+ */
+void
+UnlockBuffer(Buffer buffer)
+{
+	BufferDesc *buf_hdr;
+
+	Assert(BufferIsPinned(buffer));
+	if (BufferIsLocal(buffer))
+		return;					/* local buffers need no lock */
+
+	buf_hdr = GetBufferDescriptor(buffer - 1);
+	BufferLockUnlock(buffer, buf_hdr);
+}
+
+/*
+ * Acquire the content_lock for the buffer.
  */
 void
-LockBuffer(Buffer buffer, BufferLockMode mode)
+LockBufferInternal(Buffer buffer, BufferLockMode mode)
 {
-	BufferDesc *buf;
+	BufferDesc *buf_hdr;
+
+	/*
+	 * We can't wait if we haven't got a PGPROC.  This should only occur
+	 * during bootstrap or shared memory initialization.  Put an Assert here
+	 * to catch unsafe coding practices.
+	 */
+	Assert(!(MyProc == NULL && IsUnderPostmaster));
+
+	/* handled in LockBuffer() wrapper */
+	Assert(mode != BUFFER_LOCK_UNLOCK);
 
 	Assert(BufferIsPinned(buffer));
 	if (BufferIsLocal(buffer))
 		return;					/* local buffers need no lock */
 
-	buf = GetBufferDescriptor(buffer - 1);
+	buf_hdr = GetBufferDescriptor(buffer - 1);
 
-	if (mode == BUFFER_LOCK_UNLOCK)
-		LWLockRelease(BufferDescriptorGetContentLock(buf));
-	else if (mode == BUFFER_LOCK_SHARE)
-		LWLockAcquire(BufferDescriptorGetContentLock(buf), LW_SHARED);
+	if (mode == BUFFER_LOCK_SHARE)
+		BufferLockAcquire(buffer, buf_hdr, BUFFER_LOCK_SHARE);
+	else if (mode == BUFFER_LOCK_SHARE_EXCLUSIVE)
+		BufferLockAcquire(buffer, buf_hdr, BUFFER_LOCK_SHARE_EXCLUSIVE);
 	else if (mode == BUFFER_LOCK_EXCLUSIVE)
-		LWLockAcquire(BufferDescriptorGetContentLock(buf), LW_EXCLUSIVE);
+		BufferLockAcquire(buffer, buf_hdr, BUFFER_LOCK_EXCLUSIVE);
 	else
 		elog(ERROR, "unrecognized buffer lock mode: %d", mode);
 }
@@ -5687,8 +6358,7 @@ ConditionalLockBuffer(Buffer buffer)
 
 	buf = GetBufferDescriptor(buffer - 1);
 
-	return LWLockConditionalAcquire(BufferDescriptorGetContentLock(buf),
-									LW_EXCLUSIVE);
+	return BufferLockConditional(buffer, buf, BUFFER_LOCK_EXCLUSIVE);
 }
 
 /*
@@ -6625,7 +7295,25 @@ ResOwnerReleaseBufferPin(Datum res)
 	if (BufferIsLocal(buffer))
 		UnpinLocalBufferNoOwner(buffer);
 	else
+	{
+		PrivateRefCountEntry *ref;
+
+		ref = GetPrivateRefCountEntry(buffer, false);
+
+		/*
+		 * If the buffer was locked at the time of the resowner release,
+		 * release the lock now. This should only happen after errors.
+		 */
+		if (ref->data.lockmode != BUFFER_LOCK_UNLOCK)
+		{
+			BufferDesc *buf = GetBufferDescriptor(buffer - 1);
+
+			HOLD_INTERRUPTS();	/* match the upcoming RESUME_INTERRUPTS */
+			BufferLockUnlock(buffer, buf);
+		}
+
 		UnpinBufferNoOwner(GetBufferDescriptor(buffer - 1));
+	}
 }
 
 static char *
@@ -6927,16 +7615,12 @@ buffer_stage_common(PgAioHandle *ioh, bool is_write, bool is_temp)
 		 */
 		if (is_write && !is_temp)
 		{
-			LWLock	   *content_lock;
-
-			content_lock = BufferDescriptorGetContentLock(buf_hdr);
-
-			Assert(LWLockHeldByMe(content_lock));
+			Assert(BufferLockHeldByMe(buf_hdr));
 
 			/*
 			 * Lock is now owned by AIO subsystem.
 			 */
-			LWLockDisown(content_lock);
+			BufferLockDisown(buffer, buf_hdr);
 		}
 
 		/*
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index 1e5e368a5dc..39ae7cdf856 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -285,6 +285,9 @@ ABI_compatibility:
 Section: ClassName - WaitEventBuffer
 
 BUFFER_CLEANUP	"Waiting to acquire an exclusive pin on a buffer. Buffer pin waits can be protracted if another process holds an open cursor that last read data from the buffer in question."
+BUFFER_SHARED	"Waiting to acquire shared lock on a buffer."
+BUFFER_SHARE_EXCLUSIVE	"Waiting to acquire share exclusive lock on a buffer."
+BUFFER_EXCLUSIVE	"Waiting to acquire exclusive lock on a buffer."
 
 ABI_compatibility:
 
-- 
2.48.1.76.g4e746b1a31.dirty

