Get rid of WALBufMappingLock

Started by Yura Sokolov12 months ago47 messages
#1Yura Sokolov
y.sokolov@postgrespro.ru
2 attachment(s)

Good day, hackers.

During discussion of Increasing NUM_XLOGINSERT_LOCKS [1]/messages/by-id/flat/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru, Andres Freund
used benchmark which creates WAL records very intensively. While I this
it is not completely fair (1MB log records are really rare), it pushed
me to analyze write-side waiting of XLog machinery.

First I tried to optimize WaitXLogInsertionsToFinish, but without great
success (yet).

While profiling, I found a lot of time is spend in the memory clearing
under global WALBufMappingLock:

MemSet((char *) NewPage, 0, XLOG_BLCKSZ);

It is obvious scalability bottleneck.

So "challenge was accepted".

Certainly, backend should initialize pages without exclusive lock. But
which way to ensure pages were initialized? In other words, how to
ensure XLogCtl->InitializedUpTo is correct.

I've tried to play around WALBufMappingLock with holding it for a short
time and spinning on XLogCtl->xlblocks[nextidx]. But in the end I found
WALBufMappingLock is useless at all.

Instead of holding lock, it is better to allow backends to cooperate:
- I bound ConditionVariable to each xlblocks entry,
- every backend now checks every required block pointed by
InitializedUpto was successfully initialized or sleeps on its condvar,
- when backend sure block is initialized, it tries to update
InitializedUpTo with conditional variable.

Andres's benchmark looks like:

c=100 && install/bin/psql -c checkpoint -c 'select pg_switch_wal()'
postgres && install/bin/pgbench -n -M prepared -c$c -j$c -f <(echo
"SELECT pg_logical_emit_message(true, 'test', repeat('0',
1024*1024));";) -P1 -T45 postgres

So, it generate 1M records as fast as possible for 45 seconds.

Test machine is Ryzen 5825U (8c/16th) limited to 2GHz.
Config:

max_connections = 1000
shared_buffers = 1024MB
fsync = off
wal_sync_method = fdatasync
full_page_writes = off
wal_buffers = 1024MB
checkpoint_timeout = 1d

Results are: "average for 45 sec" /"1 second max outlier"

Results for master @ d3d098316913 :
25 clients: 2908 /3230
50 clients: 2759 /3130
100 clients: 2641 /2933
200 clients: 2419 /2707
400 clients: 1928 /2377
800 clients: 1689 /2266

With v0-0001-Get-rid-of-WALBufMappingLock.patch :
25 clients: 3103 /3583
50 clients: 3183 /3706
100 clients: 3106 /3559
200 clients: 2902 /3427
400 clients: 2303 /2717
800 clients: 1925 /2329

Combined with v0-0002-several-attempts-to-lock-WALInsertLocks.patch

No WALBufMappingLock + attempts on XLogInsertLock:
25 clients: 3518 /3750
50 clients: 3355 /3548
100 clients: 3226 /3460
200 clients: 3092 /3299
400 clients: 2575 /2801
800 clients: 1946 /2341

This results are with untouched NUM_XLOGINSERT_LOCKS == 8.

[1]: /messages/by-id/flat/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru
/messages/by-id/flat/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru

PS.
Increasing NUM_XLOGINSERT_LOCKS to 64 gives:
25 clients: 3457 /3624
50 clients: 3215 /3500
100 clients: 2750 /3000
200 clients: 2535 /2729
400 clients: 2163 /2400
800 clients: 1700 /2060

While doing this on master:
25 clients 2645 /2953
50 clients: 2562 /2968
100 clients: 2364 /2756
200 clients: 2266 /2564
400 clients: 1868 /2228
800 clients: 1527 /2133

So, patched version with increased NUM_XLOGINSERT_LOCKS looks no worse
than unpatched without increasing num of locks.

-------
regards
Yura Sokolov aka funny-falcon

Attachments:

v0-0001-Get-rid-of-WALBufMappingLock.patchtext/x-patch; charset=UTF-8; name=v0-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 236b69ae1f524c7e8488da7244966e631324a0e3 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v0 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers usign
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 186 +++++++++++-------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 122 insertions(+), 68 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index bf3dbda901d..0cc2273fef1 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,10 +302,7 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
+ * (TODO: describe AdvanceXLInsertBuffer)
  *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
@@ -444,6 +441,12 @@ typedef struct XLogCtlInsert
 	WALInsertLockPadded *WALInsertLocks;
 } XLogCtlInsert;
 
+typedef struct XLBlocks
+{
+	pg_atomic_uint64 bound;
+	ConditionVariable condvar;
+}			XLBlocks;
+
 /*
  * Total shared-memory state for XLOG.
  */
@@ -472,25 +475,35 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities, perhaps
+	 * waiting on conditional variable bound to buffer.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
-	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
+	XLBlocks   *xlblocks;		/* 1st byte ptr-s + XLOG_BLCKSZ (and condvar
+								 * for) */
 	int			XLogCacheBlck;	/* highest allocated xlog buffer index */
 
 	/*
@@ -810,8 +823,8 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that ahead of insertions
 	 * to avoid that from happening in the critical path.
 	 *
 	 *----------
@@ -1671,7 +1684,7 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 	expectedEndPtr = ptr;
 	expectedEndPtr += XLOG_BLCKSZ - ptr % XLOG_BLCKSZ;
 
-	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 	if (expectedEndPtr != endptr)
 	{
 		XLogRecPtr	initializedUpto;
@@ -1702,11 +1715,11 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 		WALInsertLockUpdateInsertingAt(initializedUpto);
 
 		AdvanceXLInsertBuffer(ptr, tli, false);
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 
 		if (expectedEndPtr != endptr)
-			elog(PANIC, "could not find WAL buffer for %X/%X",
-				 LSN_FORMAT_ARGS(ptr));
+			elog(PANIC, "could not find WAL buffer for %X/%X: %X/%X != %X/%X",
+				 LSN_FORMAT_ARGS(ptr), LSN_FORMAT_ARGS(expectedEndPtr), LSN_FORMAT_ARGS(endptr));
 	}
 	else
 	{
@@ -1803,7 +1816,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * First verification step: check that the correct page is present in
 		 * the WAL buffers.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1835,7 +1848,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * Second verification step: check that the page we read from wasn't
 		 * evicted while we were copying the data.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1991,32 +2004,46 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
+	XLogRecPtr	InitializedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * Try to initialize pages we need in WAL buffer.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
-
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2058,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2079,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2086,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2138,12 +2161,38 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 */
 		pg_write_barrier();
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, NewPageEndPtr);
+		ConditionVariableBroadcast(&XLogCtl->xlblocks[nextidx].condvar);
+
+		InitializedPtr = pg_atomic_read_u64(&XLogCtl->InitializedUpTo);
+		if (InitializedPtr == NewPageBeginPtr)
+			pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &InitializedPtr, NewPageEndPtr);
 
 		npages++;
+
+		ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	}
+
+	/*
+	 * Now we need to advance XLogCtl->InitializedUpTo. All backends will do
+	 * this job cooperatively: it is better than waiting on single lock.
+	 */
+	InitializedPtr = pg_atomic_read_u64(&XLogCtl->InitializedUpTo);
+	while (upto >= InitializedPtr)
+	{
+		nextidx = XLogRecPtrToBufIdx(InitializedPtr);
+
+		/*
+		 * InitializedPtr could fall into past, so we don't use check for
+		 * equality here
+		 */
+		while (InitializedPtr + XLOG_BLCKSZ > pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound))
+			ConditionVariableSleep(&XLogCtl->xlblocks[nextidx].condvar, WAIT_EVENT_WAL_BUFFER_INIT);
+		ConditionVariableCancelSleep();
+
+		if (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &InitializedPtr, InitializedPtr + XLOG_BLCKSZ))
+			InitializedPtr += XLOG_BLCKSZ;
 	}
-	LWLockRelease(WALBufMappingLock);
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -2356,7 +2405,7 @@ XLogWrite(XLogwrtRqst WriteRqst, TimeLineID tli, bool flexible)
 		 * if we're passed a bogus WriteRqst.Write that is past the end of the
 		 * last page that's been initialized by AdvanceXLInsertBuffer.
 		 */
-		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx]);
+		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx].bound);
 
 		if (LogwrtResult.Write >= EndPtr)
 			elog(PANIC, "xlog write request %X/%X is past end of log %X/%X",
@@ -4899,7 +4948,7 @@ XLOGShmemSize(void)
 	/* WAL insertion locks, plus alignment */
 	size = add_size(size, mul_size(sizeof(WALInsertLockPadded), NUM_XLOGINSERT_LOCKS + 1));
 	/* xlblocks array */
-	size = add_size(size, mul_size(sizeof(pg_atomic_uint64), XLOGbuffers));
+	size = add_size(size, mul_size(sizeof(XLBlocks), XLOGbuffers));
 	/* extra alignment padding for XLOG I/O buffers */
 	size = add_size(size, Max(XLOG_BLCKSZ, PG_IO_ALIGN_SIZE));
 	/* and the buffers themselves */
@@ -4977,12 +5026,13 @@ XLOGShmemInit(void)
 	 * needed here.
 	 */
 	allocptr = ((char *) XLogCtl) + sizeof(XLogCtlData);
-	XLogCtl->xlblocks = (pg_atomic_uint64 *) allocptr;
-	allocptr += sizeof(pg_atomic_uint64) * XLOGbuffers;
+	XLogCtl->xlblocks = (XLBlocks *) allocptr;
+	allocptr += sizeof(XLBlocks) * XLOGbuffers;
 
 	for (i = 0; i < XLOGbuffers; i++)
 	{
-		pg_atomic_init_u64(&XLogCtl->xlblocks[i], InvalidXLogRecPtr);
+		pg_atomic_init_u64(&XLogCtl->xlblocks[i].bound, InvalidXLogRecPtr);
+		ConditionVariableInit(&XLogCtl->xlblocks[i].condvar);
 	}
 
 	/* WAL insertion locks. Ensure they're aligned to the full padded size */
@@ -5023,6 +5073,9 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
 }
 
 /*
@@ -6041,8 +6094,8 @@ StartupXLOG(void)
 		memcpy(page, endOfRecoveryInfo->lastPage, len);
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx].bound, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6051,8 +6104,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index 0b53cba807d..ad2a1aa4ca1 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -154,6 +154,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -309,7 +310,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.43.0

v0-0002-several-attempts-to-lock-WALInsertLocks.patchtext/x-patch; charset=UTF-8; name=v0-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From bc52c5ceec65bee210982c3df6f292f36db8ddd5 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v0 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 47 ++++++++++++++++++-------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 0cc2273fef1..c7cd8b2fa38 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1383,8 +1384,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
-
+	int attempts = 2;
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
 	 * the one we used last time.  If the system isn't particularly busy, it's
@@ -1396,29 +1396,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32 rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1; /* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+	/*
+	 * If we couldn't get the lock immediately, try another lock next
+	 * time.  On a system with more insertion locks than concurrent
+	 * inserters, this causes all the inserters to eventually migrate to a
+	 * lock that no-one else is using.  On a system with more inserters
+	 * than locks, it still helps to distribute the inserters evenly
+	 * across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.43.0

#2Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Yura Sokolov (#1)
Re: Get rid of WALBufMappingLock

19.01.2025 03:11, Yura Sokolov пишет:

Good day, hackers.

During discussion of Increasing NUM_XLOGINSERT_LOCKS [1], Andres Freund
used benchmark which creates WAL records very intensively. While I this
it is not completely fair (1MB log records are really rare), it pushed
me to analyze write-side waiting of XLog machinery.

First I tried to optimize WaitXLogInsertionsToFinish, but without great
success (yet).

While profiling, I found a lot of time is spend in the memory clearing
under global WALBufMappingLock:

    MemSet((char *) NewPage, 0, XLOG_BLCKSZ);

It is obvious scalability bottleneck.

So "challenge was accepted".

Certainly, backend should initialize pages without exclusive lock. But
which way to ensure pages were initialized? In other words, how to
ensure XLogCtl->InitializedUpTo is correct.

I've tried to play around WALBufMappingLock with holding it for a short
time and spinning on XLogCtl->xlblocks[nextidx]. But in the end I found
WALBufMappingLock is useless at all.

Instead of holding lock, it is better to allow backends to cooperate:
- I bound ConditionVariable to each xlblocks entry,
- every backend now checks every required block pointed by
InitializedUpto was successfully initialized or sleeps on its condvar,
- when backend sure block is initialized, it tries to update
InitializedUpTo with conditional variable.

Andres's benchmark looks like:

  c=100 && install/bin/psql -c checkpoint -c 'select pg_switch_wal()'
postgres && install/bin/pgbench -n -M prepared -c$c -j$c -f <(echo
"SELECT pg_logical_emit_message(true, 'test', repeat('0',
1024*1024));";) -P1 -T45 postgres

So, it generate 1M records as fast as possible for 45 seconds.

Test machine is Ryzen 5825U (8c/16th) limited to 2GHz.
Config:

  max_connections = 1000
  shared_buffers = 1024MB
  fsync = off
  wal_sync_method = fdatasync
  full_page_writes = off
  wal_buffers = 1024MB
  checkpoint_timeout = 1d

Results are: "average for 45 sec"  /"1 second max outlier"

Results for master @ d3d098316913 :
  25  clients: 2908  /3230
  50  clients: 2759  /3130
  100 clients: 2641  /2933
  200 clients: 2419  /2707
  400 clients: 1928  /2377
  800 clients: 1689  /2266

With v0-0001-Get-rid-of-WALBufMappingLock.patch :
  25  clients: 3103  /3583
  50  clients: 3183  /3706
  100 clients: 3106  /3559
  200 clients: 2902  /3427
  400 clients: 2303  /2717
  800 clients: 1925  /2329

Combined with v0-0002-several-attempts-to-lock-WALInsertLocks.patch

No WALBufMappingLock + attempts on XLogInsertLock:
  25  clients: 3518  /3750
  50  clients: 3355  /3548
  100 clients: 3226  /3460
  200 clients: 3092  /3299
  400 clients: 2575  /2801
  800 clients: 1946  /2341

This results are with untouched NUM_XLOGINSERT_LOCKS == 8.

[1] /messages/by-id/flat/3b11fdc2-9793-403d-
b3d4-67ff9a00d447%40postgrespro.ru

PS.
Increasing NUM_XLOGINSERT_LOCKS to 64 gives:
  25  clients: 3457  /3624
  50  clients: 3215  /3500
  100 clients: 2750  /3000
  200 clients: 2535  /2729
  400 clients: 2163  /2400
  800 clients: 1700  /2060

While doing this on master:
  25  clients  2645  /2953
  50  clients: 2562  /2968
  100 clients: 2364  /2756
  200 clients: 2266  /2564
  400 clients: 1868  /2228
  800 clients: 1527  /2133

So, patched version with increased NUM_XLOGINSERT_LOCKS looks no worse
than unpatched without increasing num of locks.

I'm too brave... or too sleepy (it's 3:30am)...
But I took the risk of sending a patch to commitfest:
https://commitfest.postgresql.org/52/5511/

------
regards
Yura Sokolov aka funny-falcon

#3Alexander Korotkov
aekorotkov@gmail.com
In reply to: Yura Sokolov (#1)
2 attachment(s)
Re: Get rid of WALBufMappingLock

Hi!

On Sun, Jan 19, 2025 at 2:11 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

During discussion of Increasing NUM_XLOGINSERT_LOCKS [1], Andres Freund
used benchmark which creates WAL records very intensively. While I this
it is not completely fair (1MB log records are really rare), it pushed
me to analyze write-side waiting of XLog machinery.

First I tried to optimize WaitXLogInsertionsToFinish, but without great
success (yet).

While profiling, I found a lot of time is spend in the memory clearing
under global WALBufMappingLock:

MemSet((char *) NewPage, 0, XLOG_BLCKSZ);

It is obvious scalability bottleneck.

So "challenge was accepted".

Certainly, backend should initialize pages without exclusive lock. But
which way to ensure pages were initialized? In other words, how to
ensure XLogCtl->InitializedUpTo is correct.

I've tried to play around WALBufMappingLock with holding it for a short
time and spinning on XLogCtl->xlblocks[nextidx]. But in the end I found
WALBufMappingLock is useless at all.

Instead of holding lock, it is better to allow backends to cooperate:
- I bound ConditionVariable to each xlblocks entry,
- every backend now checks every required block pointed by
InitializedUpto was successfully initialized or sleeps on its condvar,
- when backend sure block is initialized, it tries to update
InitializedUpTo with conditional variable.

Looks reasonable for me, but having ConditionVariable per xlog buffer
seems overkill for me. Find an attached revision, where I've
implemented advancing InitializedUpTo without ConditionVariable.
After initialization of each buffer there is attempt to do CAS for
InitializedUpTo in a loop. So, multiple processes will try to advance
InitializedUpTo, they could hijack initiative from each other, but
there is always a leader which will finish the work.

There is only one ConditionVariable to wait for InitializedUpTo being advanced.

I didn't benchmark my version, just checked that tests passed.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v1-0002-several-attempts-to-lock-WALInsertLocks.patchapplication/octet-stream; name=v1-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From b94d681fa0b39f265e133c09d6df8595a9b51f94 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v1 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 47 ++++++++++++++++++-------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c70ac26026e..33d26fc37cf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1384,8 +1385,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
-
+	int attempts = 2;
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
 	 * the one we used last time.  If the system isn't particularly busy, it's
@@ -1397,29 +1397,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32 rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1; /* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+	/*
+	 * If we couldn't get the lock immediately, try another lock next
+	 * time.  On a system with more insertion locks than concurrent
+	 * inserters, this causes all the inserters to eventually migrate to a
+	 * lock that no-one else is using.  On a system with more inserters
+	 * than locks, it still helps to distribute the inserters evenly
+	 * across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.39.5 (Apple Git-154)

v1-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v1-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 683c008b71d173dea34810ba445bba6f51f52af6 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v1 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers usign
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 176 +++++++++++-------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 111 insertions(+), 69 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 9c270e7d466..c70ac26026e 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,10 +302,7 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
+ * (TODO: describe AdvanceXLInsertBuffer)
  *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
@@ -444,6 +441,11 @@ typedef struct XLogCtlInsert
 	WALInsertLockPadded *WALInsertLocks;
 } XLogCtlInsert;
 
+typedef struct XLBlocks
+{
+	pg_atomic_uint64 bound;
+}			XLBlocks;
+
 /*
  * Total shared-memory state for XLOG.
  */
@@ -472,25 +474,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities, perhaps
+	 * waiting on conditional variable bound to buffer.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
+
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
-	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
+	XLBlocks   *xlblocks;		/* 1st byte ptr-s + XLOG_BLCKSZ (and condvar
+								 * for) */
 	int			XLogCacheBlck;	/* highest allocated xlog buffer index */
 
 	/*
@@ -810,8 +824,8 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that ahead of insertions
 	 * to avoid that from happening in the critical path.
 	 *
 	 *----------
@@ -1671,7 +1685,7 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 	expectedEndPtr = ptr;
 	expectedEndPtr += XLOG_BLCKSZ - ptr % XLOG_BLCKSZ;
 
-	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 	if (expectedEndPtr != endptr)
 	{
 		XLogRecPtr	initializedUpto;
@@ -1702,11 +1716,11 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 		WALInsertLockUpdateInsertingAt(initializedUpto);
 
 		AdvanceXLInsertBuffer(ptr, tli, false);
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 
 		if (expectedEndPtr != endptr)
-			elog(PANIC, "could not find WAL buffer for %X/%X",
-				 LSN_FORMAT_ARGS(ptr));
+			elog(PANIC, "could not find WAL buffer for %X/%X: %X/%X != %X/%X",
+				 LSN_FORMAT_ARGS(ptr), LSN_FORMAT_ARGS(expectedEndPtr), LSN_FORMAT_ARGS(endptr));
 	}
 	else
 	{
@@ -1803,7 +1817,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * First verification step: check that the correct page is present in
 		 * the WAL buffers.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1835,7 +1849,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * Second verification step: check that the page we read from wasn't
 		 * evicted while we were copying the data.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1991,32 +2005,45 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * Try to initialize pages we need in WAL buffer.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
-
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2058,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2079,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2086,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2138,12 +2161,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 */
 		pg_write_barrier();
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, NewPageEndPtr);
+		//ConditionVariableBroadcast(&XLogCtl->xlblocks[nextidx].condvar);
 
-		npages++;
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound) != NewPageEndPtr)
+			{
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -2356,7 +2393,7 @@ XLogWrite(XLogwrtRqst WriteRqst, TimeLineID tli, bool flexible)
 		 * if we're passed a bogus WriteRqst.Write that is past the end of the
 		 * last page that's been initialized by AdvanceXLInsertBuffer.
 		 */
-		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx]);
+		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx].bound);
 
 		if (LogwrtResult.Write >= EndPtr)
 			elog(PANIC, "xlog write request %X/%X is past end of log %X/%X",
@@ -4920,7 +4957,7 @@ XLOGShmemSize(void)
 	/* WAL insertion locks, plus alignment */
 	size = add_size(size, mul_size(sizeof(WALInsertLockPadded), NUM_XLOGINSERT_LOCKS + 1));
 	/* xlblocks array */
-	size = add_size(size, mul_size(sizeof(pg_atomic_uint64), XLOGbuffers));
+	size = add_size(size, mul_size(sizeof(XLBlocks), XLOGbuffers));
 	/* extra alignment padding for XLOG I/O buffers */
 	size = add_size(size, Max(XLOG_BLCKSZ, PG_IO_ALIGN_SIZE));
 	/* and the buffers themselves */
@@ -4998,12 +5035,12 @@ XLOGShmemInit(void)
 	 * needed here.
 	 */
 	allocptr = ((char *) XLogCtl) + sizeof(XLogCtlData);
-	XLogCtl->xlblocks = (pg_atomic_uint64 *) allocptr;
-	allocptr += sizeof(pg_atomic_uint64) * XLOGbuffers;
+	XLogCtl->xlblocks = (XLBlocks *) allocptr;
+	allocptr += sizeof(XLBlocks) * XLOGbuffers;
 
 	for (i = 0; i < XLOGbuffers; i++)
 	{
-		pg_atomic_init_u64(&XLogCtl->xlblocks[i], InvalidXLogRecPtr);
+		pg_atomic_init_u64(&XLogCtl->xlblocks[i].bound, InvalidXLogRecPtr);
 	}
 
 	/* WAL insertion locks. Ensure they're aligned to the full padded size */
@@ -5044,6 +5081,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6062,8 +6103,8 @@ StartupXLOG(void)
 		memcpy(page, endOfRecoveryInfo->lastPage, len);
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx].bound, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6113,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

#4Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Alexander Korotkov (#3)
2 attachment(s)
Re: Get rid of WALBufMappingLock

07.02.2025 01:26, Alexander Korotkov пишет:

Hi!

On Sun, Jan 19, 2025 at 2:11 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

During discussion of Increasing NUM_XLOGINSERT_LOCKS [1], Andres Freund
used benchmark which creates WAL records very intensively. While I this
it is not completely fair (1MB log records are really rare), it pushed
me to analyze write-side waiting of XLog machinery.

First I tried to optimize WaitXLogInsertionsToFinish, but without great
success (yet).

While profiling, I found a lot of time is spend in the memory clearing
under global WALBufMappingLock:

MemSet((char *) NewPage, 0, XLOG_BLCKSZ);

It is obvious scalability bottleneck.

So "challenge was accepted".

Certainly, backend should initialize pages without exclusive lock. But
which way to ensure pages were initialized? In other words, how to
ensure XLogCtl->InitializedUpTo is correct.

I've tried to play around WALBufMappingLock with holding it for a short
time and spinning on XLogCtl->xlblocks[nextidx]. But in the end I found
WALBufMappingLock is useless at all.

Instead of holding lock, it is better to allow backends to cooperate:
- I bound ConditionVariable to each xlblocks entry,
- every backend now checks every required block pointed by
InitializedUpto was successfully initialized or sleeps on its condvar,
- when backend sure block is initialized, it tries to update
InitializedUpTo with conditional variable.

Looks reasonable for me, but having ConditionVariable per xlog buffer
seems overkill for me. Find an attached revision, where I've
implemented advancing InitializedUpTo without ConditionVariable.
After initialization of each buffer there is attempt to do CAS for
InitializedUpTo in a loop. So, multiple processes will try to advance
InitializedUpTo, they could hijack initiative from each other, but
there is always a leader which will finish the work.

There is only one ConditionVariable to wait for InitializedUpTo being advanced.

I didn't benchmark my version, just checked that tests passed.

Good day, Alexander.

I've got mixed but quite close result for both approaches (single or many
ConditionVariable) on the notebook. Since I have no access to larger
machine, I can't prove "many" is way better (or discover it worse).

Given patch after cleanup looks a bit smaller and clearer, I agree to keep
just single condition variable.

Cleaned version is attached.

I've changed condition for broadcast a bit ("less" instead "not equal"):
- buffer's border may already go into future,
- and then other backend will reach not yet initialized buffer and will
broadcast.

-------
regards
Yura Sokolov aka funny-falcon

Attachments:

v2-0001-Get-rid-of-WALBufMappingLock.patchtext/x-patch; charset=UTF-8; name=v2-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 709ef74a8424fe626e2a2170eb9a8a1493e23cb6 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v2 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers using
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 144 +++++++++++-------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 90 insertions(+), 58 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 9c270e7d466..c4b80ede5da 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -472,22 +467,33 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
+
+	/* Notification for update of InitializedUpTo. */
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,43 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
-	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
-	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	/* Try to initialize pages we need in WAL buffer. */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
-
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2048,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2069,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2076,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2139,11 +2152,25 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+			NewPageEndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			if (NewPageEndPtr < NewPageBeginPtr + XLOG_BLCKSZ)
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+			if (NewPageEndPtr != NewPageBeginPtr + XLOG_BLCKSZ)
+				break;
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5071,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6094,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6103,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.43.0

v2-0002-several-attempts-to-lock-WALInsertLocks.patchtext/x-patch; charset=UTF-8; name=v2-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From 0ec1841eace0bf108e1f07e882e0da9c78e464a0 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v2 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 47 ++++++++++++++++++-------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c4b80ede5da..8f6fd77aac4 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,8 +1377,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
-
+	int attempts = 2;
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
 	 * the one we used last time.  If the system isn't particularly busy, it's
@@ -1389,29 +1389,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32 rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1; /* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+	/*
+	 * If we couldn't get the lock immediately, try another lock next
+	 * time.  On a system with more insertion locks than concurrent
+	 * inserters, this causes all the inserters to eventually migrate to a
+	 * lock that no-one else is using.  On a system with more inserters
+	 * than locks, it still helps to distribute the inserters evenly
+	 * across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.43.0

#5Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Yura Sokolov (#4)
2 attachment(s)
Re: Get rid of WALBufMappingLock

07.02.2025 14:02, Yura Sokolov пишет:

07.02.2025 01:26, Alexander Korotkov пишет:

Hi!

On Sun, Jan 19, 2025 at 2:11 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

During discussion of Increasing NUM_XLOGINSERT_LOCKS [1], Andres Freund
used benchmark which creates WAL records very intensively. While I this
it is not completely fair (1MB log records are really rare), it pushed
me to analyze write-side waiting of XLog machinery.

First I tried to optimize WaitXLogInsertionsToFinish, but without great
success (yet).

While profiling, I found a lot of time is spend in the memory clearing
under global WALBufMappingLock:

MemSet((char *) NewPage, 0, XLOG_BLCKSZ);

It is obvious scalability bottleneck.

So "challenge was accepted".

Certainly, backend should initialize pages without exclusive lock. But
which way to ensure pages were initialized? In other words, how to
ensure XLogCtl->InitializedUpTo is correct.

I've tried to play around WALBufMappingLock with holding it for a short
time and spinning on XLogCtl->xlblocks[nextidx]. But in the end I found
WALBufMappingLock is useless at all.

Instead of holding lock, it is better to allow backends to cooperate:
- I bound ConditionVariable to each xlblocks entry,
- every backend now checks every required block pointed by
InitializedUpto was successfully initialized or sleeps on its condvar,
- when backend sure block is initialized, it tries to update
InitializedUpTo with conditional variable.

Looks reasonable for me, but having ConditionVariable per xlog buffer
seems overkill for me. Find an attached revision, where I've
implemented advancing InitializedUpTo without ConditionVariable.
After initialization of each buffer there is attempt to do CAS for
InitializedUpTo in a loop. So, multiple processes will try to advance
InitializedUpTo, they could hijack initiative from each other, but
there is always a leader which will finish the work.

There is only one ConditionVariable to wait for InitializedUpTo being advanced.

I didn't benchmark my version, just checked that tests passed.

Good day, Alexander.

Seems I was mistaken twice.

I've got mixed but quite close result for both approaches (single or many
ConditionVariable) on the notebook. Since I have no access to larger
machine, I can't prove "many" is way better (or discover it worse).

1. "many condvars" (my variant) is strictly worse with num locks = 64 and
when pg_logical_emit_message emits just 1kB instead of 1MB.

Therefore, "single condvar" is strictly better.

Given patch after cleanup looks a bit smaller and clearer, I agree to keep
just single condition variable.

Cleaned version is attached.

I've changed condition for broadcast a bit ("less" instead "not equal"):
- buffer's border may already go into future,
- and then other backend will reach not yet initialized buffer and will
broadcast.

2. I've inserted abort if "buffer's border went into future", and wasn't
able to trigger it.

So I returned update-loop's body to your variant.

-------
regards
Yura Sokolov aka funny-falcon

Attachments:

v3-0001-Get-rid-of-WALBufMappingLock.patchtext/x-patch; charset=UTF-8; name=v3-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 47cfcc49c364d6e67b68082558f0a55ff531ab86 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v3 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers using
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 145 +++++++++++-------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 91 insertions(+), 58 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 9c270e7d466..b7ed84728df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -472,22 +467,33 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
+
+	/* Notification for update of InitializedUpTo. */
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,43 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
-	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
-	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	/* Try to initialize pages we need in WAL buffer. */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
-
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2048,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2069,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2076,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2139,11 +2152,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5072,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6095,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6104,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.43.0

v3-0002-several-attempts-to-lock-WALInsertLocks.patchtext/x-patch; charset=UTF-8; name=v3-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From ea102b7ee46392a02a0e23eef96b419c9b889842 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v3 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 47 ++++++++++++++++++-------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index b7ed84728df..7f0b3d425bf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,8 +1377,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
-
+	int attempts = 2;
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
 	 * the one we used last time.  If the system isn't particularly busy, it's
@@ -1389,29 +1389,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32 rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1; /* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+	/*
+	 * If we couldn't get the lock immediately, try another lock next
+	 * time.  On a system with more insertion locks than concurrent
+	 * inserters, this causes all the inserters to eventually migrate to a
+	 * lock that no-one else is using.  On a system with more inserters
+	 * than locks, it still helps to distribute the inserters evenly
+	 * across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.43.0

#6Alexander Korotkov
aekorotkov@gmail.com
In reply to: Yura Sokolov (#5)
Re: Get rid of WALBufMappingLock

On Fri, Feb 7, 2025 at 1:24 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

07.02.2025 14:02, Yura Sokolov пишет:

07.02.2025 01:26, Alexander Korotkov пишет:

Hi!

On Sun, Jan 19, 2025 at 2:11 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

During discussion of Increasing NUM_XLOGINSERT_LOCKS [1], Andres Freund
used benchmark which creates WAL records very intensively. While I this
it is not completely fair (1MB log records are really rare), it pushed
me to analyze write-side waiting of XLog machinery.

First I tried to optimize WaitXLogInsertionsToFinish, but without great
success (yet).

While profiling, I found a lot of time is spend in the memory clearing
under global WALBufMappingLock:

MemSet((char *) NewPage, 0, XLOG_BLCKSZ);

It is obvious scalability bottleneck.

So "challenge was accepted".

Certainly, backend should initialize pages without exclusive lock. But
which way to ensure pages were initialized? In other words, how to
ensure XLogCtl->InitializedUpTo is correct.

I've tried to play around WALBufMappingLock with holding it for a short
time and spinning on XLogCtl->xlblocks[nextidx]. But in the end I found
WALBufMappingLock is useless at all.

Instead of holding lock, it is better to allow backends to cooperate:
- I bound ConditionVariable to each xlblocks entry,
- every backend now checks every required block pointed by
InitializedUpto was successfully initialized or sleeps on its condvar,
- when backend sure block is initialized, it tries to update
InitializedUpTo with conditional variable.

Looks reasonable for me, but having ConditionVariable per xlog buffer
seems overkill for me. Find an attached revision, where I've
implemented advancing InitializedUpTo without ConditionVariable.
After initialization of each buffer there is attempt to do CAS for
InitializedUpTo in a loop. So, multiple processes will try to advance
InitializedUpTo, they could hijack initiative from each other, but
there is always a leader which will finish the work.

There is only one ConditionVariable to wait for InitializedUpTo being advanced.

I didn't benchmark my version, just checked that tests passed.

Good day, Alexander.

Seems I was mistaken twice.

I've got mixed but quite close result for both approaches (single or many
ConditionVariable) on the notebook. Since I have no access to larger
machine, I can't prove "many" is way better (or discover it worse).

1. "many condvars" (my variant) is strictly worse with num locks = 64 and
when pg_logical_emit_message emits just 1kB instead of 1MB.

Therefore, "single condvar" is strictly better.

Given patch after cleanup looks a bit smaller and clearer, I agree to keep
just single condition variable.

Cleaned version is attached.

I've changed condition for broadcast a bit ("less" instead "not equal"):
- buffer's border may already go into future,
- and then other backend will reach not yet initialized buffer and will
broadcast.

2. I've inserted abort if "buffer's border went into future", and wasn't
able to trigger it.

So I returned update-loop's body to your variant.

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

Regarding 0002 patch, it looks generally reasonable. But are 2
attempts always optimal? Are there cases of regression, or cases when
more attempts are even better? Could we have there some
self-adjusting mechanism like what we have for spinlocks?

------
Regards,
Alexander Korotkov
Supabase

#7Alexander Korotkov
aekorotkov@gmail.com
In reply to: Alexander Korotkov (#6)
Re: Get rid of WALBufMappingLock

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.
2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

------
Regards,
Alexander Korotkov
Supabase

#8Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Alexander Korotkov (#7)
2 attachment(s)
Re: Get rid of WALBufMappingLock

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Regarding 0002 patch, it looks generally reasonable. But are 2
attempts always optimal? Are there cases of regression, or cases when
more attempts are even better? Could we have there some
self-adjusting mechanism like what we have for spinlocks?

Well, I chose to perform 3 probes (2 conditional attempts + 1
unconditional) based on intuition. I have some experience in building hash
tables, and cuckoo-hashing theory tells 2 probes is usually enough to reach
50% fill-rate, and 3 probes enough for ~75% fill rate. Since each probe is
cache miss, it is hardly sensible to do more probes.

3 probes did better than 2 in other benchmark [1]/messages/by-id/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru, although there were
NUM_XLOGINSERT_LOCK increased.

Excuse me for not bencmarking different choices here. I'll try to do
measurements in next days.

[1]: /messages/by-id/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru

-------
regards
Yura Sokolov aka funny-falcon

Attachments:

v4-0001-Get-rid-of-WALBufMappingLock.patchtext/x-patch; charset=UTF-8; name=v4-0001-Get-rid-of-WALBufMappingLock.patchDownload
From d243176a52faf6a57edd5719cfcbe8b3d1bbb919 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v4 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers using
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 172 ++++++++++++------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 120 insertions(+), 56 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index fd389565123..83172b1024b 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -472,22 +467,33 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
+
+	/* Notification for update of InitializedUpTo. */
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,57 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
-	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	/*----------
+	 * Try to initialize pages we need in WAL buffer.
+	 *
+	 * To perform that we:
+	 * 1) Reserve some page for initialization.
+	 * 2) Ensure old content of buffer is written to disk either by writting it,
+	 *    or waiting for concurrent writer to do it.
+	 * 3) Clear page and initialize its header.
+	 * 4) Mark page as initialized by writting to xlblocks entry.
+	 * 5) Advance XLogCtl->InitializedUpTo.
+	 * 6) When no pending pages remain until upto, wait until its initialization
+	 *    complete (ie InitializedUpTo is advanced enough).
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2062,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2083,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2090,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2139,11 +2166,43 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Advance XLogCtl->InitializedUpTo.
+		 *
+		 * If compare_exchange failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing. In this case we inform all waiters InitializedUpTo was
+		 * advanced.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5103,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6126,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6135,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.43.0

v4-0002-several-attempts-to-lock-WALInsertLocks.patchtext/x-patch; charset=UTF-8; name=v4-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From 242e5d22edc2db8be71bf8eb36390a5f17a713c6 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v4 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 46 +++++++++++++++++++------------
 1 file changed, 28 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 83172b1024b..8ab8f1947df 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,7 +1377,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
+	int			attempts = 2;
 
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
@@ -1389,29 +1390,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32		rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1;	/* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+
+	/*
+	 * If we couldn't get the lock immediately, try another lock next time. On
+	 * a system with more insertion locks than concurrent inserters, this
+	 * causes all the inserters to eventually migrate to a lock that no-one
+	 * else is using.  On a system with more inserters than locks, it still
+	 * helps to distribute the inserters evenly across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.43.0

#9Alexander Korotkov
aekorotkov@gmail.com
In reply to: Yura Sokolov (#8)
2 attachment(s)
Re: Get rid of WALBufMappingLock

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

Regarding 0002 patch, it looks generally reasonable. But are 2
attempts always optimal? Are there cases of regression, or cases when
more attempts are even better? Could we have there some
self-adjusting mechanism like what we have for spinlocks?

Well, I chose to perform 3 probes (2 conditional attempts + 1
unconditional) based on intuition. I have some experience in building hash
tables, and cuckoo-hashing theory tells 2 probes is usually enough to reach
50% fill-rate, and 3 probes enough for ~75% fill rate. Since each probe is
cache miss, it is hardly sensible to do more probes.

3 probes did better than 2 in other benchmark [1], although there were
NUM_XLOGINSERT_LOCK increased.

Excuse me for not bencmarking different choices here. I'll try to do
measurements in next days.

[1] /messages/by-id/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru

Ok, let's wait for your measurements.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v5-0002-several-attempts-to-lock-WALInsertLocks.patchapplication/octet-stream; name=v5-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From b94d681fa0b39f265e133c09d6df8595a9b51f94 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v5 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 47 ++++++++++++++++++-------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c70ac26026e..33d26fc37cf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1384,8 +1385,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
-
+	int attempts = 2;
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
 	 * the one we used last time.  If the system isn't particularly busy, it's
@@ -1397,29 +1397,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32 rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1; /* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+	/*
+	 * If we couldn't get the lock immediately, try another lock next
+	 * time.  On a system with more insertion locks than concurrent
+	 * inserters, this causes all the inserters to eventually migrate to a
+	 * lock that no-one else is using.  On a system with more inserters
+	 * than locks, it still helps to distribute the inserters evenly
+	 * across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.39.5 (Apple Git-154)

v5-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v5-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 683c008b71d173dea34810ba445bba6f51f52af6 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v5 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers usign
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 176 +++++++++++-------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 111 insertions(+), 69 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 9c270e7d466..c70ac26026e 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,10 +302,7 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
+ * (TODO: describe AdvanceXLInsertBuffer)
  *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
@@ -444,6 +441,11 @@ typedef struct XLogCtlInsert
 	WALInsertLockPadded *WALInsertLocks;
 } XLogCtlInsert;
 
+typedef struct XLBlocks
+{
+	pg_atomic_uint64 bound;
+}			XLBlocks;
+
 /*
  * Total shared-memory state for XLOG.
  */
@@ -472,25 +474,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities, perhaps
+	 * waiting on conditional variable bound to buffer.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
+
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
-	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
+	XLBlocks   *xlblocks;		/* 1st byte ptr-s + XLOG_BLCKSZ (and condvar
+								 * for) */
 	int			XLogCacheBlck;	/* highest allocated xlog buffer index */
 
 	/*
@@ -810,8 +824,8 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that ahead of insertions
 	 * to avoid that from happening in the critical path.
 	 *
 	 *----------
@@ -1671,7 +1685,7 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 	expectedEndPtr = ptr;
 	expectedEndPtr += XLOG_BLCKSZ - ptr % XLOG_BLCKSZ;
 
-	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 	if (expectedEndPtr != endptr)
 	{
 		XLogRecPtr	initializedUpto;
@@ -1702,11 +1716,11 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 		WALInsertLockUpdateInsertingAt(initializedUpto);
 
 		AdvanceXLInsertBuffer(ptr, tli, false);
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 
 		if (expectedEndPtr != endptr)
-			elog(PANIC, "could not find WAL buffer for %X/%X",
-				 LSN_FORMAT_ARGS(ptr));
+			elog(PANIC, "could not find WAL buffer for %X/%X: %X/%X != %X/%X",
+				 LSN_FORMAT_ARGS(ptr), LSN_FORMAT_ARGS(expectedEndPtr), LSN_FORMAT_ARGS(endptr));
 	}
 	else
 	{
@@ -1803,7 +1817,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * First verification step: check that the correct page is present in
 		 * the WAL buffers.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1835,7 +1849,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * Second verification step: check that the page we read from wasn't
 		 * evicted while we were copying the data.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1991,32 +2005,45 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * Try to initialize pages we need in WAL buffer.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
-
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2058,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2079,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2086,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2138,12 +2161,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 */
 		pg_write_barrier();
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, NewPageEndPtr);
+		//ConditionVariableBroadcast(&XLogCtl->xlblocks[nextidx].condvar);
 
-		npages++;
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound) != NewPageEndPtr)
+			{
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -2356,7 +2393,7 @@ XLogWrite(XLogwrtRqst WriteRqst, TimeLineID tli, bool flexible)
 		 * if we're passed a bogus WriteRqst.Write that is past the end of the
 		 * last page that's been initialized by AdvanceXLInsertBuffer.
 		 */
-		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx]);
+		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx].bound);
 
 		if (LogwrtResult.Write >= EndPtr)
 			elog(PANIC, "xlog write request %X/%X is past end of log %X/%X",
@@ -4920,7 +4957,7 @@ XLOGShmemSize(void)
 	/* WAL insertion locks, plus alignment */
 	size = add_size(size, mul_size(sizeof(WALInsertLockPadded), NUM_XLOGINSERT_LOCKS + 1));
 	/* xlblocks array */
-	size = add_size(size, mul_size(sizeof(pg_atomic_uint64), XLOGbuffers));
+	size = add_size(size, mul_size(sizeof(XLBlocks), XLOGbuffers));
 	/* extra alignment padding for XLOG I/O buffers */
 	size = add_size(size, Max(XLOG_BLCKSZ, PG_IO_ALIGN_SIZE));
 	/* and the buffers themselves */
@@ -4998,12 +5035,12 @@ XLOGShmemInit(void)
 	 * needed here.
 	 */
 	allocptr = ((char *) XLogCtl) + sizeof(XLogCtlData);
-	XLogCtl->xlblocks = (pg_atomic_uint64 *) allocptr;
-	allocptr += sizeof(pg_atomic_uint64) * XLOGbuffers;
+	XLogCtl->xlblocks = (XLBlocks *) allocptr;
+	allocptr += sizeof(XLBlocks) * XLOGbuffers;
 
 	for (i = 0; i < XLOGbuffers; i++)
 	{
-		pg_atomic_init_u64(&XLogCtl->xlblocks[i], InvalidXLogRecPtr);
+		pg_atomic_init_u64(&XLogCtl->xlblocks[i].bound, InvalidXLogRecPtr);
 	}
 
 	/* WAL insertion locks. Ensure they're aligned to the full padded size */
@@ -5044,6 +5081,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6062,8 +6103,8 @@ StartupXLOG(void)
 		memcpy(page, endOfRecoveryInfo->lastPage, len);
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx].bound, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6113,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

#10Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Alexander Korotkov (#9)
Re: Get rid of WALBufMappingLock

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

-------
regards
Yura Sokolov aka funny-falcon

#11Alexander Korotkov
aekorotkov@gmail.com
In reply to: Yura Sokolov (#10)
2 attachment(s)
Re: Get rid of WALBufMappingLock

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v6-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v6-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 683c008b71d173dea34810ba445bba6f51f52af6 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Sat, 18 Jan 2025 23:50:09 +0300
Subject: [PATCH v6 1/2] Get rid of WALBufMappingLock

Allow many backends to concurrently initialize XLog buffers.
This way `MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` is not under single
LWLock in exclusive mode.

Algorithm:
- backend first reserves page for initialization,
- then it ensures it was written out,
- this it initialized it and signals concurrent initializers usign
  ConditionVariable,
- when enough pages reserved for initialization for this backend, it
  ensures all required pages completes initialization.

Many backends concurrently reserve pages, initialize them, and advance
XLogCtl->InitializedUpTo to point latest initialized page.
---
 src/backend/access/transam/xlog.c             | 176 +++++++++++-------
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 111 insertions(+), 69 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 9c270e7d466..c70ac26026e 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,10 +302,7 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
+ * (TODO: describe AdvanceXLInsertBuffer)
  *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
@@ -444,6 +441,11 @@ typedef struct XLogCtlInsert
 	WALInsertLockPadded *WALInsertLocks;
 } XLogCtlInsert;
 
+typedef struct XLBlocks
+{
+	pg_atomic_uint64 bound;
+}			XLBlocks;
+
 /*
  * Total shared-memory state for XLOG.
  */
@@ -472,25 +474,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logWriteResult;	/* last byte + 1 written out */
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
+	/*
+	 * Latest initialized or reserved for inititalization page in the cache
+	 * (last byte position + 1).
+	 *
+	 * It should be advanced before identity of a buffer will be changed to.
+	 * To change the identity of a buffer that's still dirty, the old page
+	 * needs to be written out first, and for that you need WALWriteLock, and
+	 * you need to ensure that there are no in-progress insertions to the page
+	 * by calling WaitXLogInsertionsToFinish().
+	 */
+	pg_atomic_uint64 InitializeReserved;
+
 	/*
 	 * Latest initialized page in the cache (last byte position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
-	 * WaitXLogInsertionsToFinish().
+	 * It is updated to successfully initialized buffer's identities, perhaps
+	 * waiting on conditional variable bound to buffer.
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializedUpTo;
+
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
-	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
+	XLBlocks   *xlblocks;		/* 1st byte ptr-s + XLOG_BLCKSZ (and condvar
+								 * for) */
 	int			XLogCacheBlck;	/* highest allocated xlog buffer index */
 
 	/*
@@ -810,8 +824,8 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that ahead of insertions
 	 * to avoid that from happening in the critical path.
 	 *
 	 *----------
@@ -1671,7 +1685,7 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 	expectedEndPtr = ptr;
 	expectedEndPtr += XLOG_BLCKSZ - ptr % XLOG_BLCKSZ;
 
-	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+	endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 	if (expectedEndPtr != endptr)
 	{
 		XLogRecPtr	initializedUpto;
@@ -1702,11 +1716,11 @@ GetXLogBuffer(XLogRecPtr ptr, TimeLineID tli)
 		WALInsertLockUpdateInsertingAt(initializedUpto);
 
 		AdvanceXLInsertBuffer(ptr, tli, false);
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 
 		if (expectedEndPtr != endptr)
-			elog(PANIC, "could not find WAL buffer for %X/%X",
-				 LSN_FORMAT_ARGS(ptr));
+			elog(PANIC, "could not find WAL buffer for %X/%X: %X/%X != %X/%X",
+				 LSN_FORMAT_ARGS(ptr), LSN_FORMAT_ARGS(expectedEndPtr), LSN_FORMAT_ARGS(endptr));
 	}
 	else
 	{
@@ -1803,7 +1817,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * First verification step: check that the correct page is present in
 		 * the WAL buffers.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1835,7 +1849,7 @@ WALReadFromBuffers(char *dstbuf, XLogRecPtr startptr, Size count,
 		 * Second verification step: check that the page we read from wasn't
 		 * evicted while we were copying the data.
 		 */
-		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx]);
+		endptr = pg_atomic_read_u64(&XLogCtl->xlblocks[idx].bound);
 		if (expectedEndPtr != endptr)
 			break;
 
@@ -1991,32 +2005,45 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * Try to initialize pages we need in WAL buffer.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
-
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/* Actually reserve the page for initialization. */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved, &ReservedPtr, ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2058,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2079,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2086,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2138,12 +2161,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 */
 		pg_write_barrier();
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx].bound, NewPageEndPtr);
+		//ConditionVariableBroadcast(&XLogCtl->xlblocks[nextidx].condvar);
 
-		npages++;
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx].bound) != NewPageEndPtr)
+			{
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -2356,7 +2393,7 @@ XLogWrite(XLogwrtRqst WriteRqst, TimeLineID tli, bool flexible)
 		 * if we're passed a bogus WriteRqst.Write that is past the end of the
 		 * last page that's been initialized by AdvanceXLInsertBuffer.
 		 */
-		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx]);
+		XLogRecPtr	EndPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[curridx].bound);
 
 		if (LogwrtResult.Write >= EndPtr)
 			elog(PANIC, "xlog write request %X/%X is past end of log %X/%X",
@@ -4920,7 +4957,7 @@ XLOGShmemSize(void)
 	/* WAL insertion locks, plus alignment */
 	size = add_size(size, mul_size(sizeof(WALInsertLockPadded), NUM_XLOGINSERT_LOCKS + 1));
 	/* xlblocks array */
-	size = add_size(size, mul_size(sizeof(pg_atomic_uint64), XLOGbuffers));
+	size = add_size(size, mul_size(sizeof(XLBlocks), XLOGbuffers));
 	/* extra alignment padding for XLOG I/O buffers */
 	size = add_size(size, Max(XLOG_BLCKSZ, PG_IO_ALIGN_SIZE));
 	/* and the buffers themselves */
@@ -4998,12 +5035,12 @@ XLOGShmemInit(void)
 	 * needed here.
 	 */
 	allocptr = ((char *) XLogCtl) + sizeof(XLogCtlData);
-	XLogCtl->xlblocks = (pg_atomic_uint64 *) allocptr;
-	allocptr += sizeof(pg_atomic_uint64) * XLOGbuffers;
+	XLogCtl->xlblocks = (XLBlocks *) allocptr;
+	allocptr += sizeof(XLBlocks) * XLOGbuffers;
 
 	for (i = 0; i < XLOGbuffers; i++)
 	{
-		pg_atomic_init_u64(&XLogCtl->xlblocks[i], InvalidXLogRecPtr);
+		pg_atomic_init_u64(&XLogCtl->xlblocks[i].bound, InvalidXLogRecPtr);
 	}
 
 	/* WAL insertion locks. Ensure they're aligned to the full padded size */
@@ -5044,6 +5081,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6062,8 +6103,8 @@ StartupXLOG(void)
 		memcpy(page, endOfRecoveryInfo->lastPage, len);
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
-		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx].bound, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6113,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

v6-0002-several-attempts-to-lock-WALInsertLocks.patchapplication/octet-stream; name=v6-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From b94d681fa0b39f265e133c09d6df8595a9b51f94 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v6 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 47 ++++++++++++++++++-------------
 1 file changed, 28 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c70ac26026e..33d26fc37cf 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1384,8 +1385,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
-
+	int attempts = 2;
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
 	 * the one we used last time.  If the system isn't particularly busy, it's
@@ -1397,29 +1397,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32 rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1; /* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+	/*
+	 * If we couldn't get the lock immediately, try another lock next
+	 * time.  On a system with more insertion locks than concurrent
+	 * inserters, this causes all the inserters to eventually migrate to a
+	 * lock that no-one else is using.  On a system with more inserters
+	 * than locks, it still helps to distribute the inserters evenly
+	 * across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.39.5 (Apple Git-154)

#12Pavel Borisov
pashkin.elfe@gmail.com
In reply to: Alexander Korotkov (#11)
Re: Get rid of WALBufMappingLock

Hi, Yura and Alexander!

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

The overall goal to avoid locking looks good. Both patches look right
to me. The only thing I'm slightly concerned about if patchset could
demonstrate performance differences in any real workload. I appreciate
it as a beautiful optimization anyway.

I have the following proposals to change the code and comments:

For patch 0001:

- Struct XLBlocks contains just one pg_atomic_uint64 member. Is it
still needed as a struct? These changes make a significant volume of
changes to the patch, being noop. Maybe it was inherited from v1 and
not needed anymore.

- Furthermore when xlblocks became a struct comments like:

and xlblocks values certainly do. xlblocks values are changed

need to be changed to xlblocks.bound. This could be avoided by
changing back xlblocks from type XLBlocks * to pg_atomic_uint64 *

- It's worth more detailed commenting
InitializedUpTo/InitializedUpToCondVar than:
+        * It is updated to successfully initialized buffer's
identities, perhaps
+        * waiting on conditional variable bound to buffer.

"perhaps waiting" could also be in style "maybe/even while AAA waits BBB"

"lock-free with cooperation with" -> "lock-free accompanied by changes to..."

- Comment inside typedef struct XLogCtlData:
/* 1st byte ptr-s + XLOG_BLCKSZ (and condvar * for) */
need to be returned back
/* 1st byte ptr-s + XLOG_BLCKSZ */

- Commented out code for cleanup in the final patch:
//ConditionVariableBroadcast(&XLogCtl->xlblocks[nextidx].condvar);

- in AdvanceXLInsertBuffer()
npages initialised to 0 but it is not increased anymore
Block under

if (XLOG_DEBUG && npages > 0)

became unreachable

(InvalidXLogRecPtr + 1) is essentially 0+1 and IMO this semantically
calls for adding #define FirstValidXLogRecPtr 1

Typo in a commit message: %s/usign/using/g

For patch 0002:

I think Yura's explanation from above in this thread need to get place
in a commit message and in a comment to this:

int attempts = 2;

Comments around:
"try another lock next time" could be modified to reflect that we do
repeat twice

Kind regards,
Pavel Borisov
Supabase

#13Alexander Korotkov
aekorotkov@gmail.com
In reply to: Pavel Borisov (#12)
2 attachment(s)
Re: Get rid of WALBufMappingLock

Hi, Pavel!

On Thu, Feb 13, 2025 at 6:39 PM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

Oh, sorry, I really did wrong. I've done git format-patch for wrong
local branch for v5 and v6. Patches I've sent for v5 and v6 are
actually the same as my v1. This is really pity. Please, find the
right version of patchset attached.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v7-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v7-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 5ea7d62b2ed010fad71ed29e91db7786073bd3fd Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Thu, 13 Feb 2025 01:20:20 +0200
Subject: [PATCH v7 1/2] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization,
 * Ensure the page is written out,
 * once the page is initialized, signal concurrent initializers using
   ConditionVariable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a ConditionVariable.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.
---
 src/backend/access/transam/xlog.c             | 184 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 136 insertions(+), 52 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index fd389565123..c54042c48c1 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,32 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first. To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free with cooperation with InitializeReserved+InitializedUpTo and
+	 * check for write position.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,70 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2075,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2096,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,19 +2103,26 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
-		 * Mark the xlblock with InvalidXLogRecPtr and issue a write barrier
-		 * before initializing. Otherwise, the old page may be partially
-		 * zeroed but look valid.
+		 * Mark the xlblock with (InvalidXLogRecPtr+1) and issue a write
+		 * barrier before initializing. Otherwise, the old page may be
+		 * partially zeroed but look valid.
 		 */
-		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr);
+		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], InvalidXLogRecPtr + 1);
 		pg_write_barrier();
 
 		/*
@@ -2139,11 +2179,50 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5123,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6146,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6155,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

v7-0002-several-attempts-to-lock-WALInsertLocks.patchapplication/octet-stream; name=v7-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From e9ce97a2b1ec8d2c0931d5381ec2a1be94265c45 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v7 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 46 +++++++++++++++++++------------
 1 file changed, 28 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c54042c48c1..f3147f06fbb 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,7 +1377,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
+	int			attempts = 2;
 
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
@@ -1389,29 +1390,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32		rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1;	/* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+
+	/*
+	 * If we couldn't get the lock immediately, try another lock next time. On
+	 * a system with more insertion locks than concurrent inserters, this
+	 * causes all the inserters to eventually migrate to a lock that no-one
+	 * else is using.  On a system with more inserters than locks, it still
+	 * helps to distribute the inserters evenly across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.39.5 (Apple Git-154)

#14Pavel Borisov
pashkin.elfe@gmail.com
In reply to: Alexander Korotkov (#13)
Re: Get rid of WALBufMappingLock

Hi, Alexander!

On Fri, 14 Feb 2025 at 00:59, Alexander Korotkov <aekorotkov@gmail.com> wrote:

Hi, Pavel!

On Thu, Feb 13, 2025 at 6:39 PM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

Oh, sorry, I really did wrong. I've done git format-patch for wrong
local branch for v5 and v6. Patches I've sent for v5 and v6 are
actually the same as my v1. This is really pity. Please, find the
right version of patchset attached.

I've rechecked v7. In v6 a proposal from [1]/messages/by-id/d6799557-e352-42c8-80cc-ed36e3b8893c@postgrespro.ru was not reflected. Now it
landed in v7.

Other changes are not regarding code behavior. The things from my
previous review that still could apply to v7:

For 0001:

Comment change proposed:
"lock-free with cooperation with" -> "lock-free accompanied by changes
to..." (maybe other variant)

I propose a new define:
#define FirstValidXLogRecPtr 1
While FirstValidXLogRecPtr = InvalidXLogRecPtr + 1 is true in the code
that has no semantical meaning and it's better to avoid using direct
arithmetics to relate meaning of FirstValidXLogRecPtr from
InvalidXLogRecPtr.

For 0002 both comments proposals from my message applied to v6 apply
to v7 as well

[1]: /messages/by-id/d6799557-e352-42c8-80cc-ed36e3b8893c@postgrespro.ru

Regards,
Pavel Borisov
Supabase

#15Alexander Korotkov
aekorotkov@gmail.com
In reply to: Pavel Borisov (#14)
2 attachment(s)
Re: Get rid of WALBufMappingLock

On Fri, Feb 14, 2025 at 11:45 AM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Fri, 14 Feb 2025 at 00:59, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 6:39 PM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

Oh, sorry, I really did wrong. I've done git format-patch for wrong
local branch for v5 and v6. Patches I've sent for v5 and v6 are
actually the same as my v1. This is really pity. Please, find the
right version of patchset attached.

I've rechecked v7. In v6 a proposal from [1] was not reflected. Now it
landed in v7.

Other changes are not regarding code behavior. The things from my
previous review that still could apply to v7:

For 0001:

Comment change proposed:
"lock-free with cooperation with" -> "lock-free accompanied by changes
to..." (maybe other variant)

Good catch. I've rephrased this comment even more.

I propose a new define:
#define FirstValidXLogRecPtr 1
While FirstValidXLogRecPtr = InvalidXLogRecPtr + 1 is true in the code
that has no semantical meaning and it's better to avoid using direct
arithmetics to relate meaning of FirstValidXLogRecPtr from
InvalidXLogRecPtr.

Makes sense, but I'm not sure if this change is required at all. I've
reverted this to the state of master, and everything seems to work.

For 0002 both comments proposals from my message applied to v6 apply
to v7 as well

Thank you for pointing. For now, I'm concentrated on improvements on
0001. Probably Yura could work on your notes to 0002.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v8-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v8-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 162f70e25646e03333c5ad8e1f8fe6b776a2beb9 Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Thu, 13 Feb 2025 01:20:20 +0200
Subject: [PATCH v8 1/2] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization,
 * Ensure the page is written out,
 * once the page is initialized, signal concurrent initializers using
   ConditionVariable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a ConditionVariable.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 176 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 132 insertions(+), 48 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a50fd99d9e5..c232ba2d599 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,32 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first. To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,70 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2075,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2096,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,10 +2103,17 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
@@ -2139,11 +2179,50 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5123,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6146,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6155,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

v8-0002-several-attempts-to-lock-WALInsertLocks.patchapplication/octet-stream; name=v8-0002-several-attempts-to-lock-WALInsertLocks.patchDownload
From 8ebbb843307145f81f5ebd84cc36db5d4b31065e Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v8 2/2] several attempts to lock WALInsertLocks

---
 src/backend/access/transam/xlog.c | 46 +++++++++++++++++++------------
 1 file changed, 28 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c232ba2d599..e5c1eb9cbd2 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,7 +1377,7 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
+	int			attempts = 2;
 
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
@@ -1389,29 +1390,38 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
-	MyLockNo = lockToTry;
+	if (lockDelta == 0)
+	{
+		uint32		rng = pg_prng_uint32(&pg_global_prng_state);
+
+		lockToTry = rng % NUM_XLOGINSERT_LOCKS;
+		lockDelta = ((rng >> 16) % NUM_XLOGINSERT_LOCKS) | 1;	/* must be odd */
+	}
 
 	/*
 	 * The insertingAt value is initially set to 0, as we don't know our
 	 * insert location yet.
 	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
-	{
-		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
-		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
-	}
+	MyLockNo = lockToTry;
+retry:
+	if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+		return;
+
+	/*
+	 * If we couldn't get the lock immediately, try another lock next time. On
+	 * a system with more insertion locks than concurrent inserters, this
+	 * causes all the inserters to eventually migrate to a lock that no-one
+	 * else is using.  On a system with more inserters than locks, it still
+	 * helps to distribute the inserters evenly across the locks.
+	 */
+	lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+	MyLockNo = lockToTry;
+	if (--attempts)
+		goto retry;
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.39.5 (Apple Git-154)

#16Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Alexander Korotkov (#15)
2 attachment(s)
Re: Get rid of WALBufMappingLock

14.02.2025 13:24, Alexander Korotkov пишет:

On Fri, Feb 14, 2025 at 11:45 AM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Fri, 14 Feb 2025 at 00:59, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 6:39 PM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

Oh, sorry, I really did wrong. I've done git format-patch for wrong
local branch for v5 and v6. Patches I've sent for v5 and v6 are
actually the same as my v1. This is really pity. Please, find the
right version of patchset attached.

I've rechecked v7. In v6 a proposal from [1] was not reflected. Now it
landed in v7.

Other changes are not regarding code behavior. The things from my
previous review that still could apply to v7:

For 0001:

Comment change proposed:
"lock-free with cooperation with" -> "lock-free accompanied by changes
to..." (maybe other variant)

Good catch. I've rephrased this comment even more.

I propose a new define:
#define FirstValidXLogRecPtr 1
While FirstValidXLogRecPtr = InvalidXLogRecPtr + 1 is true in the code
that has no semantical meaning and it's better to avoid using direct
arithmetics to relate meaning of FirstValidXLogRecPtr from
InvalidXLogRecPtr.

Makes sense, but I'm not sure if this change is required at all. I've
reverted this to the state of master, and everything seems to work.

For 0002 both comments proposals from my message applied to v6 apply
to v7 as well

Thank you for pointing. For now, I'm concentrated on improvements on
0001. Probably Yura could work on your notes to 0002.

I wrote good commit message for 0002 with calculated probabilities and
simple Ruby program which calculates them to explain choice of 2
conditional attempts. (At least I hope the message is good). And added
simple comment before `int attempts = 2;`

Also I simplified 0002 a bit to look a bit prettier (ie without goto), and
added static assert on NUM_XLOGINSERT_LOCKS being power of 2.

(0001 patch is same as for v8)

-------
regards
Yura Sokolov aka funny-falcon

Attachments:

v9-0001-Get-rid-of-WALBufMappingLock.patchtext/x-patch; charset=UTF-8; name=v9-0001-Get-rid-of-WALBufMappingLock.patchDownload
From f9e4289000627801506226b0cd140872639e95ab Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Thu, 13 Feb 2025 01:20:20 +0200
Subject: [PATCH v9 1/2] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization,
 * Ensure the page is written out,
 * once the page is initialized, signal concurrent initializers using
   ConditionVariable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a ConditionVariable.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 176 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 132 insertions(+), 48 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a50fd99d9e5..c232ba2d599 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,32 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first. To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,70 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2075,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2096,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,10 +2103,17 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
@@ -2139,11 +2179,50 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5123,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6146,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6155,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.43.0

v9-0002-Several-attempts-to-lock-WALInsertLocks.patchtext/x-patch; charset=UTF-8; name=v9-0002-Several-attempts-to-lock-WALInsertLocks.patchDownload
From f0d9cc2140a10ca8d2b16a73e4be889f3665d89a Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v9 2/2] Several attempts to lock WALInsertLocks.

Birthday paradox tells we could get collision with 34% probability with
just 3, 58% with 4 and 79% with 5 concurrent processes
(when NUM_XLOGINSERT_LOCKS == 8).

Trying several times to lock conditionally with just linear increase by
random delta, will reduce probability of blocking greatly:
- with 1 conditional attempt - 4%, 14% and 33% respectively
- with 2 conditional attempts - 0%, 2.1% and 9.7%
- with 3 conditional attempts - 0%, 0% and 1.85%

Probabilities are calculated with simple Ruby program:
  def try_add(arr, n)
    a, b = rand(8), rand(8)|1
    n.times {
      (arr << a; break) unless arr.include?(a)
      a = (a + b) % 8
    }
  end
  def calc_prob(n, k)
    300000.times.
       map{ ar=[]; n.times{ try_add(ar, k) }; ar}.
       count{|a| a.length < n} / 300000.0 * 100
  end
  (3..5).each{|n| (1..5).each{|k| p [n,k,calc_prob(n, k).round(2)]}}

Given every attempt is a cache miss, 2 attempts following non-conditional
lock looks to be good compromise.
---
 src/backend/access/transam/xlog.c | 42 +++++++++++++++++++------------
 1 file changed, 26 insertions(+), 16 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c232ba2d599..e9fce1c8d9c 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,7 +1377,11 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
+	/*
+	 * Two conditional attempts looks to be good compromise between good
+	 * probability to acquire lock and cache misses on every attempt.
+	 */
+	int			attempts = 2;
 
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
@@ -1389,29 +1394,34 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
+	if (lockDelta == 0)
+	{
 		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
+		lockDelta = pg_prng_uint32(&pg_global_prng_state) % NUM_XLOGINSERT_LOCKS;
+		lockDelta |= 1;			/* must be odd */
+	}
+
 	MyLockNo = lockToTry;
 
-	/*
-	 * The insertingAt value is initially set to 0, as we don't know our
-	 * insert location yet.
-	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
+	while (attempts-- > 0)
 	{
+		if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+			return;
+
 		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
+		 * If we couldn't get the lock immediately, try another lock. On a
+		 * system with more insertion locks than concurrent inserters, this
+		 * causes all the inserters to eventually migrate to a lock that
+		 * no-one else is using.  On a system with more inserters than locks,
+		 * it still helps to distribute the inserters evenly across the locks.
 		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
+		lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+		MyLockNo = lockToTry;
 	}
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.43.0

#17Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Yura Sokolov (#16)
2 attachment(s)
Re: Get rid of WALBufMappingLock

14.02.2025 17:09, Yura Sokolov пишет:

14.02.2025 13:24, Alexander Korotkov пишет:

On Fri, Feb 14, 2025 at 11:45 AM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Fri, 14 Feb 2025 at 00:59, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 6:39 PM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

Oh, sorry, I really did wrong. I've done git format-patch for wrong
local branch for v5 and v6. Patches I've sent for v5 and v6 are
actually the same as my v1. This is really pity. Please, find the
right version of patchset attached.

I've rechecked v7. In v6 a proposal from [1] was not reflected. Now it
landed in v7.

Other changes are not regarding code behavior. The things from my
previous review that still could apply to v7:

For 0001:

Comment change proposed:
"lock-free with cooperation with" -> "lock-free accompanied by changes
to..." (maybe other variant)

Good catch. I've rephrased this comment even more.

I propose a new define:
#define FirstValidXLogRecPtr 1
While FirstValidXLogRecPtr = InvalidXLogRecPtr + 1 is true in the code
that has no semantical meaning and it's better to avoid using direct
arithmetics to relate meaning of FirstValidXLogRecPtr from
InvalidXLogRecPtr.

Makes sense, but I'm not sure if this change is required at all. I've
reverted this to the state of master, and everything seems to work.

For 0002 both comments proposals from my message applied to v6 apply
to v7 as well

Thank you for pointing. For now, I'm concentrated on improvements on
0001. Probably Yura could work on your notes to 0002.

I wrote good commit message for 0002 with calculated probabilities and
simple Ruby program which calculates them to explain choice of 2
conditional attempts. (At least I hope the message is good). And added
simple comment before `int attempts = 2;`

Also I simplified 0002 a bit to look a bit prettier (ie without goto), and
added static assert on NUM_XLOGINSERT_LOCKS being power of 2.

(0001 patch is same as for v8)

-------
regards
Yura Sokolov aka funny-falcon

Oops, forgot to add StaticAssert into v9-0002.

Attachments:

v10-0001-Get-rid-of-WALBufMappingLock.patchtext/x-patch; charset=UTF-8; name=v10-0001-Get-rid-of-WALBufMappingLock.patchDownload
From f9e4289000627801506226b0cd140872639e95ab Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Thu, 13 Feb 2025 01:20:20 +0200
Subject: [PATCH v10 1/2] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization,
 * Ensure the page is written out,
 * once the page is initialized, signal concurrent initializers using
   ConditionVariable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a ConditionVariable.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 176 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 132 insertions(+), 48 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index a50fd99d9e5..c232ba2d599 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,32 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first. To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +816,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +1997,70 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2075,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2096,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					PendingWalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,10 +2103,17 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
+#ifdef USE_ASSERT_CHECKING
+		{
+			XLogRecPtr	storedBound = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
+
+			Assert(storedBound == OldPageRqstPtr || storedBound == InvalidXLogRecPtr);
+		}
+#endif
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
@@ -2139,11 +2179,50 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5044,6 +5123,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6063,7 +6146,7 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
 	}
 	else
 	{
@@ -6072,8 +6155,9 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.43.0

v10-0002-Several-attempts-to-lock-WALInsertLocks.patchtext/x-patch; charset=UTF-8; name=v10-0002-Several-attempts-to-lock-WALInsertLocks.patchDownload
From 6b22d2e63d061be465e9e9f5ce2fefdaae6ebf0e Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.sokolov@postgrespro.ru>
Date: Thu, 16 Jan 2025 15:30:57 +0300
Subject: [PATCH v10 2/2] Several attempts to lock WALInsertLocks.

Birthday paradox tells we could get collision with 34% probability with
just 3, 58% with 4 and 79% with 5 concurrent processes
(when NUM_XLOGINSERT_LOCKS == 8).

Trying several times to lock conditionally with just linear increase by
random delta, will reduce probability of blocking greatly:
- with 1 conditional attempt - 4%, 14% and 33% respectively
- with 2 conditional attempts - 0%, 2.1% and 9.7%
- with 3 conditional attempts - 0%, 0% and 1.85%

Probabilities are calculated with simple Ruby program:
  def try_add(arr, n)
    a, b = rand(8), rand(8)|1
    n.times {
      (arr << a; break) unless arr.include?(a)
      a = (a + b) % 8
    }
  end
  def calc_prob(n, k)
    300000.times.
       map{ ar=[]; n.times{ try_add(ar, k) }; ar}.
       count{|a| a.length < n} / 300000.0 * 100
  end
  (3..5).each{|n| (1..5).each{|k| p [n,k,calc_prob(n, k).round(2)]}}

Given every attempt is a cache miss, 2 attempts following non-conditional
lock looks to be good compromise.
---
 src/backend/access/transam/xlog.c | 44 ++++++++++++++++++++-----------
 1 file changed, 28 insertions(+), 16 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index c232ba2d599..13fb1c35ad8 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -68,6 +68,7 @@
 #include "catalog/pg_database.h"
 #include "common/controldata_utils.h"
 #include "common/file_utils.h"
+#include "common/pg_prng.h"
 #include "executor/instrument.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
@@ -1376,7 +1377,11 @@ CopyXLogRecordToWAL(int write_len, bool isLogSwitch, XLogRecData *rdata,
 static void
 WALInsertLockAcquire(void)
 {
-	bool		immed;
+	/*
+	 * Two conditional attempts looks to be good compromise between good
+	 * probability to acquire lock and cache misses on every attempt.
+	 */
+	int			attempts = 2;
 
 	/*
 	 * It doesn't matter which of the WAL insertion locks we acquire, so try
@@ -1389,29 +1394,36 @@ WALInsertLockAcquire(void)
 	 * (semi-)randomly.  This allows the locks to be used evenly if you have a
 	 * lot of very short connections.
 	 */
-	static int	lockToTry = -1;
+	static uint32 lockToTry = 0;
+	static uint32 lockDelta = 0;
 
-	if (lockToTry == -1)
+	if (lockDelta == 0)
+	{
 		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
+		lockDelta = pg_prng_uint32(&pg_global_prng_state) % NUM_XLOGINSERT_LOCKS;
+		lockDelta |= 1;			/* must be odd */
+		StaticAssertStmt((NUM_XLOGINSERT_LOCKS & (NUM_XLOGINSERT_LOCKS - 1)) == 0,
+						 "NUM_XLOGINSERT_LOCKS must be power of 2");
+	}
+
 	MyLockNo = lockToTry;
 
-	/*
-	 * The insertingAt value is initially set to 0, as we don't know our
-	 * insert location yet.
-	 */
-	immed = LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
-	if (!immed)
+	while (attempts-- > 0)
 	{
+		if (LWLockConditionalAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE))
+			return;
+
 		/*
-		 * If we couldn't get the lock immediately, try another lock next
-		 * time.  On a system with more insertion locks than concurrent
-		 * inserters, this causes all the inserters to eventually migrate to a
-		 * lock that no-one else is using.  On a system with more inserters
-		 * than locks, it still helps to distribute the inserters evenly
-		 * across the locks.
+		 * If we couldn't get the lock immediately, try another lock. On a
+		 * system with more insertion locks than concurrent inserters, this
+		 * causes all the inserters to eventually migrate to a lock that
+		 * no-one else is using.  On a system with more inserters than locks,
+		 * it still helps to distribute the inserters evenly across the locks.
 		 */
-		lockToTry = (lockToTry + 1) % NUM_XLOGINSERT_LOCKS;
+		lockToTry = (lockToTry + lockDelta) % NUM_XLOGINSERT_LOCKS;
+		MyLockNo = lockToTry;
 	}
+	LWLockAcquire(&WALInsertLocks[MyLockNo].l.lock, LW_EXCLUSIVE);
 }
 
 /*
-- 
2.43.0

#18Alexander Korotkov
aekorotkov@gmail.com
In reply to: Yura Sokolov (#17)
Re: Get rid of WALBufMappingLock

Hi!

On Fri, Feb 14, 2025 at 4:11 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

14.02.2025 17:09, Yura Sokolov пишет:

14.02.2025 13:24, Alexander Korotkov пишет:

On Fri, Feb 14, 2025 at 11:45 AM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Fri, 14 Feb 2025 at 00:59, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 6:39 PM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

On Thu, 13 Feb 2025 at 14:08, Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Thu, Feb 13, 2025 at 11:45 AM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

13.02.2025 12:34, Alexander Korotkov пишет:

On Wed, Feb 12, 2025 at 8:16 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

08.02.2025 13:07, Alexander Korotkov пишет:

On Fri, Feb 7, 2025 at 1:39 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

Good, thank you. I think 0001 patch is generally good, but needs some
further polishing, e.g. more comments explaining how does it work.

I tried to add more comments. I'm not good at, so recommendations are welcome.

Two things comes to my mind worth rechecking about 0001.
1) Are XLogCtl->InitializeReserved, XLogCtl->InitializedUpTo and
XLogCtl->xlblocks always page-aligned? Because algorithm seems to be
sensitive to that. If so, I would propose to explicitly comment that
and add corresponding asserts.

They're certainly page aligned, since they are page borders.
I added assert on alignment of InitializeReserved for the sanity.

2) Check if there are concurrency issues between
AdvanceXLInsertBuffer() and switching to the new WAL file.

There are no issues:
1. CopyXLogRecordToWAL for isLogSwitch follows same protocol, ie uses
GetXLogBuffer to zero-out WAL page.
2. WALINSERT_SPECIAL_SWITCH forces exclusive lock on all insertion locks,
so switching wal is not concurrent. (Although, there is no need in this
exclusiveness, imho.)

Good, thank you. I've also revised commit message and comments.

But I see another issue with this patch. In the worst case, we do
XLogWrite() by ourselves, and it could potentially could error out.
Without patch, that would cause WALBufMappingLock be released and
XLogCtl->InitializedUpTo not advanced. With the patch, that would
cause other processes infinitely waiting till we finish the
initialization.

Possible solution would be to save position of the page to be
initialized, and set it back to XLogCtl->InitializeReserved on error
(everywhere we do LWLockReleaseAll()). We also must check that on
error we only set XLogCtl->InitializeReserved to the past, because
there could be multiple concurrent failures. Also we need to
broadcast XLogCtl->InitializedUpToCondVar to wake up waiters.

The single place where AdvanceXLInsertBuffer is called outside of critical
section is in XLogBackgroundFlush. All other call stacks will issue server
restart if XLogWrite will raise error inside of AdvanceXLInsertBuffer.

XLogBackgroundFlush explicitely avoids writing buffers by passing
opportunistic=true parameter.

Therefore, error in XLogWrite will not cause hang in AdvanceXLInsertBuffer
since server will shutdown/restart.

Perhaps, we just need to insert `Assert(CritSectionCount > 0);` before call
to XLogWrite here?

You're correct. I just reflected this in the next revision of the patch.

I've looked at the patchset v6.

Oh, sorry, I really did wrong. I've done git format-patch for wrong
local branch for v5 and v6. Patches I've sent for v5 and v6 are
actually the same as my v1. This is really pity. Please, find the
right version of patchset attached.

I've rechecked v7. In v6 a proposal from [1] was not reflected. Now it
landed in v7.

Other changes are not regarding code behavior. The things from my
previous review that still could apply to v7:

For 0001:

Comment change proposed:
"lock-free with cooperation with" -> "lock-free accompanied by changes
to..." (maybe other variant)

Good catch. I've rephrased this comment even more.

I propose a new define:
#define FirstValidXLogRecPtr 1
While FirstValidXLogRecPtr = InvalidXLogRecPtr + 1 is true in the code
that has no semantical meaning and it's better to avoid using direct
arithmetics to relate meaning of FirstValidXLogRecPtr from
InvalidXLogRecPtr.

Makes sense, but I'm not sure if this change is required at all. I've
reverted this to the state of master, and everything seems to work.

For 0002 both comments proposals from my message applied to v6 apply
to v7 as well

Thank you for pointing. For now, I'm concentrated on improvements on
0001. Probably Yura could work on your notes to 0002.

I wrote good commit message for 0002 with calculated probabilities and
simple Ruby program which calculates them to explain choice of 2
conditional attempts. (At least I hope the message is good). And added
simple comment before `int attempts = 2;`

Also I simplified 0002 a bit to look a bit prettier (ie without goto), and
added static assert on NUM_XLOGINSERT_LOCKS being power of 2.

(0001 patch is same as for v8)

Oops, forgot to add StaticAssert into v9-0002.

Thank you. I'm planning to push 0001 if there is no objections. And
I'm planning to do more review/revision of 0002.

------
Regards,
Alexander Korotkov
Supabase

#19Kirill Reshke
reshkekirill@gmail.com
In reply to: Yura Sokolov (#17)
Re: Get rid of WALBufMappingLock

Hi!
I spotted a typo in v10:

+ /*
+ * Page at nextidx wasn't initialized yet, so we cann't move
+ * InitializedUpto further. It will be moved by backend which
+ * will initialize nextidx.
+ */

cann't - > can't

moved by backend -> moved by the backend

--
Best regards,
Kirill Reshke

#20Victor Yegorov
vyegorov@gmail.com
In reply to: Yura Sokolov (#17)
Re: Get rid of WALBufMappingLock

Hey.

I find “Get rid of WALBufMappingLock" commit message misleading, 'cos Lock
it's being replaced by CV, actually.

Should the subject be changed to “Replace WALBufMappingLock with
ConditionVariable” instead?

--
Victor Yegorov

#21Pavel Borisov
pashkin.elfe@gmail.com
In reply to: Victor Yegorov (#20)
Re: Get rid of WALBufMappingLock

Hi, Victor!

On Mon, 17 Feb 2025 at 12:47, Victor Yegorov <vyegorov@gmail.com> wrote:

Hey.

I find “Get rid of WALBufMappingLock" commit message misleading, 'cos Lock it's being replaced by CV, actually.

Should the subject be changed to “Replace WALBufMappingLock with ConditionVariable” instead?

The patch replaces WALBufMappingLock with a lockless algorithm based
on atomic variables and CV. Mentioning only CV in the head is only a
part of implementation. Also, the header should better reflect what is
done on the whole, than the implementation details. So I'd rather see
a header like "Replace WALBufMappingLock by lockless algorithm" or
"Initialize WAL buffers concurrently without using WALBufMappingLock"
or something like that.

Kind regards,
Pavel Borisov
Supabase

#22Pavel Borisov
pashkin.elfe@gmail.com
In reply to: Pavel Borisov (#21)
Re: Get rid of WALBufMappingLock

On Mon, 17 Feb 2025 at 13:20, Pavel Borisov <pashkin.elfe@gmail.com> wrote:

Hi, Victor!

On Mon, 17 Feb 2025 at 12:47, Victor Yegorov <vyegorov@gmail.com> wrote:

Hey.

I find “Get rid of WALBufMappingLock" commit message misleading, 'cos Lock it's being replaced by CV, actually.

Should the subject be changed to “Replace WALBufMappingLock with ConditionVariable” instead?

The patch replaces WALBufMappingLock with a lockless algorithm based
on atomic variables and CV. Mentioning only CV in the head is only a
part of implementation. Also, the header should better reflect what is
done on the whole, than the implementation details. So I'd rather see
a header like "Replace WALBufMappingLock by lockless algorithm" or
"Initialize WAL buffers concurrently without using WALBufMappingLock"
or something like that.

Update: I see the patch is already committed, so we're late with the
naming proposals. I don't see problem with existing commit message
TBH.

Kind regards,
Pavel Borisov

#23Pavel Borisov
pashkin.elfe@gmail.com
In reply to: Pavel Borisov (#22)
1 attachment(s)
Re: Get rid of WALBufMappingLock

Hi, Kirill!
Per your report, I revised the comment to fix typos. Also some little
changes in grammar.

Kind regards,
Pavel Borisov

Attachments:

0001-Fix-typo-and-grammar-in-comment-introduced-by-6a2275.patchapplication/octet-stream; name=0001-Fix-typo-and-grammar-in-comment-introduced-by-6a2275.patchDownload
From 3a9e203e3a9c976e315101cfb19f29e8b3ee57b3 Mon Sep 17 00:00:00 2001
From: Pavel Borisov <pashkin.elfe@gmail.com>
Date: Mon, 17 Feb 2025 13:34:01 +0400
Subject: [PATCH] Fix typo and grammar in comment introduced by 6a2275b

Reported-by: Kirill Reshke
---
 src/backend/access/transam/xlog.c | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 75d5554c77c..010afffa482 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -2201,9 +2201,9 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
 			{
 				/*
-				 * Page at nextidx wasn't initialized yet, so we cann't move
-				 * InitializedUpto further. It will be moved by backend which
-				 * will initialize nextidx.
+				 * Page at nextidx hasn't been initialized yet, so we cannot move
+				 * InitializedUpto further. It will be moved by backend that
+				 * initializes nextidx.
 				 */
 				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
 				break;
-- 
2.39.2 (Apple Git-143)

#24Pavel Borisov
pashkin.elfe@gmail.com
In reply to: Pavel Borisov (#23)
1 attachment(s)
Re: Get rid of WALBufMappingLock

Oops, I send wrong patch as a fix.
The right one is attached.

Pavel

Show quoted text

On Mon, 17 Feb 2025 at 13:40, Pavel Borisov <pashkin.elfe@gmail.com> wrote:

Hi, Kirill!
Per your report, I revised the comment to fix typos. Also some little
changes in grammar.

Kind regards,
Pavel Borisov

Attachments:

v2-0001-Fix-typo-and-grammar-in-comment-introduced-by-6a2.patchapplication/octet-stream; name=v2-0001-Fix-typo-and-grammar-in-comment-introduced-by-6a2.patchDownload
From 67c33efa611170f2a19247aff4d746794f52823f Mon Sep 17 00:00:00 2001
From: Pavel Borisov <pashkin.elfe@gmail.com>
Date: Mon, 17 Feb 2025 13:34:01 +0400
Subject: [PATCH v2] Fix typo and grammar in comment introduced by 6a2275b

Reported-by: Kirill Reshke
---
 src/backend/access/transam/xlog.c | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 75d5554c77c..06ead0bb0fe 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -2201,9 +2201,9 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
 			{
 				/*
-				 * Page at nextidx wasn't initialized yet, so we cann't move
-				 * InitializedUpto further. It will be moved by backend which
-				 * will initialize nextidx.
+				 * Page at nextidx hasn't been initialized yet, so we can't move
+				 * InitializedUpto further. It will be moved by the backend that
+				 * initializes nextidx.
 				 */
 				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
 				break;
-- 
2.39.2 (Apple Git-143)

#25Alexander Korotkov
aekorotkov@gmail.com
In reply to: Pavel Borisov (#24)
Re: Get rid of WALBufMappingLock

On Mon, Feb 17, 2025 at 11:44 AM Pavel Borisov <pashkin.elfe@gmail.com> wrote:

Oops, I send wrong patch as a fix.
The right one is attached.

Pavel

I've spotted the failure on the buildfarm.
https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=batta&amp;dt=2025-02-17%2008%3A05%3A03
I can't quickly guess the reason. I'm going to revert patch for now,
then we investigate

------
Regards,
Alexander Korotkov
Supabase

#26Tom Lane
tgl@sss.pgh.pa.us
In reply to: Alexander Korotkov (#25)
Re: Get rid of WALBufMappingLock

Alexander Korotkov <aekorotkov@gmail.com> writes:

I've spotted the failure on the buildfarm.
https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=batta&amp;dt=2025-02-17%2008%3A05%3A03
I can't quickly guess the reason. I'm going to revert patch for now,
then we investigate

This timeout failure on hachi looks suspicious as well:

https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hachi&amp;dt=2025-02-17%2003%3A05%3A03

Might be relevant that they are both aarch64?

regards, tom lane

#27Michael Paquier
michael@paquier.xyz
In reply to: Tom Lane (#26)
Re: Get rid of WALBufMappingLock

On Mon, Feb 17, 2025 at 11:25:05AM -0500, Tom Lane wrote:

This timeout failure on hachi looks suspicious as well:

https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hachi&amp;dt=2025-02-17%2003%3A05%3A03

Might be relevant that they are both aarch64?

Just logged into the host. The logs of the timed out run are still
around, and the last information I can see is from lastcommand.log,
which seems to have frozen in time when the timeout has begun its
vacuuming work:
ok 73 + index_including_gist 353 ms
# parallel group (16 tests): create_cast errors create_aggregate drop_if_exists infinite_recurse

gokiburi is on the same host, and it is currently frozen in time when
trying to fetch a WAL buffer. One of the stack traces:
#2 0x000000000084ec48 in WaitEventSetWaitBlock (set=0xd34ce0,
cur_timeout=-1, occurred_events=0xffffffffadd8, nevents=1) at
latch.c:1571
#3 WaitEventSetWait (set=0xd34ce0, timeout=-1,
occurred_events=occurred_events@entry=0xffffffffadd8,
nevents=nevents@entry=1, wait_event_info=<optimized out>,
wait_event_info@entry=134217781) at latch.c:1519
#4 0x000000000084e964 in WaitLatch (latch=<optimized out>,
wakeEvents=wakeEvents@entry=33, timeout=timeout@entry=-1,
wait_event_info=wait_event_info@entry=134217781) at latch.c:538
#5 0x000000000085d2f8 in ConditionVariableTimedSleep
(cv=0xffffec0799b0, timeout=-1, wait_event_info=134217781) at
condition_variable.c:163
#6 0x000000000085d1ec in ConditionVariableSleep
(cv=0xfffffffffffffffc, wait_event_info=1) at condition_variable.c:98
#7 0x000000000055f4f4 in AdvanceXLInsertBuffer
(upto=upto@entry=112064880, tli=tli@entry=1, opportunistic=false) at
xlog.c:2224
#8 0x0000000000568398 in GetXLogBuffer (ptr=ptr@entry=112064880,
tli=tli@entry=1) at xlog.c:1710
#9 0x000000000055c650 in CopyXLogRecordToWAL (write_len=80,
isLogSwitch=false, rdata=0xcc49b0 <hdr_rdt>, StartPos=<optimized out>,
EndPos=<optimized out>, tli=1) at xlog.c:1245
#10 XLogInsertRecord (rdata=rdata@entry=0xcc49b0 <hdr_rdt>,
fpw_lsn=fpw_lsn@entry=112025520, flags=0 '\000', num_fpi=<optimized
out>, num_fpi@entry=0, topxid_included=false) at xlog.c:928
#11 0x000000000056b870 in XLogInsert (rmid=rmid@entry=16 '\020',
info=<optimized out>, info@entry=16 '\020') at xloginsert.c:523
#12 0x0000000000537acc in addLeafTuple (index=0xffffebf32950,
state=0xffffffffd5e0, leafTuple=0xe43870, current=<optimized out>,
parent=<optimized out>,

So, yes, something looks really wrong with this patch. Sounds
plausible to me that some other buildfarm animals could be stuck
without their owners knowing about it. It's proving to be a good idea
to force a timeout value in the configuration file of these animals..
--
Michael

#28Alexander Korotkov
aekorotkov@gmail.com
In reply to: Michael Paquier (#27)
Re: Get rid of WALBufMappingLock

On Tue, Feb 18, 2025 at 2:21 AM Michael Paquier <michael@paquier.xyz> wrote:

On Mon, Feb 17, 2025 at 11:25:05AM -0500, Tom Lane wrote:

This timeout failure on hachi looks suspicious as well:

https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hachi&amp;dt=2025-02-17%2003%3A05%3A03

Might be relevant that they are both aarch64?

Just logged into the host. The logs of the timed out run are still
around, and the last information I can see is from lastcommand.log,
which seems to have frozen in time when the timeout has begun its
vacuuming work:
ok 73 + index_including_gist 353 ms
# parallel group (16 tests): create_cast errors create_aggregate drop_if_exists infinite_recurse

gokiburi is on the same host, and it is currently frozen in time when
trying to fetch a WAL buffer. One of the stack traces:
#2 0x000000000084ec48 in WaitEventSetWaitBlock (set=0xd34ce0,
cur_timeout=-1, occurred_events=0xffffffffadd8, nevents=1) at
latch.c:1571
#3 WaitEventSetWait (set=0xd34ce0, timeout=-1,
occurred_events=occurred_events@entry=0xffffffffadd8,
nevents=nevents@entry=1, wait_event_info=<optimized out>,
wait_event_info@entry=134217781) at latch.c:1519
#4 0x000000000084e964 in WaitLatch (latch=<optimized out>,
wakeEvents=wakeEvents@entry=33, timeout=timeout@entry=-1,
wait_event_info=wait_event_info@entry=134217781) at latch.c:538
#5 0x000000000085d2f8 in ConditionVariableTimedSleep
(cv=0xffffec0799b0, timeout=-1, wait_event_info=134217781) at
condition_variable.c:163
#6 0x000000000085d1ec in ConditionVariableSleep
(cv=0xfffffffffffffffc, wait_event_info=1) at condition_variable.c:98
#7 0x000000000055f4f4 in AdvanceXLInsertBuffer
(upto=upto@entry=112064880, tli=tli@entry=1, opportunistic=false) at
xlog.c:2224
#8 0x0000000000568398 in GetXLogBuffer (ptr=ptr@entry=112064880,
tli=tli@entry=1) at xlog.c:1710
#9 0x000000000055c650 in CopyXLogRecordToWAL (write_len=80,
isLogSwitch=false, rdata=0xcc49b0 <hdr_rdt>, StartPos=<optimized out>,
EndPos=<optimized out>, tli=1) at xlog.c:1245
#10 XLogInsertRecord (rdata=rdata@entry=0xcc49b0 <hdr_rdt>,
fpw_lsn=fpw_lsn@entry=112025520, flags=0 '\000', num_fpi=<optimized
out>, num_fpi@entry=0, topxid_included=false) at xlog.c:928
#11 0x000000000056b870 in XLogInsert (rmid=rmid@entry=16 '\020',
info=<optimized out>, info@entry=16 '\020') at xloginsert.c:523
#12 0x0000000000537acc in addLeafTuple (index=0xffffebf32950,
state=0xffffffffd5e0, leafTuple=0xe43870, current=<optimized out>,
parent=<optimized out>,

So, yes, something looks really wrong with this patch. Sounds
plausible to me that some other buildfarm animals could be stuck
without their owners knowing about it. It's proving to be a good idea
to force a timeout value in the configuration file of these animals..

Tom, Michael, thank you for the information.
This patch will be better tested before next attempt.

------
Regards,
Alexander Korotkov
Supabase

#29Alexander Korotkov
aekorotkov@gmail.com
In reply to: Alexander Korotkov (#28)
1 attachment(s)
Re: Get rid of WALBufMappingLock

On Tue, Feb 18, 2025 at 2:29 AM Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Tue, Feb 18, 2025 at 2:21 AM Michael Paquier <michael@paquier.xyz> wrote:

On Mon, Feb 17, 2025 at 11:25:05AM -0500, Tom Lane wrote:

This timeout failure on hachi looks suspicious as well:

https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=hachi&amp;dt=2025-02-17%2003%3A05%3A03

Might be relevant that they are both aarch64?

Just logged into the host. The logs of the timed out run are still
around, and the last information I can see is from lastcommand.log,
which seems to have frozen in time when the timeout has begun its
vacuuming work:
ok 73 + index_including_gist 353 ms
# parallel group (16 tests): create_cast errors create_aggregate drop_if_exists infinite_recurse

gokiburi is on the same host, and it is currently frozen in time when
trying to fetch a WAL buffer. One of the stack traces:
#2 0x000000000084ec48 in WaitEventSetWaitBlock (set=0xd34ce0,
cur_timeout=-1, occurred_events=0xffffffffadd8, nevents=1) at
latch.c:1571
#3 WaitEventSetWait (set=0xd34ce0, timeout=-1,
occurred_events=occurred_events@entry=0xffffffffadd8,
nevents=nevents@entry=1, wait_event_info=<optimized out>,
wait_event_info@entry=134217781) at latch.c:1519
#4 0x000000000084e964 in WaitLatch (latch=<optimized out>,
wakeEvents=wakeEvents@entry=33, timeout=timeout@entry=-1,
wait_event_info=wait_event_info@entry=134217781) at latch.c:538
#5 0x000000000085d2f8 in ConditionVariableTimedSleep
(cv=0xffffec0799b0, timeout=-1, wait_event_info=134217781) at
condition_variable.c:163
#6 0x000000000085d1ec in ConditionVariableSleep
(cv=0xfffffffffffffffc, wait_event_info=1) at condition_variable.c:98
#7 0x000000000055f4f4 in AdvanceXLInsertBuffer
(upto=upto@entry=112064880, tli=tli@entry=1, opportunistic=false) at
xlog.c:2224
#8 0x0000000000568398 in GetXLogBuffer (ptr=ptr@entry=112064880,
tli=tli@entry=1) at xlog.c:1710
#9 0x000000000055c650 in CopyXLogRecordToWAL (write_len=80,
isLogSwitch=false, rdata=0xcc49b0 <hdr_rdt>, StartPos=<optimized out>,
EndPos=<optimized out>, tli=1) at xlog.c:1245
#10 XLogInsertRecord (rdata=rdata@entry=0xcc49b0 <hdr_rdt>,
fpw_lsn=fpw_lsn@entry=112025520, flags=0 '\000', num_fpi=<optimized
out>, num_fpi@entry=0, topxid_included=false) at xlog.c:928
#11 0x000000000056b870 in XLogInsert (rmid=rmid@entry=16 '\020',
info=<optimized out>, info@entry=16 '\020') at xloginsert.c:523
#12 0x0000000000537acc in addLeafTuple (index=0xffffebf32950,
state=0xffffffffd5e0, leafTuple=0xe43870, current=<optimized out>,
parent=<optimized out>,

So, yes, something looks really wrong with this patch. Sounds
plausible to me that some other buildfarm animals could be stuck
without their owners knowing about it. It's proving to be a good idea
to force a timeout value in the configuration file of these animals..

Tom, Michael, thank you for the information.
This patch will be better tested before next attempt.

It seems that I managed to reproduce the issue on my Raspberry PI 4.
After running our test suite in a loop for 2 days I found one timeout.

I have hypothesis on why it might happen. We don't have protection
against two backends in parallel get ReservedPtr mapped to a single
XLog buffer. I've talked to Yura off-list about that. He pointer out
that XLogWrite() should issue a PANIC in that case, which we didn't
observe. However, I'm not sure this analysis is complete.

One way or another, we need protection against this situation any way.
The updated patch is attached. Now, after acquiring ReservedPtr it
waits till OldPageRqstPtr gets initialized. Additionally I've to
implement more accurate calculation of OldPageRqstPtr. I run tests
with new patch on my Raspberry in a loop. Let's see how it goes.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v11-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v11-0001-Get-rid-of-WALBufMappingLock.patchDownload
From ea01c32b547cd11c52461aabcc93443d5d312cf9 Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Mon, 17 Feb 2025 04:19:01 +0200
Subject: [PATCH v11] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization using XLogCtl->InitializeReserved,
 * ensure the page is written out,
 * once the page is initialized, try to advance XLogCtl->InitializedUpTo and
   signal to waiters using XLogCtl->InitializedUpToCondVar condition
   variable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a
   XLogCtl->InitializedUpToCondVar.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: Yura Sokolov <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 184 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 139 insertions(+), 49 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 919314f8258..d21c2b8ada3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -301,11 +301,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -472,21 +467,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * First initialized page in the cache (first byte position).
+	 */
+	XLogRecPtr	InitializedFrom;
+
+	/*
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first.  To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -809,9 +820,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1990,32 +2001,78 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLogCtl->InitializedFrom + XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/*
+		 * Wait till page gets correctly initialized up to OldPageRqstPtr.
+		 */
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
+		while (pg_atomic_read_u64(&XLogCtl->InitializedUpTo) < OldPageRqstPtr)
+			ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+		ConditionVariableCancelSleep();
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2030,14 +2087,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2059,9 +2108,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					pgWalUsage.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2069,11 +2115,9 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
-
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
@@ -2138,11 +2182,50 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5068,6 +5151,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6087,7 +6174,8 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		XLogCtl->InitializedFrom = endOfRecoveryInfo->lastPageBeginPtr;
 	}
 	else
 	{
@@ -6096,8 +6184,10 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
+		XLogCtl->InitializedFrom = EndOfLog;
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

#30Michael Paquier
michael@paquier.xyz
In reply to: Alexander Korotkov (#29)
Re: Get rid of WALBufMappingLock

On Tue, Feb 25, 2025 at 05:19:29PM +0200, Alexander Korotkov wrote:

It seems that I managed to reproduce the issue on my Raspberry PI 4.
After running our test suite in a loop for 2 days I found one timeout.

Hmm. It's surprising to not see a higher occurence. My buildfarm
host has caught that on its first run after the patch, for two
different animals which are both on the same machine.

One way or another, we need protection against this situation any way.
The updated patch is attached. Now, after acquiring ReservedPtr it
waits till OldPageRqstPtr gets initialized. Additionally I've to
implement more accurate calculation of OldPageRqstPtr. I run tests
with new patch on my Raspberry in a loop. Let's see how it goes.

Perhaps you'd prefer that I do more tests with your patch? This is
time-consuming for you. This is not a review of the internals of the
patch, and I cannot give you access to the host, but if my stuff is
the only place where we have a good reproducibility of the issue, I'm
OK to grab some time and run a couple of checks to avoid again a
freeze of the buildfarm.
--
Michael

#31Andrey Borodin
x4mmm@yandex-team.ru
In reply to: Alexander Korotkov (#29)
Re: Get rid of WALBufMappingLock

On 25 Feb 2025, at 20:19, Alexander Korotkov <aekorotkov@gmail.com> wrote:

Hi!

One little piece of code looks suspicious to me. But I was not raising concern because I see similar code everywhere in the codebase. But know Kirill asked to me explain what is going on and I cannot.

This seems to be relevant… so.

+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
   // Assume ConditionVariableBroadcast() happened here, but before next line
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();

Won’t this sleep wait forever?

I see about 20 other occurrences of similar code, so, perhaps, everything is fine. But I would greatly appreciate a little pointers on why it works.

Best regards, Andrey Borodin.

#32Alexander Korotkov
aekorotkov@gmail.com
In reply to: Michael Paquier (#30)
1 attachment(s)
Re: Get rid of WALBufMappingLock

Hi, Michael!

On Wed, Feb 26, 2025 at 3:04 AM Michael Paquier <michael@paquier.xyz> wrote:

On Tue, Feb 25, 2025 at 05:19:29PM +0200, Alexander Korotkov wrote:

It seems that I managed to reproduce the issue on my Raspberry PI 4.
After running our test suite in a loop for 2 days I found one timeout.

Hmm. It's surprising to not see a higher occurence. My buildfarm
host has caught that on its first run after the patch, for two
different animals which are both on the same machine.

One way or another, we need protection against this situation any way.
The updated patch is attached. Now, after acquiring ReservedPtr it
waits till OldPageRqstPtr gets initialized. Additionally I've to
implement more accurate calculation of OldPageRqstPtr. I run tests
with new patch on my Raspberry in a loop. Let's see how it goes.

Perhaps you'd prefer that I do more tests with your patch? This is
time-consuming for you. This is not a review of the internals of the
patch, and I cannot give you access to the host, but if my stuff is
the only place where we have a good reproducibility of the issue, I'm
OK to grab some time and run a couple of checks to avoid again a
freeze of the buildfarm.

Thank you for offering the help. Updated version of patch is attached
(I've added one memory barrier there just in case). I would
appreciate if you could run it on batta, hachi or similar hardware.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v12-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v12-0001-Get-rid-of-WALBufMappingLock.patchDownload
From ab8579e92438487e32abf91ae3c1d36683511d52 Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Mon, 17 Feb 2025 04:19:01 +0200
Subject: [PATCH v12] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization using XLogCtl->InitializeReserved,
 * ensure the page is written out,
 * once the page is initialized, try to advance XLogCtl->InitializedUpTo and
   signal to waiters using XLogCtl->InitializedUpToCondVar condition
   variable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a
   XLogCtl->InitializedUpToCondVar.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: Yura Sokolov <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 186 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 141 insertions(+), 49 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 799fc739e18..43792ef3a5c 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * First initialized page in the cache (first byte position).
+	 */
+	XLogRecPtr	InitializedFrom;
+
+	/*
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first.  To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +821,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +2002,78 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 */
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLogCtl->InitializedFrom + XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/*
+		 * Wait till page gets correctly initialized up to OldPageRqstPtr.
+		 */
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
+		while (pg_atomic_read_u64(&XLogCtl->InitializedUpTo) < OldPageRqstPtr)
+			ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+		ConditionVariableCancelSleep();
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2088,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2109,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					pgWalUsage.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,11 +2116,9 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
-
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
@@ -2139,11 +2183,52 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
+
+	pg_read_barrier();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5069,6 +5154,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6088,7 +6177,8 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		XLogCtl->InitializedFrom = endOfRecoveryInfo->lastPageBeginPtr;
 	}
 	else
 	{
@@ -6097,8 +6187,10 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
+		XLogCtl->InitializedFrom = EndOfLog;
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

#33Michael Paquier
michael@paquier.xyz
In reply to: Alexander Korotkov (#32)
Re: Get rid of WALBufMappingLock

On Wed, Feb 26, 2025 at 01:48:47PM +0200, Alexander Korotkov wrote:

Thank you for offering the help. Updated version of patch is attached
(I've added one memory barrier there just in case). I would
appreciate if you could run it on batta, hachi or similar hardware.

Doing a revert of the revert done in 3fb58625d18f proves that
reproducing the error is not really difficult. I've done a make
installcheck-world USE_MODULE_DB=1 -j N without assertions, and the
point that saw a failure quickly is one of the tests of pgbench:
PANIC: could not find WAL buffer for 0/19D366

This one happened for the test "concurrent OID generation" and CREATE
TYPE. Of course, as it is a race condition, it is random, but it's
taking me only a couple of minutes to see the original issue on my
buildfarm host. With assertion failures enabled, same story, and same
failure from the pgbench TAP test.

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

One thing I would consider doing if you want to gain confidence is
tests like the one I saw causing problems with pgbench, with DDL
patterns stressing specific paths like this CREATE TYPE case.
--
Michael

#34Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Andrey Borodin (#31)
Re: Get rid of WALBufMappingLock

26.02.2025 11:52, Andrey Borodin wrote:

On 25 Feb 2025, at 20:19, Alexander Korotkov <aekorotkov@gmail.com> wrote:

Hi!

One little piece of code looks suspicious to me. But I was not raising concern because I see similar code everywhere in the codebase. But know Kirill asked to me explain what is going on and I cannot.

This seems to be relevant… so.

+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
// Assume ConditionVariableBroadcast() happened here, but before next line
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();

Won’t this sleep wait forever?

Because ConditionVariableSleep doesn't sleep for the first time.
It just performs ConditionVariablePrepareToSleep and immediately returns.
So actual condition of `while` loop is checked at least twice before going
to sleep.

I see about 20 other occurrences of similar code, so, perhaps, everything is fine. But I would greatly appreciate a little pointers on why it works.

-------
regards
Yura Sokolov aka funny-falcon

#35Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Alexander Korotkov (#32)
Re: Get rid of WALBufMappingLock

26.02.2025 14:48, Alexander Korotkov пишет:

Hi, Michael!

On Wed, Feb 26, 2025 at 3:04 AM Michael Paquier <michael@paquier.xyz> wrote:

On Tue, Feb 25, 2025 at 05:19:29PM +0200, Alexander Korotkov wrote:

It seems that I managed to reproduce the issue on my Raspberry PI 4.
After running our test suite in a loop for 2 days I found one timeout.

Hmm. It's surprising to not see a higher occurence. My buildfarm
host has caught that on its first run after the patch, for two
different animals which are both on the same machine.

One way or another, we need protection against this situation any way.
The updated patch is attached. Now, after acquiring ReservedPtr it
waits till OldPageRqstPtr gets initialized. Additionally I've to
implement more accurate calculation of OldPageRqstPtr. I run tests
with new patch on my Raspberry in a loop. Let's see how it goes.

Perhaps you'd prefer that I do more tests with your patch? This is
time-consuming for you. This is not a review of the internals of the
patch, and I cannot give you access to the host, but if my stuff is
the only place where we have a good reproducibility of the issue, I'm
OK to grab some time and run a couple of checks to avoid again a
freeze of the buildfarm.

Thank you for offering the help. Updated version of patch is attached
(I've added one memory barrier there just in case). I would
appreciate if you could run it on batta, hachi or similar hardware.

Good day, Alexander.

Checked your additions to patch. They're clear and robust.

-------
regards
Yura Sokolov aka funny-falcon

#36Álvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Michael Paquier (#33)
Re: Get rid of WALBufMappingLock

On 2025-Feb-28, Michael Paquier wrote:

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

Heh, no amount of testing can prove lack of bugs; but for sure "it looks
different now, so it must be correct" must be the weakest proof of
correctness I've heard of!

--
Álvaro Herrera Breisgau, Deutschland — https://www.EnterpriseDB.com/
"I am amazed at [the pgsql-sql] mailing list for the wonderful support, and
lack of hesitasion in answering a lost soul's question, I just wished the rest
of the mailing list could be like this." (Fotis)
/messages/by-id/200606261359.k5QDxE2p004593@auth-smtp.hol.gr

#37Michael Paquier
michael@paquier.xyz
In reply to: Álvaro Herrera (#36)
Re: Get rid of WALBufMappingLock

On Fri, Feb 28, 2025 at 02:13:23PM +0100, Álvaro Herrera wrote:

On 2025-Feb-28, Michael Paquier wrote:

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

Heh, no amount of testing can prove lack of bugs; but for sure "it looks
different now, so it must be correct" must be the weakest proof of
correctness I've heard of!

Err, okay. I did use the word "stable" with tests rather than
"correct", and I implied upthread that I did not check the correctness
nor the internals of the patch. If my words held the meaning you
are implying, well, my apologies for the confusion, I guess. I only
tested the patch and it was stable while I've noticed a few diffs with
the previous version, but I did *not* check its internals at all, nor
do I mean that I endorse its logic. I hope that's clear now.
--
Michael

#38Alexander Korotkov
aekorotkov@gmail.com
In reply to: Álvaro Herrera (#36)
Re: Get rid of WALBufMappingLock

On Fri, Feb 28, 2025 at 3:13 PM Álvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2025-Feb-28, Michael Paquier wrote:

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

Heh, no amount of testing can prove lack of bugs; but for sure "it looks
different now, so it must be correct" must be the weakest proof of
correctness I've heard of!

Michael just volunteered to help Yura and me with testing. He wan't
intended to be reviewer. And he reported that tests looks much more
stable now. I think he is absolutely correct with this.

------
Regards,
Alexander Korotkov
Supabase

#39Alexander Korotkov
aekorotkov@gmail.com
In reply to: Michael Paquier (#37)
Re: Get rid of WALBufMappingLock

On Fri, Feb 28, 2025 at 3:44 PM Michael Paquier <michael@paquier.xyz> wrote:

On Fri, Feb 28, 2025 at 02:13:23PM +0100, Álvaro Herrera wrote:

On 2025-Feb-28, Michael Paquier wrote:

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

Heh, no amount of testing can prove lack of bugs; but for sure "it looks
different now, so it must be correct" must be the weakest proof of
correctness I've heard of!

Err, okay. I did use the word "stable" with tests rather than
"correct", and I implied upthread that I did not check the correctness
nor the internals of the patch. If my words held the meaning you
are implying, well, my apologies for the confusion, I guess. I only
tested the patch and it was stable while I've noticed a few diffs with
the previous version, but I did *not* check its internals at all, nor
do I mean that I endorse its logic. I hope that's clear now.

Got it. Michael, thank you very much for your help.

------
Regards,
Alexander Korotkov
Supabase

#40Alexander Korotkov
aekorotkov@gmail.com
In reply to: Alexander Korotkov (#38)
1 attachment(s)
Re: Get rid of WALBufMappingLock

On Sun, Mar 2, 2025 at 1:58 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Fri, Feb 28, 2025 at 3:13 PM Álvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2025-Feb-28, Michael Paquier wrote:

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

Heh, no amount of testing can prove lack of bugs; but for sure "it looks
different now, so it must be correct" must be the weakest proof of
correctness I've heard of!

Michael just volunteered to help Yura and me with testing. He wan't
intended to be reviewer. And he reported that tests looks much more
stable now. I think he is absolutely correct with this.

Nevertheless, I don't think the bug has gone in v12. I managed to
reproduce it on my local Raspberry PI 4. The attached version of
patch fixes the bug for me. It adds memory barriers surrounding
pg_atomic_compare_exchange_u64(). That certainly not right given this
function should already provide full memory barrier semantics. But my
investigation shows it doesn't. I'm going to start a separate thread
about this.

Also, new version of patch contains fix of potential integer overflow
during OldPageRqstPtr computation sent off-list my me by Yura.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v13-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v13-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 6b97cdbaf055a46fe69d78d4eed3d4693b058151 Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Mon, 17 Feb 2025 04:19:01 +0200
Subject: [PATCH v13] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization using XLogCtl->InitializeReserved,
 * ensure the page is written out,
 * once the page is initialized, try to advance XLogCtl->InitializedUpTo and
   signal to waiters using XLogCtl->InitializedUpToCondVar condition
   variable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a
   XLogCtl->InitializedUpToCondVar.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: Yura Sokolov <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 190 +++++++++++++-----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 145 insertions(+), 49 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 799fc739e18..14c4853d7a9 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * First initialized page in the cache (first byte position).
+	 */
+	XLogRecPtr	InitializedFrom;
+
+	/*
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first.  To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +821,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +2002,78 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
+	 */
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLogCtl->InitializedFrom + XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - (XLogRecPtr) XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/*
+		 * Wait till page gets correctly initialized up to OldPageRqstPtr.
+		 */
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
+		while (pg_atomic_read_u64(&XLogCtl->InitializedUpTo) < OldPageRqstPtr)
+			ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+		ConditionVariableCancelSleep();
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2088,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2109,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					pgWalUsage.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,11 +2116,9 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
-
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
@@ -2139,11 +2183,56 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		pg_write_barrier();
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		pg_write_barrier();
+
+		/*
+		 * Try to advance XLogCtl->InitializedUpTo.
+		 *
+		 * If the CAS operation failed, then some of previous pages are not
+		 * initialized yet, and this backend gives up.
+		 *
+		 * Since initializer of next page might give up on advancing of
+		 * InitializedUpTo, this backend have to attempt advancing until it
+		 * find page "in the past" or concurrent backend succeeded at
+		 * advancing.  When we finish advancing XLogCtl->InitializedUpTo, we
+		 * notify all the waiters with XLogCtl->InitializedUpToCondVar.
+		 */
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			pg_read_barrier();
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
+
+	pg_read_barrier();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5069,6 +5158,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6088,7 +6181,8 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		XLogCtl->InitializedFrom = endOfRecoveryInfo->lastPageBeginPtr;
 	}
 	else
 	{
@@ -6097,8 +6191,10 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
+		XLogCtl->InitializedFrom = EndOfLog;
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

#41Alexander Korotkov
aekorotkov@gmail.com
In reply to: Alexander Korotkov (#40)
1 attachment(s)
Re: Get rid of WALBufMappingLock

On Fri, Mar 7, 2025 at 5:08 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Sun, Mar 2, 2025 at 1:58 PM Alexander Korotkov <aekorotkov@gmail.com> wrote:

On Fri, Feb 28, 2025 at 3:13 PM Álvaro Herrera <alvherre@alvh.no-ip.org> wrote:

On 2025-Feb-28, Michael Paquier wrote:

Saying that, I have also done similar tests with your v12 for a couple
of hours and this looks stable under installcheck-world. I can see
that you've reworked quite a bit the surroundings of InitializedFrom
in this one. If you apply that once again at some point, the
buildfarm will be judge in the long-term, but I am rather confident by
saying that the situation looks better here, at least.

Heh, no amount of testing can prove lack of bugs; but for sure "it looks
different now, so it must be correct" must be the weakest proof of
correctness I've heard of!

Michael just volunteered to help Yura and me with testing. He wan't
intended to be reviewer. And he reported that tests looks much more
stable now. I think he is absolutely correct with this.

Nevertheless, I don't think the bug has gone in v12. I managed to
reproduce it on my local Raspberry PI 4. The attached version of
patch fixes the bug for me. It adds memory barriers surrounding
pg_atomic_compare_exchange_u64(). That certainly not right given this
function should already provide full memory barrier semantics. But my
investigation shows it doesn't. I'm going to start a separate thread
about this.

Also, new version of patch contains fix of potential integer overflow
during OldPageRqstPtr computation sent off-list my me by Yura.

So, as we finally clarified CAS doesn't guarantee full memory barrier
on failure. Also, it's not clear when barriers are guaranteed on
success. In ARM without LSE implementation, read barrier is provided
before change of value and write barrier after change of value. So,
it appears that both explicit barriers I've added are required. This
revision also comes with format proof of the algorithm.

------
Regards,
Alexander Korotkov
Supabase

Attachments:

v14-0001-Get-rid-of-WALBufMappingLock.patchapplication/octet-stream; name=v14-0001-Get-rid-of-WALBufMappingLock.patchDownload
From 346cf0a8dd09d938c2fd71831866b7f15eb61f69 Mon Sep 17 00:00:00 2001
From: Alexander Korotkov <akorotkov@postgresql.org>
Date: Mon, 17 Feb 2025 04:19:01 +0200
Subject: [PATCH v14] Get rid of WALBufMappingLock

Allow multiple backends to initialize WAL buffers concurrently.  This way
`MemSet((char *) NewPage, 0, XLOG_BLCKSZ);` can run in parallel without
taking a single LWLock in exclusive mode.

The new algorithm works as follows:
 * reserve a page for initialization using XLogCtl->InitializeReserved,
 * ensure the page is written out,
 * once the page is initialized, try to advance XLogCtl->InitializedUpTo and
   signal to waiters using XLogCtl->InitializedUpToCondVar condition
   variable,
 * repeat previous steps until we reserve initialization up to the target
   WAL position,
 * wait until concurrent initialization finishes using a
   XLogCtl->InitializedUpToCondVar.

Now, multiple backends can, in parallel, concurrently reserve pages,
initialize them, and advance XLogCtl->InitializedUpTo to point to the latest
initialized page.

Author: Yura Sokolov <y.sokolov@postgrespro.ru>
Co-authored-by: Alexander Korotkov <aekorotkov@gmail.com>
Reviewed-by: Pavel Borisov <pashkin.elfe@gmail.com>
---
 src/backend/access/transam/xlog.c             | 234 ++++++++++++++----
 .../utils/activity/wait_event_names.txt       |   2 +-
 src/include/storage/lwlocklist.h              |   2 +-
 3 files changed, 189 insertions(+), 49 deletions(-)

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 799fc739e18..97bc52a6aac 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -302,11 +302,6 @@ static bool doPageWrites;
  * so it's a plain spinlock.  The other locks are held longer (potentially
  * over I/O operations), so we use LWLocks for them.  These locks are:
  *
- * WALBufMappingLock: must be held to replace a page in the WAL buffer cache.
- * It is only held while initializing and changing the mapping.  If the
- * contents of the buffer being replaced haven't been written yet, the mapping
- * lock is released while the write is done, and reacquired afterwards.
- *
  * WALWriteLock: must be held to write WAL buffers to disk (XLogWrite or
  * XLogFlush).
  *
@@ -473,21 +468,37 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * First initialized page in the cache (first byte position).
+	 */
+	XLogRecPtr	InitializedFrom;
+
+	/*
+	 * Latest reserved for inititalization page in the cache (last byte
+	 * position + 1).
 	 *
-	 * To change the identity of a buffer (and InitializedUpTo), you need to
-	 * hold WALBufMappingLock.  To change the identity of a buffer that's
+	 * To change the identity of a buffer, you need to advance
+	 * InitializeReserved first.  To change the identity of a buffer that's
 	 * still dirty, the old page needs to be written out first, and for that
 	 * you need WALWriteLock, and you need to ensure that there are no
 	 * in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	XLogRecPtr	InitializedUpTo;
+	pg_atomic_uint64 InitializeReserved;
+
+	/*
+	 * Latest initialized page in the cache (last byte position + 1).
+	 *
+	 * InitializedUpTo is updated after the buffer initialization.  After
+	 * update, waiters got notification using InitializedUpToCondVar.
+	 */
+	pg_atomic_uint64 InitializedUpTo;
+	ConditionVariable InitializedUpToCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
-	 * and xlblocks values certainly do.  xlblocks values are protected by
-	 * WALBufMappingLock.
+	 * and xlblocks values certainly do.  xlblocks values are changed
+	 * lock-free according to the check for the xlog write position and are
+	 * accompanied by changes of InitializeReserved and InitializedUpTo.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -810,9 +821,9 @@ XLogInsertRecord(XLogRecData *rdata,
 	 * fullPageWrites from changing until the insertion is finished.
 	 *
 	 * Step 2 can usually be done completely in parallel. If the required WAL
-	 * page is not initialized yet, you have to grab WALBufMappingLock to
-	 * initialize it, but the WAL writer tries to do that ahead of insertions
-	 * to avoid that from happening in the critical path.
+	 * page is not initialized yet, you have to go through AdvanceXLInsertBuffer,
+	 * which will ensure it is initialized. But the WAL writer tries to do that
+	 * ahead of insertions to avoid that from happening in the critical path.
 	 *
 	 *----------
 	 */
@@ -1991,32 +2002,79 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
 	XLogPageHeader NewPage;
+	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
-	LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-
 	/*
-	 * Now that we have the lock, check if someone initialized the page
-	 * already.
+	 * We must run the loop below inside the critical section as we expect
+	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
+	 * already run inside the critical section. Except for WAL writer, which
+	 * passed 'opportunistic == true', and therefore we don't perform
+	 * operations that could error out.
+	 *
+	 * Start an explicit critical section anyway though.
+	 */
+	Assert(CritSectionCount > 0 || opportunistic);
+	START_CRIT_SECTION();
+
+	/*--
+	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
+	 * initialization.  Multiple process can initialize different buffers with
+	 * this loop in parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * 2. Initialize the reserved page.
+	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
 	 */
-	while (upto >= XLogCtl->InitializedUpTo || opportunistic)
+	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
+	while (upto >= ReservedPtr || opportunistic)
 	{
-		nextidx = XLogRecPtrToBufIdx(XLogCtl->InitializedUpTo);
+		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
 
 		/*
-		 * Get ending-offset of the buffer page we need to replace (this may
-		 * be zero if the buffer hasn't been used yet).  Fall through if it's
-		 * already written out.
+		 * Get ending-offset of the buffer page we need to replace.
+		 *
+		 * We don't lookup into xlblocks, but rather calculate position we
+		 * must wait to be written. If it was written, xlblocks will have this
+		 * position (or uninitialized)
 		 */
-		OldPageRqstPtr = pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]);
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (ReservedPtr + XLOG_BLCKSZ > XLogCtl->InitializedFrom + XLOG_BLCKSZ * XLOGbuffers)
+			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - (XLogRecPtr) XLOG_BLCKSZ * XLOGbuffers;
+		else
+			OldPageRqstPtr = InvalidXLogRecPtr;
+
+		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
 		{
 			/*
-			 * Nope, got work to do. If we just want to pre-initialize as much
-			 * as we can without flushing, give up now.
+			 * If we just want to pre-initialize as much as we can without
+			 * flushing, give up now.
 			 */
-			if (opportunistic)
-				break;
+			upto = ReservedPtr - 1;
+			break;
+		}
+
+		/*
+		 * Attempt to reserve the page for initialization.  Failure means that
+		 * this page got reserved by another process.
+		 */
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
+											&ReservedPtr,
+											ReservedPtr + XLOG_BLCKSZ))
+			continue;
+
+		/*
+		 * Wait till page gets correctly initialized up to OldPageRqstPtr.
+		 */
+		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
+		while (pg_atomic_read_u64(&XLogCtl->InitializedUpTo) < OldPageRqstPtr)
+			ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+		ConditionVariableCancelSleep();
+		Assert(pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) == OldPageRqstPtr);
+
+		/* Fall through if it's already written out. */
+		if (LogwrtResult.Write < OldPageRqstPtr)
+		{
+			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
@@ -2031,14 +2089,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			RefreshXLogWriteResult(LogwrtResult);
 			if (LogwrtResult.Write < OldPageRqstPtr)
 			{
-				/*
-				 * Must acquire write lock. Release WALBufMappingLock first,
-				 * to make sure that all insertions that we need to wait for
-				 * can finish (up to this same position). Otherwise we risk
-				 * deadlock.
-				 */
-				LWLockRelease(WALBufMappingLock);
-
 				WaitXLogInsertionsToFinish(OldPageRqstPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
@@ -2060,9 +2110,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					pgWalUsage.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
-				/* Re-acquire WALBufMappingLock and retry */
-				LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
-				continue;
 			}
 		}
 
@@ -2070,11 +2117,9 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = XLogCtl->InitializedUpTo;
+		NewPageBeginPtr = ReservedPtr;
 		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
 
-		Assert(XLogRecPtrToBufIdx(NewPageBeginPtr) == nextidx);
-
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
 		/*
@@ -2138,12 +2183,100 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 */
 		pg_write_barrier();
 
+		/*-----
+		 * Update the value of XLogCtl->xlblocks[nextidx] and try to advance
+		 * XLogCtl->InitializedUpTo in a lock-less manner.
+		 *
+		 * First, let's provide a formal proof of the algorithm.  Let it be 'n'
+		 * process with the following variables in shared memory:
+		 *	f - an array of 'n' boolean flags,
+		 *	v - atomic integer variable.
+		 *
+		 * Also, let
+		 *	i - a number of a process,
+		 *	j - local integer variable,
+		 * CAS(var, oldval, newval) - compare-and-swap atomic operation
+		 *							  returning true on success,
+		 * write_barrier()/read_barrier() - memory barriers.
+		 *
+		 * The pseudocode for each process is the following.
+		 *
+		 *	j := i
+		 *	f[i] := true
+		 *	write_barrier()
+		 *	while CAS(v, j, j + 1):
+		 *		j := j + 1
+		 *		read_barrier()
+		 *		if not f[j]:
+		 *			break
+		 *
+		 * Let's prove that v eventually reaches the value of n.
+		 * 1. Prove by contradiction.  Assume v doesn't reach n and stucks
+		 *	  on k, where k < n.
+		 * 2. Process k attempts CAS(v, k, k + 1).  1). If, as we assumed, v
+		 *	  gets stuck at k, then this CAS operation must fail.  Therefore,
+		 *    v < k when process k attempts CAS(v, k, k + 1).
+		 * 3. If, as we assumed, v gets stuck at k, then the value k of v
+		 *	  must be achieved by some process m, where m < k.  The process
+		 *	  m must observe f[k] == false.  Otherwise, it will later attempt
+		 *	  CAS(v, k, k + 1) with success.
+		 * 4. Therefore, corresponding read_barrier() (while j == k) on
+		 *	  process m happend before write_barrier() of process k.  But then
+		 *	  process k attempts CAS(v, k, k + 1) after process m successfully
+		 *	  incremented v to k, and that CAS operation must succeed.
+		 *	  That leads to a contradiction.  So, there is no such k (k < n)
+		 *    where v gets stuck.  Q.E.D.
+		 *
+		 * To apply this proof to the code below, we assume
+		 * XLogCtl->InitializedUpTo will play the role of v with XLOG_BLCKSZ
+		 * granularity.  We also assume setting XLogCtl->xlblocks[nextidx] to
+		 * NewPageEndPtr to play the role of setting f[i] to true.  Also, note
+		 * that processes can't concurrently map different xlog locations to
+		 * the same nextidx because we previously requested that
+		 * XLogCtl->InitializedUpTo >= OldPageRqstPtr.  So, a xlog buffer can
+		 * be taken for initialization only once the previous initialization
+		 * takes effect on XLogCtl->InitializedUpTo.
+		 */
+
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
-		XLogCtl->InitializedUpTo = NewPageEndPtr;
+
+		pg_write_barrier();
+
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		{
+			NewPageBeginPtr = NewPageEndPtr;
+			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
+			nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+
+			pg_read_barrier();
+
+			if (pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) != NewPageEndPtr)
+			{
+				/*
+				 * Page at nextidx wasn't initialized yet, so we cann't move
+				 * InitializedUpto further. It will be moved by backend which
+				 * will initialize nextidx.
+				 */
+				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				break;
+			}
+		}
 
 		npages++;
 	}
-	LWLockRelease(WALBufMappingLock);
+
+	END_CRIT_SECTION();
+
+	/*
+	 * All the pages in WAL buffer before 'upto' were reserved for
+	 * initialization.  However, some pages might be reserved by concurrent
+	 * processes.  Wait till they finish initialization.
+	 */
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
+		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	ConditionVariableCancelSleep();
+
+	pg_read_barrier();
 
 #ifdef WAL_DEBUG
 	if (XLOG_DEBUG && npages > 0)
@@ -5069,6 +5202,10 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logWriteResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
+
+	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
 }
 
 /*
@@ -6088,7 +6225,8 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedUpTo = endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		XLogCtl->InitializedFrom = endOfRecoveryInfo->lastPageBeginPtr;
 	}
 	else
 	{
@@ -6097,8 +6235,10 @@ StartupXLOG(void)
 		 * let the first attempt to insert a log record to initialize the next
 		 * buffer.
 		 */
-		XLogCtl->InitializedUpTo = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
+		XLogCtl->InitializedFrom = EndOfLog;
 	}
+	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..ccf73781d81 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -155,6 +155,7 @@ REPLICATION_SLOT_DROP	"Waiting for a replication slot to become inactive so it c
 RESTORE_COMMAND	"Waiting for <xref linkend="guc-restore-command"/> to complete."
 SAFE_SNAPSHOT	"Waiting to obtain a valid snapshot for a <literal>READ ONLY DEFERRABLE</literal> transaction."
 SYNC_REP	"Waiting for confirmation from a remote server during synchronous replication."
+WAL_BUFFER_INIT	"Waiting on WAL buffer to be initialized."
 WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
@@ -310,7 +311,6 @@ XidGen	"Waiting to allocate a new transaction ID."
 ProcArray	"Waiting to access the shared per-process data structures (typically, to get a snapshot or report a session's transaction ID)."
 SInvalRead	"Waiting to retrieve messages from the shared catalog invalidation queue."
 SInvalWrite	"Waiting to add a message to the shared catalog invalidation queue."
-WALBufMapping	"Waiting to replace a page in WAL buffers."
 WALWrite	"Waiting for WAL buffers to be written to disk."
 ControlFile	"Waiting to read or update the <filename>pg_control</filename> file or create a new WAL file."
 MultiXactGen	"Waiting to read or update shared multixact state."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..ff897515769 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -37,7 +37,7 @@ PG_LWLOCK(3, XidGen)
 PG_LWLOCK(4, ProcArray)
 PG_LWLOCK(5, SInvalRead)
 PG_LWLOCK(6, SInvalWrite)
-PG_LWLOCK(7, WALBufMapping)
+/* 7 was WALBufMapping */
 PG_LWLOCK(8, WALWrite)
 PG_LWLOCK(9, ControlFile)
 /* 10 was CheckpointLock */
-- 
2.39.5 (Apple Git-154)

#42Tomas Vondra
tomas@vondra.me
In reply to: Alexander Korotkov (#40)
5 attachment(s)
Re: Get rid of WALBufMappingLock

Hi,

I've briefly looked at this patch this week, and done a bit of testing.
I don't have any comments about the correctness - it does seem correct
to me and I haven't noticed any crashes/issues, but I'm not familiar
with the WALBufMappingLock enough to have insightful opinions.

I have however decided to do a bit of benchmarking, to better understand
the possible benefits of the change. I happen to have access to an Azure
machine with 2x AMD EPYC 9V33X (176 cores in total), and NVMe SSD that
can do ~1.5GB/s.

The benchmark script (attached) uses the workload mentioned by Andres
some time ago [1]

SELECT pg_logical_emit_message(true, 'test', repeat('0', $SIZE));

with clients (1..196) and sizes 8K, 64K and 1024K. The aggregated
results look like this (this is throughput):

| 8 | 64 | 1024
clients | master patched | master patched | master patched
---------------------------------------------------------------------
1 | 11864 12035 | 7419 7345 | 968 940
4 | 26311 26919 | 12414 12308 | 1304 1293
8 | 38742 39651 | 14316 14539 | 1348 1348
16 | 57299 59917 | 15405 15871 | 1304 1279
32 | 74857 82598 | 17589 17126 | 1233 1233
48 | 87596 95495 | 18616 18160 | 1199 1227
64 | 89982 97715 | 19033 18910 | 1196 1221
96 | 92853 103448 | 19694 19706 | 1190 1210
128 | 95392 103324 | 20085 19873 | 1188 1213
160 | 94933 102236 | 20227 20323 | 1180 1214
196 | 95933 103341 | 20448 20513 | 1188 1199

To put this into a perspective, this throughput relative to master:

clients | 8 64 1024
----------------------------------
1 | 101% 99% 97%
4 | 102% 99% 99%
8 | 102% 102% 100%
16 | 105% 103% 98%
32 | 110% 97% 100%
48 | 109% 98% 102%
64 | 109% 99% 102%
96 | 111% 100% 102%
128 | 108% 99% 102%
160 | 108% 100% 103%
196 | 108% 100% 101%

That does not seem like a huge improvement :-( Yes, there's 1-10%
speedup for the small (8K) size, but for larger chunks it's a wash.

Looking at the pgbench progress, I noticed stuff like this:

...
progress: 13.0 s, 103575.2 tps, lat 0.309 ms stddev 0.071, 0 failed
progress: 14.0 s, 102685.2 tps, lat 0.312 ms stddev 0.072, 0 failed
progress: 15.0 s, 102853.9 tps, lat 0.311 ms stddev 0.072, 0 failed
progress: 16.0 s, 103146.0 tps, lat 0.310 ms stddev 0.075, 0 failed
progress: 17.0 s, 57168.1 tps, lat 0.560 ms stddev 0.153, 0 failed
progress: 18.0 s, 50495.9 tps, lat 0.634 ms stddev 0.060, 0 failed
progress: 19.0 s, 50927.0 tps, lat 0.628 ms stddev 0.066, 0 failed
progress: 20.0 s, 50986.7 tps, lat 0.628 ms stddev 0.062, 0 failed
progress: 21.0 s, 50652.3 tps, lat 0.632 ms stddev 0.061, 0 failed
progress: 22.0 s, 63792.9 tps, lat 0.502 ms stddev 0.168, 0 failed
progress: 23.0 s, 103109.9 tps, lat 0.310 ms stddev 0.072, 0 failed
progress: 24.0 s, 103503.8 tps, lat 0.309 ms stddev 0.071, 0 failed
progress: 25.0 s, 101984.2 tps, lat 0.314 ms stddev 0.073, 0 failed
progress: 26.0 s, 102923.1 tps, lat 0.311 ms stddev 0.072, 0 failed
progress: 27.0 s, 103973.1 tps, lat 0.308 ms stddev 0.072, 0 failed
...

i.e. it fluctuates a lot. I suspected this is due to the SSD doing funny
things (it's a virtual SSD, I'm not sure what model is that behind the
curtains). So I decided to try running the benchmark on tmpfs, to get
the storage out of the way and get the "best case" results.

This makes the pgbench progress perfectly "smooth" (no jumps like in the
output above), and the comparison looks like this:

| 8 | 64 | 1024
clients | master patched | master patched | master patched
---------|---------------------|--------------------|----------------
1 | 32449 32032 | 19289 20344 | 3108 3081
4 | 68779 69256 | 24585 29912 | 2915 3449
8 | 79787 100655 | 28217 39217 | 3182 4086
16 | 113024 148968 | 42969 62083 | 5134 5712
32 | 125884 170678 | 44256 71183 | 4910 5447
48 | 125571 166695 | 44693 76411 | 4717 5215
64 | 122096 160470 | 42749 83754 | 4631 5103
96 | 120170 154145 | 42696 86529 | 4556 5020
128 | 119204 152977 | 40880 88163 | 4529 5047
160 | 116081 152708 | 42263 88066 | 4512 5000
196 | 115364 152455 | 40765 88602 | 4505 4952

and the comparison to master:

clients 8 64 1024
-----------------------------------------
1 99% 105% 99%
4 101% 122% 118%
8 126% 139% 128%
16 132% 144% 111%
32 136% 161% 111%
48 133% 171% 111%
64 131% 196% 110%
96 128% 203% 110%
128 128% 216% 111%
160 132% 208% 111%
196 132% 217% 110%

Yes, with tmpfs the impact looks much more significant. For 8K the
speedup is ~1.3x, for 64K it's up to ~2x, for 1M it's ~1.1x.

That being said, I wonder how big is the impact for practical workloads.
ISTM this workload is pretty narrow / extreme, it'd be much easier if we
had an example of a more realistic workload, benefiting from this. Of
course, it may be the case that there are multiple related bottlenecks,
and we'd need to fix all of them - in which case it'd be silly to block
the improvements on the grounds that it alone does not help.

Another thought is that this is testing the "good case". Can anyone
think of a workload that would be made worse by the patch?

regards

--
Tomas Vondra

Attachments:

wal-lock-test.shapplication/x-shellscript; name=wal-lock-test.shDownload
patched-tmpfs.csvtext/csv; charset=UTF-8; name=patched-tmpfs.csvDownload
master-tmpfs.csvtext/csv; charset=UTF-8; name=master-tmpfs.csvDownload
master-ssd.csvtext/csv; charset=UTF-8; name=master-ssd.csvDownload
patched-ssd.csvtext/csv; charset=UTF-8; name=patched-ssd.csvDownload
#43Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Tomas Vondra (#42)
Re: Get rid of WALBufMappingLock

Good day, Tomas

14.03.2025 17:30, Tomas Vondra wrote:

Yes, with tmpfs the impact looks much more significant. For 8K the
speedup is ~1.3x, for 64K it's up to ~2x, for 1M it's ~1.1x.

That being said, I wonder how big is the impact for practical workloads.
ISTM this workload is pretty narrow / extreme, it'd be much easier if we
had an example of a more realistic workload, benefiting from this. Of
course, it may be the case that there are multiple related bottlenecks,
and we'd need to fix all of them - in which case it'd be silly to block
the improvements on the grounds that it alone does not help.

Yes, I found this bottleneck when I did experiments with increasing
NUM_XLOGINSERT_LOCKS [1]/messages/by-id/flat/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru. For this patch to be more valuable, there should
be more parallel xlog inserters.

That is why I initially paired this patch with patch that reduces
contention on WALInsertLocks ("0002-Several attempts to lock
WALInsertLock", last version at [2]/messages/by-id/c31158a3-7c26-4b26-90df-2df8f7bbe736@postgrespro.ru).

Certainly, largest bottleneck is WALWriteLock around writting buffers and
especially fsync-ing them after write. But this intermediate bottleneck of
WALBufMappingLock is also worth to be removed.

[1]: /messages/by-id/flat/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru
/messages/by-id/flat/3b11fdc2-9793-403d-b3d4-67ff9a00d447@postgrespro.ru
[2]: /messages/by-id/c31158a3-7c26-4b26-90df-2df8f7bbe736@postgrespro.ru

-------
regards
Yura Sokolov aka funny-falcon

#44Yura Sokolov
y.sokolov@postgrespro.ru
In reply to: Tomas Vondra (#42)
2 attachment(s)
Re: Get rid of WALBufMappingLock

Good day,

14.03.2025 17:30, Tomas Vondra wrote:

Hi,

I've briefly looked at this patch this week, and done a bit of testing.
I don't have any comments about the correctness - it does seem correct
to me and I haven't noticed any crashes/issues, but I'm not familiar
with the WALBufMappingLock enough to have insightful opinions.

I have however decided to do a bit of benchmarking, to better understand
the possible benefits of the change. I happen to have access to an Azure
machine with 2x AMD EPYC 9V33X (176 cores in total), and NVMe SSD that
can do ~1.5GB/s.

The benchmark script (attached) uses the workload mentioned by Andres
some time ago [1]

SELECT pg_logical_emit_message(true, 'test', repeat('0', $SIZE));

with clients (1..196) and sizes 8K, 64K and 1024K. The aggregated
results look like this (this is throughput):

| 8 | 64 | 1024
clients | master patched | master patched | master patched
---------------------------------------------------------------------
1 | 11864 12035 | 7419 7345 | 968 940
4 | 26311 26919 | 12414 12308 | 1304 1293
8 | 38742 39651 | 14316 14539 | 1348 1348
16 | 57299 59917 | 15405 15871 | 1304 1279
32 | 74857 82598 | 17589 17126 | 1233 1233
48 | 87596 95495 | 18616 18160 | 1199 1227
64 | 89982 97715 | 19033 18910 | 1196 1221
96 | 92853 103448 | 19694 19706 | 1190 1210
128 | 95392 103324 | 20085 19873 | 1188 1213
160 | 94933 102236 | 20227 20323 | 1180 1214
196 | 95933 103341 | 20448 20513 | 1188 1199

To put this into a perspective, this throughput relative to master:

clients | 8 64 1024
----------------------------------
1 | 101% 99% 97%
4 | 102% 99% 99%
8 | 102% 102% 100%
16 | 105% 103% 98%
32 | 110% 97% 100%
48 | 109% 98% 102%
64 | 109% 99% 102%
96 | 111% 100% 102%
128 | 108% 99% 102%
160 | 108% 100% 103%
196 | 108% 100% 101%

That does not seem like a huge improvement :-( Yes, there's 1-10%
speedup for the small (8K) size, but for larger chunks it's a wash.

Looking at the pgbench progress, I noticed stuff like this:

...
progress: 13.0 s, 103575.2 tps, lat 0.309 ms stddev 0.071, 0 failed
progress: 14.0 s, 102685.2 tps, lat 0.312 ms stddev 0.072, 0 failed
progress: 15.0 s, 102853.9 tps, lat 0.311 ms stddev 0.072, 0 failed
progress: 16.0 s, 103146.0 tps, lat 0.310 ms stddev 0.075, 0 failed
progress: 17.0 s, 57168.1 tps, lat 0.560 ms stddev 0.153, 0 failed
progress: 18.0 s, 50495.9 tps, lat 0.634 ms stddev 0.060, 0 failed
progress: 19.0 s, 50927.0 tps, lat 0.628 ms stddev 0.066, 0 failed
progress: 20.0 s, 50986.7 tps, lat 0.628 ms stddev 0.062, 0 failed
progress: 21.0 s, 50652.3 tps, lat 0.632 ms stddev 0.061, 0 failed
progress: 22.0 s, 63792.9 tps, lat 0.502 ms stddev 0.168, 0 failed
progress: 23.0 s, 103109.9 tps, lat 0.310 ms stddev 0.072, 0 failed
progress: 24.0 s, 103503.8 tps, lat 0.309 ms stddev 0.071, 0 failed
progress: 25.0 s, 101984.2 tps, lat 0.314 ms stddev 0.073, 0 failed
progress: 26.0 s, 102923.1 tps, lat 0.311 ms stddev 0.072, 0 failed
progress: 27.0 s, 103973.1 tps, lat 0.308 ms stddev 0.072, 0 failed
...

i.e. it fluctuates a lot. I suspected this is due to the SSD doing funny
things (it's a virtual SSD, I'm not sure what model is that behind the
curtains). So I decided to try running the benchmark on tmpfs, to get
the storage out of the way and get the "best case" results.

This makes the pgbench progress perfectly "smooth" (no jumps like in the
output above), and the comparison looks like this:

| 8 | 64 | 1024
clients | master patched | master patched | master patched
---------|---------------------|--------------------|----------------
1 | 32449 32032 | 19289 20344 | 3108 3081
4 | 68779 69256 | 24585 29912 | 2915 3449
8 | 79787 100655 | 28217 39217 | 3182 4086
16 | 113024 148968 | 42969 62083 | 5134 5712
32 | 125884 170678 | 44256 71183 | 4910 5447
48 | 125571 166695 | 44693 76411 | 4717 5215
64 | 122096 160470 | 42749 83754 | 4631 5103
96 | 120170 154145 | 42696 86529 | 4556 5020
128 | 119204 152977 | 40880 88163 | 4529 5047
160 | 116081 152708 | 42263 88066 | 4512 5000
196 | 115364 152455 | 40765 88602 | 4505 4952

and the comparison to master:

clients 8 64 1024
-----------------------------------------
1 99% 105% 99%
4 101% 122% 118%
8 126% 139% 128%
16 132% 144% 111%
32 136% 161% 111%
48 133% 171% 111%
64 131% 196% 110%
96 128% 203% 110%
128 128% 216% 111%
160 132% 208% 111%
196 132% 217% 110%

Yes, with tmpfs the impact looks much more significant. For 8K the
speedup is ~1.3x, for 64K it's up to ~2x, for 1M it's ~1.1x.

That being said, I wonder how big is the impact for practical workloads.
ISTM this workload is pretty narrow / extreme, it'd be much easier if we
had an example of a more realistic workload, benefiting from this. Of
course, it may be the case that there are multiple related bottlenecks,
and we'd need to fix all of them - in which case it'd be silly to block
the improvements on the grounds that it alone does not help.

Another thought is that this is testing the "good case". Can anyone
think of a workload that would be made worse by the patch?

I've made similar benchmark on system with two Xeon Gold 5220R with two
Samsung SSD 970 PRO 1TB mirrored by md.

Configuration changes:
wal_sync_method = open_datasync
full_page_writes = off
synchronous_commit = off
checkpoint_timeout = 1d
max_connections = 1000
max_wal_size = 4GB
min_wal_size = 640MB

I variated wal segment size (16MB and 64MB), wal_buffers (128kB, 16MB and
1GB) and record size (1kB, 8kB and 64kB).

(I didn't bench 1MB record size, since I don't believe it is critical for
performance).

Here's results for 64MB segment size and 1GB wal_buffers:

+---------+---------+------------+--------------+----------+
| recsize | clients | master_tps | nowalbuf_tps | rel_perf |
+---------+---------+------------+--------------+----------+
| 1       | 1       | 47991.0    | 46995.0      | 0.98     |
| 1       | 4       | 171930.0   | 171166.0     | 1.0      |
| 1       | 16      | 491240.0   | 485132.0     | 0.99     |
| 1       | 64      | 514590.0   | 515534.0     | 1.0      |
| 1       | 128     | 547222.0   | 543543.0     | 0.99     |
| 1       | 256     | 543353.0   | 540802.0     | 1.0      |
| 8       | 1       | 40976.0    | 41603.0      | 1.02     |
| 8       | 4       | 89003.0    | 92008.0      | 1.03     |
| 8       | 16      | 90457.0    | 92282.0      | 1.02     |
| 8       | 64      | 89293.0    | 92022.0      | 1.03     |
| 8       | 128     | 92687.0    | 92768.0      | 1.0      |
| 8       | 256     | 91874.0    | 91665.0      | 1.0      |
| 64      | 1       | 11829.0    | 12031.0      | 1.02     |
| 64      | 4       | 11959.0    | 12832.0      | 1.07     |
| 64      | 16      | 11331.0    | 13417.0      | 1.18     |
| 64      | 64      | 11108.0    | 13588.0      | 1.22     |
| 64      | 128     | 11089.0    | 13648.0      | 1.23     |
| 64      | 256     | 10381.0    | 13542.0      | 1.3      |
+---------+---------+------------+--------------+----------+

Numbers for all configurations in attached 'improvements.out' . It shows,
removing WALBufMappingLock almost always doesn't harm performance and
usually gives measurable gain.

(Numbers are average from 4 middle runs out of 6. i.e. I threw minimum and
maximum tps from 6 runs and took average from remaining).

Also sqlite database is attached with all results. It also contains results
for patch "Several attempts to lock WALInsertLock" (named "attempts") and
cumulative patch ("nowalbuf-attempts").
Suprisingly, "Several attempts" causes measurable impact in some
configurations with hundreds of clients. So, there're more bottlenecks ahead ))

Yes, it is still not "real-world" benchmark. But it at least shows patch is
harmless.

--
regards
Yura Sokolov aka funny-falcon

Attachments:

improvements.outtext/plain; charset=UTF-8; name=improvements.outDownload
results.db.zstapplication/zstd; name=results.db.zstDownload
(�/���)���_K`��8����_��v���_��G�K�)fF��P�xY�+
�y��	Y�jD��o$�X�\1���D��C�W�1-8hE2Q��Z�r.�=���_0U����F�`d�@<� 	�� �$0o���M-�Y,L`��	������������-�`0��-b���-{����w�%�`�G���4�Y=� '0��?��5���>{����(�;
�W`�Q
�D?s
�W��
��m�9i��=�P�@�i�g��N�Q�A�=���!�����0F1��.H+rH�$�
9��s�1No�{�%��N.
��8_YU���#dQ�8L%�s�	�����:�NN�3���s���s�a�����t:E��*t��	P9���-�Y�A�K������H�%U���N��U*����[i�`������*B!�������7�x�l��It8�$�~����%e�d�t�����4��X�)oL�*;fe��B|<{���y�Z|Y+��A��o�2�k�XO�����;�j�W�-�"j��$�/})����m�8 ��{I�<��x:�;��lR��;�Z5UK^��Y'���e&t�[��h9��U��@�1���{�y�>Q��S�mb��e��l>��@������1
k�xd�x{X��B@�q����0m���S&�hv�ac<^$j�<�rn�n�A�p�P�0�!�g���[�WP6��`y��B��]����Yl���n�N�#&$��4��U\�v����x%��.U����|�=��2I~jJ�������I������I���_7k��a-_bm��a���^J���v�����J���H�z", v���m���K�q��A�{�="4n��<PGq�l=�z��ok����Z
��[J��m��&�����R{�C�VZt�]iOP�K{G�"����[8�x2#����8�=��;���6���P6�������s�>F�Q���g�a�b"��-��,�\C�@�!O~��x��<��W�y8)�n
7��\�+pl)U)�J/��	�����d
1�L2�	��z>�HN�Ls�p�9y]G�t6�N��?J-�FA@/@��F���/�#�$W%'&�AN��9T�����m��������B���H��ei�F�����Z��Oj��D��3*�5���Hk�u�ZE"V1hgI��Q`n����k6��3��To�
.9]tP^��/j�k�;>�O0@k�xb@��BG��.+�U�����f�`�`}����YYXeXqX{X��xV&V(V+V�(�A�(�������|if�6*�6����-��-)�I��?d�?pSYX��OH���5����KlHe(�fv�o4d�O������F�o�2������P!V>��).�o
�\HT������4d�Xp�,"?~����������K��cb�E�B���EYk�����6lk�BKsScc���64;�1
��E	���������Q�����zL������>��)�"�)�SDz�
����|y
���������L�I���S�i~�J'��I��#�\�X�v��B���J]E�O(�-��'_����o�fDk+������.����3La�������Nq���a0#�$2�c��Jd���wKd��2�}/���@T�_-�/�gg�����]����$�b2�b�8�}�����
H6�6��[6����a;���i���x43�0��w�YU�|C�@�!w��_q2.�3<��C�}pL8*.N�
�!p�lKg�[��
ee7���i��'�3�s�pq�=�+�r������*��^�e(�J��	s^�,9298�:9(r-tO:-��n]::�#t=EJ��a:��%�?�{R��P��5UU�z������4�"+�
f�`Ua�����
�v|np[+n]�S`V�R�}�`�Y����
���X�����:�{�����h`���1�H����=|�9��%\�kP*�/g��[�G�q��{���h���v��;��}i�P���������#��@(C��'r1M�O�"���co�ni�H��������-��nne��� �������@��@	�v���(�)�]H��U����b�����b�G����T�1F�{lcb?��"���bi����b_-���"F�)��r1�����+��W\�����g��k�)�Ri��/@Q��������"��3�����F����b�n���#����#��:b��
�����y�_L��u��R��}*�C��G��&�U���P
��/D���ap�Wq�G�c�?��H��E{$n���-�/+�9�C������'-�n+�?\�g@����E�������@���+v�Psw$v3M!�����$~m�����= �%�[���I����+~c�[�$~�z_q�F���%{Xl��b�n�aG����rD�v����w�F$c��=��}�~�����b���j�F)�"��L# C � 7��\	���p7�A�����
�����;g��r�+��LB�CyE���s������t�\O���;����i�Ow��zY�;�.B��G��&����A��J�?�9�\��9r{r�KE'FG'�NL�RiW���j���*���$��*���55
huZ��OV.k�	��(n)8��]��Do.�o
c�n���c���E	
p��`TU���^������	��	��[�P{v�T��_�u��o
*��Y���$}`�����{x-`���R�c���v3��=Nv,���z�_���?��?��`M�D�OS����F�G��� �������n7Sn�zGp�����C%����n���V��	��a����G��PT'��n���
��{���Rp�D�������� p��5�^��n�XM�
��1����vL�n�t��.!��R��oQ9���[:�p��[���?��c�N�38��
~���������
~�����G@��VU�Ib\�f�����	�p��[jsm�
f_�W�C�T�(�L1�@ ��9��[�\��G���g��
����|Q
)�J/��RNy�$M#S����b2`~`�`>a��/N!'������s�9w'�5z�������,�G��-t
���5����P�%aC����@w�n��Z�ji)�4�T�U��*�
�f�8����
�j�i��5��U�#N1:3�2��������B'���Od ����@(4bG�cX�����d �5x���Ol��R�"�����_�������"�����@���&U��5X�y-���<�`�yy��Y=XYdv�b�"���\�Y��qf�h��7�����q�������e�<"��k�3�:�m��f�D@7��a:�+-����@[����_����~���O��b�D�-�������8o�DI;v�[����>�?����?7bof{��R4{f�F�B�7�-f�����������e�����mQ���b�I3k���fd�-�^�I�4{M�o�4��7����������La�>�.
���f�Mb���JK�K�n�����]�ig�����#E��_����[[�7f������0�\J�1�eK��E��yh
3<f�T=f^*��yO���Y�~��i�N5�6��y��f���Y�W3���} �WK�A��!I\���vY3O	�p�yM��Z3�4�W!�}kf�X0r���04w��)SXd�U~��6]������L�������%���-T2���m1������f���7��o�K�J�l�����2FF��x���xg�OJ�Z��)Y�I����@b^���>pn	����#�C���Q~�)m(�(�L*���b�1��B�Y���q9��
�p.q�zR�����^����t(��
���^P5h
�t��r7rGrRr��������������K�;A���
�P��I�R8M����I��j�M]���V������j��f�a�mE���I�����=�Y8�����Z�]J���j1;���k��%�TZ��3��.�-�0�����\J�6!����K%f�T�}����+�,��` �E��Y�ej�[1BQnG	����F����[(?.�.�prO���[�5e_����/��I�������������7��_��U�^�)��_������#�mfD �k�r��(Hn�����{�����Y��r/������H�����-Q�Hn��#�"h%��}���g��d�J�)��.]��;G�)n���*�+�#��<(�u���H~_@�B$�%�A,��<���d`G�Iv�8�5��������j!a���I����PbO���R%�^Q2���K��/�2�!�2��2o�2����z�7d��R�!5��s,�t�CGHB�<�"��y/���������B�CfS�)���!�2P�T�w��J�X����J�����"��� �/J��?`��y=���&-�e�����%{H\�G��93��.o~�[��'�����B�x�)�0q��<zr��^�xkE�d�����m���}�]�����xk������o=7(|�����l*[�n�g��0F!���e�`<a�a�b"c�C��!� � ?������7�u\�N
'7�\ZB�HJ�����_10=0g0�0����:�����,�����d�T'��:���B�o���@��W��5�y�5����
sp�L��
�����1�Y���m�sA�G����F���N���F��Sj�*FA-B]W��k��j��Y�+D*GhO#�=�$��RB�9���bB{^���	��|��������+bt{@��n�Kct�����)m����f�W+�N����m�3��A����F�0P��O�FC��)T�(t��6�'����O��R�v�)T�V�[��� ��=*WW����B�4����fY�v���^u���X�5�u��5��{Qe�c�����U�r�ob�~�����\����A8Vi�Z����Z�w���7��O\�����]�E�^T��*T�z��������ak�,��V8~��m��0p���PUT�iC_��03�=<����0`�`n�+xkVO��5�G�?W���I������T�S�� ��n�(���5E/��F�K�r��L�n��������0��m	��Q�:�n��4�v�5��������������� t���;
�u;dG�����{\W=�����:�M�-d�u�U��SU������3V~U��;���
��m��e	�{_kW�6���0��]�������6�1]�u�yE��\H5������1G�I�>p,8N	�J�Z2�U�X9�hJ�J)� ��b��^�	L�)L�Yu��bNN<�O���:��A����J��2��A��R�B9�W�-�@�MN��U�B'D���N%HQH����S�jU�����U���n�6�7�������	�g�m������l�T��~M���}~8�h��d��������m�R0I�_js+ElmnS�2�]M�m�nLQ�f��m�\R��q��K���$�T��
�Y��4t\�]$0���zF`�`�ev-�2��$"�����$5�/K��S�8��h�v8��IX��
L������ 1b�4����Q��M�h��v������*����+�c�we;��
h��0 �����i� �'D�K�hV	U��Y!X�3�i���O{��"=���DD�"������������~��:���1��p�/��-�#�?�H���D�S���Kt��D�lc��D{�)�����/�O���h����b����?���P�g[�E�_X&���j&�n��f�-j�A�0*�A�l�?�
����[9�GF�:h�h�i��k���]�����P��2A��_~�)��	�A����v����5+�����J��h�
*B�K�� �Kfn!��e�"f���:�~���E�,@�����w[�'��\��J�7Gq���U7B�q��	%�>��n��2��`��%4 �f0g���b����EccQ�FF��$IDH�N����{+C�p�!
�)�/������0���������d�a�p����7��Z��4�
��������)��������E����2-z ����rhM����
J�,�8.,�').)��.m���B7�T��A.%�@�l��	��buZU�Pl����@r_)	��q�1+v������y-R���_;gPML�L/)j������CLxz�|^u}EF��9�N��cRI-^����TUU�8*�6�����������v��-��G�2��M��c��E
���
jS�����%
x��:)_OU/f�/���y�^Ls���E����@�)8�Z��JU����H����2Tc4	�+��xI������_��!�xP7b���Ut�����b�A�\qp�d���xc1���N��%�a���Q���&5���q�����K���-�?�z
��p����i��0y�P��7h63��D�|�p�n�d�kPtC����3����S�yN��R�]�R�s�d�N����R���JZ
�����+*#Ni�EsV51i�&�3?�C��Wl����tl�����w�G�S�J��A`��Hf�Gj�F�3p����`�,�1��J64��v���(��G�p:���iX�/,��x�4X� ����d]0���TpRZ����ampLZ��7c���������px O�S�.m���,�������Y����$��T�F
����(�|�%�B��z�(P�����HV>�+����'e.G���3{x�����4&��K���!�=��;��^��A:����(�.����>�q�Z�}L^7�G��Qa>!C�<{N���>E���)+� A(U �������������V��u��1Q��L������Q�������~��}�KB��� R�g+!qx�����?t@H&���O23��x�c��8�~��=��\�k��������6Xa�Tz�%���PN�#�X��m�9��^AX�M<"+���n�Zb��`��/��El��`
C����3?�����x�,�����`s���"�'�2��9v[���~uZ�&��X�)��e/��y��r�&8y	S�8F��J���@]�#s��x�E"��K���v��C�4�� 3J���~��I������0PJ4����p��E��b�O�����hm��{�p5m�<���J{��|zX����G�>"�4O�}�����4��b��������-~
���>"�+���D�(��v�?��a���F��P]�2�:8���;��w��f<M?_09���(5�Y'��S���E���W��)�����/o*��8����P���0/�� @��Bl�b�K�4$c�����y�3"��1����2M�pS�����vJ�������w	��b��1	�J|��s�.�	J�N+�:�F�����p@F�XE�Tu�{�1�3�qEAF5����2�����p�k�v���
��B*HD���{��I$�C���Q_�$$��"�."�u/�}��Z�E�h�hU,��w�K�>������C�x���C]�m��%GjPn�����������!�~��}��,�!�4=o������@&�V@��j\j�2Fh�"eFk�K�~Z
+����}Q��b��z;S�  ��s����&*)_G{�Q���!��O� ~#����P�*�������/7�
��td�E-gB6h����������h���F�������>�G�DV���8RL%��F�O��0U������(�/�U"W�u�4C����,�x�{Y7$g`tf�3�@'�3�R,1K������,��&%G��.�b,�%p1_yH�Y��5�(Q���XQ�,�d�r�D:�N��*�^�d�"���B8�F���dt&�����K
��At��U��=|���H����-�b�k8���aY�Y$�����D$!l��H����b���
^F��8��@��	�e&V����:����]C?��]���1���I0"/^����`LsU�K�(���?B���$w\A�
�E:����,Q�/x~� p��`�xQj��RS\�'V�%�d�{C���|4=��l��S����G�J������)��uo�����t%��p]�!B�}�\����oI=X�L�UQ���@���"��v�f!B��5R_K��l���21F�.���]�%����d��V?.�(b���I�H�W�d��|(NQ��P��"�c�W�@����D�qv$�X���� `�- 2>C��O����v
e��@�Y�j%���}+/��N�����\L�CD�UL%H(����������P�Y����6I������x0��$�,
���{����0�T������X.s�����N3��������K�P��E4�X�%t$����wm;��1K:���@V�!����H{��|j����{|N~��J�F��jm����S���x��we�:[����hz�c8���J�B[�������p��j�
����"�M�$h��J�I/��o�g����o7�/\`1�|��
�w����fK_�_��t<A����0	@~���D!�_(�_8��pd0����7� ���u���Gx4%b_�	1�D����Ck�O�bv�O���C�_^���\�
�B[{H�q������lG�F��OO�aQ�{b@.P����8HXg�BI�n� �Kc���A��
u�FFC�B'5EF\-Q�����(��?�6
��s�(/ (7~}���jH����r�>~LKf��������P^,0�g����N�	/6������K��y,�G=P
�w�i�Z@������#W���H�=��{ ��X��������_2.#e`1��)�t�"(���U�N����Pv�J���\�1���&�z�� ��n�Z�1��s3����l��uc�����;[7����E�����h����^���FD��������A��1$�vLgq���3��+'W��\#�`� ����	��.i�i��xmpO���0s�|��!�=���14^��e���������&�B�i��DB��a�+v!��=�	��\{=}�O��~�~�\t
������Q����������j�q�W��W��f���]�7��A#�Om�z
�L�|I�������/��7��?*�b��s�Q���q�������J��(5 �m����l������)y�'�\���|����d���n��Dv��g�DDDBM"���zxT�]u�jt
�QP2�3�4D�ek���d���Y1�D6�N`�Xlv�}�rb�����X�(���,���w�_�R�S�T\�P��8g�adbDcd�dl[�]kK���VM[��*'Mo��K7R��
r�M�%C$s$�$%Q%���:��%e#�*��D��[�-�,�,	-G-�L1&s�3�0�U5�y8B��@�G ��Y%k����2�������5����4=�8:*D%�lG�r��J���0�r��9��g=::
V��1A;���N�.��vAA�2���KTA�Z+���;�������v�[<�YI^�h7V�|A���h������������/����MD?zCm���[��3	�A��]��dl�7h��G�q��4l7��_/i�oW�,�;k�y���������.�VD3��cL����������������[v�efvx�����_ ^Y}[���{�o":v	�����CW�p{����X���{x�[y��W�<{�D�e#]����@�WE��F��yk�;p�{�6���W����]����r�5^�e����?����c���'�b}�r`�o/Y����$�[Z����oC�Y�gO�����;DV�/=�_89}��2��c'cV���k{�\�@��v�����X����(Y����C�H-����;�����K�d���d�d��Si����S�PnQ�QNR�R���%���e�����d�4�\2�t�|����,��j�sL������S���:�@����h4������P����]��GM��ae�:�D��A[�w�u�M�������i@�?��q}W��":�3�sA`S�s��0����l�t=nKtyM�i��;����.�1t������9�%���2�����
$�M}������X����m4�-zH���r��&^�u|M������^��m�-���{��V��_5�������7.2��6��Th��V��G_6:����|O���/����_'A����~����f��������=gm=�G��z�`���KO���p%=�e��X��)�o	^M��<R��l�<|C��a<d�}�'�����s\��_��FTe{�����4��[<=�l��AI	�g��y&�������������=���=�����3R�vyy�//J��e4���������2&�?g(0�g������e>����������h^�1�y9�T���]��	��[7�6��F�*��i����Rq��O5����w��0x��6��������4������2���w�~+�����?�����[�o�{	���F�7������,�������������.~�`��n�'s����/�`	�jRa��&������/�����m�
��=�R���N����Vx�*�e�]1����cf�����lx~������	��\wz�6�����YT�t�C���q8>c�l�<4B����X��)��m�:��� hx�OB
/�kmx�=b�
�Q���P*fS�C=����1�Y������tV~7R���:F���������UT��+7������:��o�oq������\�.�{��������n��4g���������I��d)����������-���s��o���2<?}�a8�%�_&ak��7X��oaS��G������	��}4���U���m)L��s���e�����[��3�P���5�P�b����������}J��1k����������<��P��oQii����}��&��Pj�+���Rl&���4
~�F��Yj�N��zQ�����KHAx��Pod�e�gXi��Vk���Y���VPk��b+�(X,��E�f
/��8�M�i��������\���bm�
s�N?o�u���
l���q�}E������nH�},"�w����{`��E9K|~�������>��q��6%�>�r(>,���LT���[��oR_w�Z|n'���@x����������9
�S��;���o�p�s>`�1�����9����_8����d|����h���Z��:eMCI������s�2����q|(�cGu�������YH��<��rY�`���^��k9K���ky�\[�W���H�E��ek����5�u�l���Th�Z>C������Z�Gj�{��z����������������zSaQo�z�:�k�Y�������zZk�f�9(���7���)�	�5��&�����~�(��O����b��p��f
�H1�a>vk���[�'�e�0�N�l�0��������Z�*�
A-��P��oTG�+���2�K�j�����[���U�
���D�0��u�Jdc���
��a��X����G|���,��(r)"3j0��1�X�8���l�!Z�IK��vj����X��wtrLs<t4E���l�������>���d�L��R�)��)�;�R�l��������o�������	d3��44�h�]KxwT���Cb�]V�w��y�
.�?����2���0
�IU�N,_��
�O��*�S��L�"���0D8�x�Z�������{��S�9#F���Ge��-����%�	qt%fbt�}��uh�*�g��O�k�1_b�%D�V��b���oxhp��(�p����<Z%	 C	=����+��n����Q�@������{'��������f������[��`����{�(�������R	4[���_���p�=u����P<
q���G���RJL\{z��
����ds]����$	SE��g�4�
 �]v2��y������j���g�b�A������%)����
G�Z�`
D����!] w��W����.�?S�O�Ih����������@�_�F��~��i}[i���z2�����a��&�j��_
�&O����/f�'4e�m�
��>���0oV0f0�q�'|�@�L���X@�o� ���� E>|(���<-&t��
N<��%�?��Ci��]N\��"t
�.�NkQ���v]����d�t�Q[�D9i��m�S>y%�H���!����!��w�a��Z�'��'u���+�X��hBL{k97����Y??�;����{Z��%�5��n�A0�@�+)!�.J(K��7f�j*B��-)(�@uHG5�L�7�����B�ISXh:�E���bi��H�&��z�z��."0��\������
���p���_��-�%�o�)�"0'.����x2�)l1��Jh0�Z���2\����!Z;���i������h�&����z��H�	��h�|L/�P�|�����[����)D����+�1����]�CpE�fc3	[���p�<N���.R�naX�z6��F!.��@M���c*b��\������
QQF�X?�}�,����&��E2���B���B����[���U��9 
jZZ��gU�	��������[^��y���,N�/A?��P&��T��P�a�������� �����=�
�����Y�$�
EmZ�k��y���@&1��nxq)j���Aq8�P��0�����/����b�����7�T4lF!��Ux!������vH�~^��~��VT7Xx�1�b�p�i�O��	}��)�'h�Q.Oc�Fg���9����,
�P������+�� wMx����5k�������[�_D�s�����(���_�A�
`:�b�����\����~	`F�7��(�m5��N��^"���Bb5�����z�9�J�9]K��&�K/�c��g���D��Jk��?K��r�r����VPZ7H��
l���O����td�!�VT�?��C�j���m�\��D�!z��:\j/3����
���v�x���%�8v���N���6��5����P�~�����#>�&9Iz��0#_@x�[<�9
��,�o� _K����/`���a����UYm(�v�������
��x�
�����)��KpLb
��
��*7 ��� �V`W-�-��tXB�
p\up�A�A���P�$BBKVn��V����2��Y^��g'4������8���-=�dz��H�����g��c)z�;�:�uJ=;/(���%�av����s����C�/��yw8���������b���"y�Jo���F�[N�7������r�7�������y�(�������L�|�3�*@1������6�nEb��q���y��33����{1��`��Qc��Z3���hi�����Wam��L�P����83�$�B���v����[l����i��Kb�W�P��9k���;��e�~��� �sg��%��hT����^v���'�������6�;���qQO�r��U���.��s�S��N--�{l����C�K��H%�@����S�$"��P�	�[�C���V�.���� ���]q��>����Y�����Z:�'	�)Y"��vI��Ry�6t1|�iX��/�����jD?b�"���T+�y�u�#��
�V�����fM��Z���-�F�d�����I��Uc��l�
��-�0�����hc$�*��������X)���L�&��s������-����bj�i�=��NAq�h�J�,���;��+$D�u�h�2!*G�A!*���Z�4Z�Y[�E�@-D�G.DcU�Z4�[�^
��E�I|�E���`�2���E�ooQ�.J�(s�NC�mq�!���u_p�7�x��Yc���O?\B�m/B����:�$���7Z	}s��v�bB�V(�
�k�:,�I����p��BO������]�l���w����9�K��a����e�0�>U���q��\���K���Q������\�����.�����{����
Dw�RCt����S���|�e��r'���F���S���]��\�u�M	���������u��3;L�����Q�.%@��c0t-h�nX��KS����c���w��l����Vw�Z�����5���5��o�*}{J��"�o/d�s���o:��`O����F,�]yx���$�}C��G�����Qj���w�Tk�qX��oI[+T�����LN{����H������������e�������b��=)�o���eO���,��W����3a�����P���0�P@:�rHU�CR0��TSj>��z�C*�8CG�TC�n��4�D���4(
��Et#���Q�aj��:�ZT;j$u��rXO�����h��p�~����Ec�#���4�Ck��O���rH�f������aH
_������VR-���'Y)9l�SZ8m��	eWna
���d�f^'l�fO=�9��k�7��P1s3���)3�T������3_1�&f�����c-�Z�2s_<����,��
��^
�������<^���~?�b�<�B����t.%x��?�{���_(m*�w�4�Yi~�,�=7���*��<w�e�o5'��acm��yb�<-���@��<��3�W�S���8��|�y�;x��M0��N~�)�N���������y�"J��71���$u��`��~1v�a���_[:�o=����#������'��uj�l�o�BR��c:������V(	�}��
:%�<HJ�5�c%^B}����K^vK�eVZ*���
��O��[�������/|\���cxY�F�g�)��]f4����]�@6����
��8�;),�_K��]�L��6�{���v?����s�~W�Z�����3��L����#�F�=�k]&����B���z��w9u���� ���0|^L �����I��0B�	e�~����GZ%��.0�]�>��n������L:��8�����A��G�����XW/������E�b�HEVE�:`�X,�+���������x�H<6��4�K4�
�;�
d$$!�'$2�L�%E$�$�$%W%���e�>i����JJ=eW�Q*�����[j��t��vy\}_����D�zuWP��WR������"	��N��O��v_�u^JDY�v�t��d�����8���C�K��#eA�����|���s�M���M��T���+��wrb?�Pe���T�/��,����H�/&h�����c�����oR��~�Fki��Z�����e4_>����|f���TB�3���ws4��]�|�����%����b���f��j�>3�g���L:_�=��]Ro����I����_#;V�}���^���M@���f�����B���V�������D|�������K�<�����>�&������S��(B�Y�������g��������;J������MN��nW������!y��a|���|�;�9��o���f@���������C��N�x�K�`b:P��@�dZ�X��i]Kb`3�l�]:0���Uk�l��c ���uydr�d�B���������P���t���K*(5��
K��!
�zK�i>mJ_��pD�@�"�%�V�f*�$j5��I=�[�����X��qVG+��JC�D1���b����b��=V��U�b;�F�l�`�3
1�1:2����X�x���(���E"!���D����l�*���93���S�*��l�?%%�H��Q�Tk���8��ge����i�]i/�H��\���	���N/������q��,��c��I�|'����y���w� ���,V����� �t��80��#�j#Q����=�9����A���`��u)��<�����TT�D��z�d�����)s��g�/pQ�8�p�1��`�\�@�W=��9�~-���I�f�HF�4k�r("��������yT�}�4�!��}aH�?��b�#T/���;`�rR����,�<�`A��I���Nj�o�.��A���+
'���;B��/����U���kOv5��]�h%���Z�/
O�������c�QBR�$dA��@M@))��L�d�$������R�O�e���,��Yj�O���uq4��Y���sN ����N��8������u��~F�o9�����NrR��}�C��3���������8��9����#5����TW�5����`&���I�)!����r_����.d�A��P^{ZQM0�\���0\2V�{`���d�+���������L�5>��?Z���@�}�n(<��n��qx&��I@J��b���3q��.L/u�E���%iC��-���@UY��G��+}U��tK�/��A�z�j������|��V���&��a��!�VV�\��F�������4Vx�RE+��'�{*,-�%�[N��"�YD����j�����Q)0���(�+I�o}Y��76�e��t��%�5��)m�S���0r�~\�b�����Ph�v<K������.����d��j�h������3�~D�,6��c�J��?�9�'����e��Q������W�:?���������?4�DD�P(;��
�����E�"��5A1�`��`Y��e�F����1PP��M/,G��j4�*��4�-J���K���2lY?�1��]3/���op�4�k_��X�=�9��
���
"F���D]_����Oba����T���
K9S���,�����;�a���c��y��>�Dk@�H��}�1+�R��$����Z���j�^��A��8D6�L�Rg���	�.	��`n�������W���\xE=�*��H�B�zvq����o�x�e�-��j�������P$_e$JQh�s��w
0�?A�Q��v#-�y�����@���?'G���m���|�p�����Y������� V�!]�?���?�kK*
�tC�.=��+�@�79D��]���A���DP+=���b���e�m5e>��H��H�V��D��nPU��(	�U�����a�f��U��Cq"�0N/*�bc�T�;a ������Gc��
��a���B`[�d��� {���V�	�8�p`����s�Py8Egt"�������N!Y>�*�_�p��U���f2V}�^o��R9��0T��}8��mU�����<�t�9���f;�U0h�R�+
�
'��2���<l]�d[�c"�)d	����*���c���H�;���aG#J�?�,�0#�T!��Y:-���i&�B��Y���X]Y��j�S��m#��M�Hz���oR�����C�jh�������b��m���?����)��B��{���"���'���K���+����2:��-�� ��W��IHr�3%�S>5�3E�1� V��������~�q.�G�����
�����G��bK�C1��ZTqN3�V��������d�I����I�H/Da�9	`oQ�-��X,���5�����R�X�]�;4�zb�����:��������X�#�Nx�!vq���B�/�
k�J�-�>!S]ih�]Fwn�{gD�*�#�~{%bj����.v����M���(&��\L�p"�6ZB�{�u����.��R���������e����N/�����/�xw}1�B�"��T�m����o#���"�OZ��1�a�E�;���;V��wEe��L���F�1��q����;����K�p���Y�C�!��u(f(h(j�E#~ � !f!�!N"
����H����_�{���H�j4mB��S4MJwcc��Z��<���=�^����H��=HM�dr+�!�W%9'9)Yd�X4���
��eU�������R���R^~Y�Ynb�`*!��l6����M(M6�IM�[�P��s��T�M��b��[P�pT�
�twXp{��^����`�,c���G-D���n�m�E��6���O
�$���f�����y��S��`>���_��9)/L�{�Wn�xm���r�{�V�ty�m���n�����
b���d�T<��\�A� ����f0_���j���1�{�1���!��M���z �;DY��"�sv���	Q[�)#H(<-cDZ���W
��D������C}v�`w+���~������k�q��m���)��<��viA���/!�,6;���oc��YE���6�%#s�[�8�G	�����{�[&p��p"�2����<0m�>���������u����5vp������7<xG��s*=�'���7�}0g~H�Vg*����
��������?x�c�J-��I����_��
�]2���!2�����w�����c
�g�7���@2n���o���0{Ga���
��&�[NZ���0���c�Uycn�F�"c��.D���"����
�����h��A�����"^Q���������8@�iZ�&��h<��.'��4�F�b�~��������c���1>n9�9N:�
�L����	9 � I�����4�l�tY5���}�Vb�J2�����+�(�(3)�KwIaI���lbj�9����c�`Va�a��64��/��OsS�uRO8m8��qjr���%�0������|�Y��Gt��m�����w|��U�`�G(�s9���A���A��;�A��N����w�;���w�/P;%�:(}���?���%�r���ui������F�@�� �@n�<����Vr�iKy��"A���	r��j��9|��,M�-�d�VA�o.kA�
��rK�
� ��vY� oO��'����p�9�dN�� s=L>�}�B����9ZtM�w���^�*B�=�m�T7Y�6!��&!s/������7�x��>����5B9y���9�H�����@ZO��H�N��FurosE����S��F!�V��;�sIZ���'O�����{�N���<�O����!���!�����a�a?� q��Xt�i���&�6EF�U���:��j1��������q�h4b$cd�dd���u��%�3�AGOGH	�;$�d"$PORTr��1[f���S
QjQ�QbR"�����n�c�d�f�h�je�1E0[0c���'f������kiZi�iJj�jvgg���s���i�I�9|x>x�x����($G��B�/���)z)#���+O��s�8����wUm�w1&��x�J/]���|��f����}����$��K��^���y�
�������O�K�(�Y��pc�����m��F�n�"b����B�T'jR�*�_-P����z��V<UA
-?���������T�
�ny*&B���G!C��7�^~k0��7���f�ynD+��9�1�����}��o����B��s5f��>��~���z2��T���_<������8_|���}V/���9�4��|��W�ww��zw�K%a��-��Y4�j�����/]����0�o���w^ |�R
����k^�����[����h������Fx�	/=�,�y����P8�G���@�K������w�`��/::�n��v�F
O��������k]���z�u����<����n�N��KO�
��Y�_/~�z��S�h��A
[b����j����b�"������"�������!k�����,���U�)���r�r�r�,[�-y,�,	-Y-�LL�&�:�NLMMMa^	[*�)	���R��]UT��x:M��]Iy���S��oX"�/%��V�0- ����xcV��UV����.5n���V"�s��>%�1�t���m�x���w�^���t���t���q��j��t��b� �����`�O�-���XZ ����<�^�����w*�4�XZ)� ���i�{�X�*�B"�9����������!�tg���`�C�0���
fd���3Y�`�� �\�\����I�y��	s��f���0��u�6o/g�-wVa���,�<����z�a�5&�<��5�Sz6��8�f���av�T�0Oq�y�]f7���
bv��1��������6k��i����m��*���S��+� ����V���/	�"�+��3��Qm�������<��6�F�����1���qP��u�p�;�8:9�92:�:F��,���I���_r������d������>�(��Pl�N9D������|�\���	��A�)�����
�m��y�y�i�t69�,��{rh�J!s6[*���B�L5~����������]���j@����=z!�+����8i1�����0C~�h�T:�2P�v����`7d7���n�!�����m0��VV���+����"��o5$��a���"0���K���]\������M��[�l�G�^��-�
}����~Xo���&����
��E�s,d�~�6���T��5���
��V���J��_��@�/�b�������j}��:�=������o6�~�B�sC,�>������P���U���U��;��w	�~�C&���}������U���j���l)�|U��w>�����,��5I|W.������x���4�@������@�/5�������Z�S]q�}*&*�?U���V^�����w��0�7�����*����� ��C���
�[@?J�M�v��m��:��Y��?��L��'��s?Z���o������t�}I��D�	����`C�����p�����PQ���xG�AdBLC,DEl���F�a�{���\��=���h�H���x�X ���0�]yV(!�I6�J�g�����X2���%�ra���XVYU^P�)�Q�o��,���|��)�8KG�U�(S3sS�
S���k�h6�4�4C5�3�����4�HN�9�*�G)����>���G'�O�������{h��2���<�������u����o��m���/��K����2D�t����J�s���uE�jcr�A�_��W����R����������y]U��O�������dIM���@Z�W���-w�Z��-��3-K��iY
G�|�d�I����1qi���r/c#���0[��%�sqH��]Qq��
�!�lYd��������7H���!U-S/!�2uJU�f��:f�lw<��7��.���t�
%�)�f����Bl���������(��#!��`H�
AV6���"kS�UK���(p��	�8p�DVz����u��������K���P������r�K����T�k�.:�K�QR��n����T
:�L������C�@aE!EY[*Q��G����bBY5GSDY�!��5&C������(6([����J��:�B��m�J	�0���:�^�c����a��:$�F��X�%�n������x\�
�@/YY���Qo�tL�0�!�T9BM(��0�p��I<#���!����K����F��F0c	c#�"��8v�;�8�9*:j$rRRr��MzH�HFI�I�Jrl�r��NIDiF	I�.,��A�c���G���)���,�l��j�5�[���B����S�����Y|�=K<=�r�rr����vH3=�P(4zz��z������9��Jnn%y217G}����-ts���������������9+K����
��ys�Ko��
��[|s�cM���qI1�05*�-��
~3g�w��9�(B�6s�b��\��{1sCq0�.*��U���[1bz��O��������]�32=�������3��q����?�uw����H!�W.���x����+��1^y)�+�g������o2c���3��n!��������n(g�+Ost^��4��U�'j�u:��u�v�AZ\�^\�3\�U��S?9WW�[�����T��e���/��W-�:3�J��Bd�������������t��M�%��=�\�����X�
���a!�F�T��4�T"Q�QR�RB�aK����e�����%+L���y�i�ilN�h������&�f����������s�p&q�q�qnr&�����{�x��/ ���}�d�s6���h����-u�v���+�����6��?o��s����������g��{���|,��|E��K��������7��9��	9�;������(���.���j�?�:#�_�L�z��r1O|��H8=�jmzv9��gWL=�K#z������_��Y�M�S	z��2��SAfzv}���#���L��[�Q�A�h�s;I���n&��9�"����({n��y_���6"���b�hs����9�D0�,��������p��i��7JT���5�m�\��y���y(^3?2�k�Ym`sO"l��bs;�56����Qt����W��;z�����J�?���a"�L@@��S��DR������@.�8N`@�:B�S*���C�i�$G��c:��y�}��SCk��GOAs��M���l���j�����N�0��5w[�mHr�=�^������_/�� �
X���0�&���k� � xD`**Y_�u�!s}~��H��6S>y�G~dd�^m���N4�����%|1j�^�yfd3�yA�s�����H�&�W�vN���~��}gE���Km�?��d&�"6����+�!E-�*b#�BbtV;T���:-��L����j��qL|�����m��������4J�����e��_J���M�!�����qO�
<�7=���2��$�OTT��6�&s��S���������"�)ZQ6` ����N�	x~�JH����7>��$v����dMD����r�OV�!02�I�pM�7�c�P'�d���M5`9��[�R'd%�X�[ ���Q�CCl1�t�����A��$���:��`���,��T�l|�:�y8��K!�69���E�&�|3"
��3`)5Y�5ku;����u�����[����.�Gy�H�S`�k1gd�\[�����������(~H��=#��~��:J��M���V��9[K���,\����E}��C�\����je;�Y��sz/�e'�b�B�j����1�h7�QQt+�To-�#T�dW�������Oe]�e����`���%�PFS�i��u������
����D���w�^	����mh���tU��o�=/8�1I����y�Wh����@�u����H��+\�uW����7z����w=v�L�<�����3��j
�@�>s��/�@���%IG4���p
�{HL�J)p�,"��}��l�d��z���xR�Yj���qY	��\)����A�\����-H�`�H��� ���D���T8hN�`z�+��)3��@������D93����i�?���;�X4�J�M�s��YO��0�����6�7�	��"�D��Z����?��9���NY��^�v0�

�]�>�i�>��D�Q�	i��D��i����0����W.��e8���u6XI�C�a�����c��hF��u8}�^�1O�Q�b����(`:.U�"CX�+��`��V1)z����
�6������,B�]��7d�����&���j}m�����������K�9�"2�U��F�d��eb�_���J���
�WH�;��� �_��Z#�8]����������M����ha�����N���q)��@b�b�����Mg�����-�x<��0����4��x��$��(���e�1f8�P�%e����|dO�A� jQ)k~��
y�!��B�^��\��C�����
�"~�nGU���a�4�	!C���T6��2��U������bpw2��W����p��%��s!@3,�������p0��o��y�+nB����������S������8?�;[���\'�d������Bu
�O�� da
4
���_q:�(�r��{J�R:f�]�8F�1��{�������#B[��F�l�Y��u���a�$��P�.����/�F{!�
o��y�I����=�V�G�1�R�#7�"������( �& ���Kv�pq������w5@���y0	1I������4�3��i#��G�\F�`=�I�8���\�����sA�E�=z�$�"�<���U����$���f�zj(��l��'.7T�g��-�#��'�K��k��{4���H��K��I^�="�x�����������#*��\z���8
W�����> 5�5�3��@��D�!-(
�m�S���l	�����q_6C�B����F�F���S\yb{��l�(����nE_Pw�o��pCV���N�S�?�����	S��^����~=�;�����
�*��t�7��%�jFr����LIn�$~y��X�����ezU��^Y����;�&�o'v�����6���F��������#�
�N����]����_��������y5]n��b� ����/r\���=*������4�r�M���)z��Fr�pW���@,��51��spH��5V����hT�]�;�UaVx3|�����0 @������K���wH�������|��A)]1%�01
d.e�J|�C4:���9�7���a �(F�C�$
l��Xc5��!����+����r9<G@*�
Ax��:�P|�U`D4�i>f����9�/KAY��
0~b
r�-��'%�c����ysX����11z��h�{�d
��\�y����{W.�H��c��u�8�WY�#�k���&���E�%?��&\Q,��v���B�;���\�|Z���0MC��{u�^�7��*�8����@�K����?�/������N����@��42}V���)]W
@',$q?P`@�%vx��$;�fTF��o���a���#���H�=���	������g��|��Q�|4�+�cJ7_�-@�"
��AT��� R+� L��Q�"��*Y{��g�����\t�|T���a$8::�njj����^!���9�,��K7,
�Q0j�g']�~����_0���
"s��D�k�Y����uBT)  �OPN�� ���gp}"���B_�qr'��W�s��S�����p����`�8M	�,����5��q�*~.f	f���e�@�}�gB�	�n<�7�T��;A��C'A�<����dNI2��� S@z�#E7lK����0������NE��VY���8�c�Y���M��4����i�_��h���u�F���\�,����Q�@�_��h=h���NY�Zx`r��Cq�e����Ns�����[E��ck��1'�U&
��fG�0�lM��z�C������fuI �����0: �n�������#��#��)��*��*�n.�$P�Y�BE��f�*�S�l�Eze�Wv��j�([E��?A��7�x0���Z?v�z�;�x�7f1U���gG����I�*8&{�1�v�u@_k�������+��5(l�{��w2��7��MS�r��O���\5�\���Z�Q�� ��L������4������f�Q�P�Uu���H��In��q=��������ZB!�X�iW7�8�P�n���R��YB\;��-�I&���E+8lz��������Rv�����l������p��T4���)BA��O�U�%�U<�V��o�'�zAZ��8,G�� �Zn�-:iS�A@0�\V�P���`kU��
X���}�z�L>P�$u�����B�@�R�!l'���X���H0����_���{��i��+��`:������_\��'����|`��}������Rq�]Fi�n�q�XhGz�kk
�fS#���6�}�V�#��[�>���s�L}�Af#S���
��	����r	�\$�����a��f<M�Y���������-�����/Ub��.���{����UL������o�/��h���?�"��K{���]���1���yqR�qMAg���&@`�o
���+��H�$��d?iu������P�5����-_��������T�*�dVGeJ��!M$��q�BF����x���'`y�������f{x))��<�h*j&��L�	���j*/2��nuG�\`���Q0�������t�B��* %/9��W���Hn-��RI���?v�����=�!/s��D�!�w�*bC��
�]��vS/��l
����^c#��n�;�Nr��	�;SS�/�)��p:E��s
�������	��2MBM(�g�6�O�5TsT�,�N%���l_0���x���1
�Vd�t��12x�3� |�:�Vn,������5eI0z����&�����P������p*� J;CX���3}�P��a�D��A���AHg�Oh>�l�Ct��+Ba���D���D(O�C��+����Ap}}p������jl��i�C�t"�A���(��P���%���0�u�%�q�s$���
g��:!Oh�=���@QFX��w7VR�]�#QUz�5�Y��P*�{�:���-���z`������;k�^OCg��O^������m�����������K6�����U�f;���b�"��I���r���d������UP�
D�C�T����N�Il�{0E)D^g����R��_��x��N�����+U��h�}h;l�F����
�9J�=l����'O'����:T0z�������{��w���G�7`B����_�%����%��3d*��,�Y��yo�2���%�����������-Zm
�H���`�X�v��f��Z�<�~��n7�;���3���F��p*>���on�c��sA����9�y����m�N.���S�	�C��L�����66���T��qB�m�!j���A��t�#��7�$�U�(K�\���	*�K�"@�z������g=�s�B����8�6	U+����*���^�z��>�ps�g��ob�����Xzn/���XY�5����v�7yN��?����E�H*D��R�C}	H�3QJ�9�;M|\��U{����l�v9@��SY�E `s��O�jaGi��a�Gj��+����R
R�u��}%6�U�#WZy{�V+>I�e��YR3��Z|S>���(��-`A�|G��b��N|$��,�@��`�W�Lf�Y_>X�-�u�G�Z��B?�����k+��:��`[m-����p����{`R�>��LE�.�
�����@����Y�������n�!�4������$>�G��t�����a�LvN�QS������6����i��y���T��2	N`)T�i���Ij�s��i��fwTh�6$��OJ�"I+����M��:xFG����}XW8���BZW}X�$�)V�B�F�\�-j���+Yn/��i
X�
�ug@��H�l����!���&����T�;�+��dW@!=D�D�fp���y��p���7 �W����2b���yp�~;����a�>*����`2�xR��=o_�w"OA;%� F^.��8�U�h>�$�m-ft�0��b�����L�q����p����bq���_i�U���J��(�?"<T�Nf�g��(y@p�k=
n6YD��[#�C�BP��XD�	q�>���tz�B^/�D��+Cd
��]��>����[����5�m�[���gz+��W���u�5�2�.0���bq�X2�����V��������8'�	,��D�l�4
FW�%��A���%]�7��N�Q�#�'4����$mr��@EIM�Tz�����,9K����?%�%w�^|��8`C�dz�������^&3����2=�&���LJ+I��d
z�B�-���`&c�3���fC7��4��v��Z���B_{����_;�y�N���6��&��p�=����������z�9����������'��]�/T��{/}��G��_�|0�_��g�}��������/��:��Cs`?RT���v|���{�H>~|��m� _��������������p^@o'�Z�L��`,�3��`;~��f�[��@b��0n�%c\����3��C&"c����2"�h���f�yn�1g�-�����.�K�	K��������e�%wI�M�m�v���me�	�ju��GX�rj��i[�b6����Q�C{k����{�q��<�����sw�&s��Dc�1g0�0�0�0�01�x�����5�%8���\@���r���HN�����o3�9mRh�h�����&�&d3vJ���pN�"N-�E'�����C>LCw��VzZB��NCg�3�������]��T���ZC�N��)�b�5(W��>�Vw�U,\C�[�|��FF0�W\� �vt���|$x�~�C���=T�[*����+x���[���d
��6�=�����+�F�c��9:����s.�����G�]����g����7$���u�2~V�]��5��?��
me���@
2Y��A�/	��AD�Dz
Lx\�PX"	������}�����D��,��6U�p����R��A�\�$�y"\�h�-�-���.�Bg5L�	�������@�9�z�������N��BiPJ��m#4�c������bO����xo���M������R�a�>zv+!�����w�\�L*�5�����y@������_�R��u��G(wq"�w�DiT��[�J�=�[��k
��:"��$[����y>W8�K�7���[�����	=&�V��f�}��M����p����D�i������eDI��2l�l�F@����U1��Yl_+�&�,.�\��t�U����B���{��M�v9���� f[���`�`��z�N�%(�b���1�V�&��:����`��l��L2
���{��{�`���w�:�6���3���Q!@���8��9���Q�?G
���e_�@;�����~N�v��RXZ�F�l	�rv6p���]j9�e'�q*��>��
�V�Fdy5��wtzzK*���l)U;��R������i-��b;��;r���+��@%,�����S�	��VP��A=0��!�	@��i�Wy&�g��`�9�{k�q.&���X���@�'4�/W��"�X���rL6i0�bk�(�U����a���u����E]�� ��BE�;c�S�d�W�P��R�Z��"6J��k~�X��[���&�W�"�����iR��m����9Pa����?1��$�*tY�E��!�
��Q��$���n]���e*�J�B��*<�i
�����b�DMW����� ��*����Z}�a=����C<����(�l��
<���Fz�vrM[������z~�M����^6��H��;�,������Nd6�1va�A�`V���
�wI�.	�.B�n������B7�@���s��	g�M�mZ_���S���AB�YQh��8�m��47CKF�J���E�!�)�@�{[�l�&x�D�i�A������a8�))��
|��#H7�3�||r��4%�����0|�s�z]��_������0�PM�N�#~\�z`�Pc���
�`�%���xgY��Y��S:&�����z�UE�H������s�W�Hw-GW���d��I�mU��sO�Sl~��x)��������,n���{�v����<�5�k<�,�������4c9~d
����\��S����%�/U���:���g�u�j��S�B�Ak��E���]�l{�nzlH!��q�1�L%Q��[�����t��������I��#����_�[�FU���4�W�M��q���Q����<�d�<�
1E�����(���5}�$������\T�����&��n��*�|���i���9�p�����T
��O9�%�p�f�~���?��o�����{��{�Zu�����}�jKT��=4�-H4��
�'�"Q�J�`B ��s�K�V[��JK��}���� �-��
'��V�:M����_vf��a`,�A0Q�8�p�E�! ��RKn,�c��r��{��Q��k1��W0�X�-d!�_U�yb��v��{u���W
�Y��d�4BO�!C�����!�Th�a�q����L�����{3d��H�X��f�����Xh��fp�NV��4�(L�(]�n�����Uh�,lK�x�R��Bn�K'm��T�s�����4#����87�.H��F����mo�i{(�h���M;�o�MNHM��'o�%k�L��WT5�E�?�'j#�pc4�#�������-��i��p@�2=���?�n�gc�����k� i�E��Z�1g�����Y����lG����V�q���93���������izbG��4��_�.F����`�,��c�KS����W�K��m�/*���_�cx�!I��0Y 54f4'���J%��h�j&�o��r���W���1b�p~)hxA�?���������������@��|�$x��Q���|�LV����Mo�e����Y��r�������x��T��0#w����
DZ��w���nV��5hH���Jd���[����;��D�,A��xfFE�&�O�������W"j.�!���P�K��y�b��0��=xgbM���P�M��z[���v~������Vv;�f�7��O���6��,~����5�oOBA�%X=ay��*2;�d�� ��P�[�0�-�'r�0|���7��r3������e���}sQ�\*���`�����;(S�nq�
K7���2�
����(�cn�K��2@�]��M���5���nYW�f����m�J�,vU�KD�H�b�g��O��'����)xc�%�k.<%�y�n$��w�S�BhGa��������EdI�L���3�����N���b0�j�7%�)$l<��V#<���@�+.x��6;�""�ac�{.�����@[�}g���5�e�;^a�&����>9O��f��JUq�����7?�-8��AhN�q=N�H>E�A���F���wQn������g���Wrs_�3�����R��W��	���_oC��9�K��>%~X�`��W�U��,���"mM�G�������������_~�������������
�t��j~t��r1��)�h�=�8����0B����9���.��o�(�j�b�q~;���:�}�����W����u|2�E������#����5�'������hb{�d��
C���&z���`T�>�(�q7�.�e���2����(_���A�����|X�SqK�����!����<),@��+-kQF�a]��>� ���}6�^3�|=b�������~@�CH��C
Z#���!X_��h&�gm��	�X�c<���d�c�lx)���3333�23��E�F����{�����-��R�����$t�z���W����_��	V��j�U�XM�����ECVm���=���Z�4��7
����O�6�b����-{��E�W���hijE�	�-���OX�l����=��S������o��O8��&�v�Q�O�5����?��g=�H`Q�(Eq���hb�1b����S����6�,�]b	-k����lIGv g�I�AR9�dE#������+��Z�7e��T���f��la1��u�G&*S���p����rz9����V���v�n�+x�\6���n�N�{�*X
�/�V�.�2_N�wd)�L��%eT�W��MO�&�E���4&
Kj��`������x����DpA����cqd��u
�E'���8�P��i���	��i������.o�<��S60an��|;�Q�O��1����[4����M�76�y���7�����s&[=�?<���{����D=��	�
�q1�r#�q�H8Q���5{��+6��d��"I�$��vHM�/���P�(��pJI%�2�r�3�9H�44O�3�g�y^���tG�)���`;8�����"#�h2��*�:N��A�mH���V�c���v�}�p���]8�������o}���>�������>H��QS�T���m���S/Z�j�TtN��[�����<z������>�SS�?������A�
:�(��E�9z)��I�*Y��)S�T���r���SUm�����(R�T5UOUT�����UmU]�WV�Ue�Y�V�Um�[W������L��u����6\�����8p<��k�<O���,� �e�l!��12��$k�f2�(;��2�,[@�i<�A��54=E�8��v�;�UZ�>��.no��[����l,������E�&�imb��h����>n���@�����x�!N����80N�����:�����Ez�N���@=R�_u_�*�;���t-����;�������O���y+���w��~�Ot#�LG�U�j����hxo�}�i�/��
��>}k�����a�6p����"F�(a��*V�h����2g��a��ou<���5=|���@
���!D��yk��#��!I�J��-y���&��9yr���E�*U��m����Yo�������Hyufe�A]��@�W�N�%�M��p�G����gg�gW��Y��p�����a�U��k�k����r�sp;�y��@� �L����Su?��7����"�c�����]b�XW���@&!�������R�)M���3�n�O&y>8o�eNJ��Iw/��t
����>�r0fI�X�������ip�J���62��y��o�%0���k��GK��yq��w��Y�3�4���
8������-��pq8���������6m�6a�&\
���{��������m���x�6t�C�������
�Eh;b�$vZ�?�/
mSlT�NWl���l���.2�"��Oj]�������h�NOg����7���QA�~�#���r]+�������{D`I��))�9���FA)$b4"J"��o��Z*�����~�Q������D���? ��Ox�XD��+�
���(�w+|�a��W����D�����D`^lQ�p>���_�0icy���RQS����u�
�t�H����.�"��9��z/(�����lJ�n����t[���B+��K��,��Od������V'"E��B�u�T`-<-����+	�u���48d{����������Yi
�n�.�l�����������@_Q�_&@��W-_czA{�z�,���`Ta8/��6�9-�
�"y��XBD�zR�����cx����+���6*
��/8��@�|�Z�g��X��k�b���f8'�����I�����7��.��p�>�,i�?��?�������cY9�Q�p~��'�m�j���i}���)����Q�LMYi������4��?����W)2�7��m���C�;�V��$$+4�F#�m�K���Z%5	 ���s�Zx��:��I���N���4$A:��:�#���H�Q��IuX�e�^\M�G�M��MR�CZ��e��d����]xh�)P�f&!V�t1�s�#���{�l�����K��sN�'�(���6���;�A���A��l��W���Z��(�D���V�O��c �,F+����m���s*��:���q��DK�Cu���"���H6������hf�u@.P�a�K��L��p��B��ow��5�����[��@j@l������[�D�V���fa18s����7�&~����~?X�������;iv���]�Q|����HR_{7����C/*�(�w/FB�F~��~��o��L���@>|�Z(l�6��+R�1}�1�}w�(����{$�OK�u���"��f�y �y�U@<��A�lG��q����x1��VLS@+���B��#�����]�f��G~�9��Fn��|�� ��V�K�H�Q�� J��el�`��F��OJ�5d+��t2h��g�g�{V���`�Bc'tR�
�q���R{����H�����������P�i�+���~���
vk��H�K��+k)\���.fx�D�|M&���1S`yc�<jq~d�BWw�r�5
��Ar�At�f������yZ�,�������D�5VD	x�b�nb-�������83jX��\0�����3#�-h���Y�}��J��-�h�T��]����z�t���_j��T�4�|/�����"j'CP�(<X�!�x����&m�B�nR��C�� P(g������������V�	����V6W�Xr�W�v�U<�K>�Y��y������W ���;�"��D����	�)h���Aj*���]��1��=�8���K{�1,�9�}�8@�������`��#��[-��$o�~�b���2/w��E��ZS\�O����5n���.���8w_�2e���f=����Qn�������Z�m��[�;~a�3���OJ<��D7^�4(�{/�i����Z[�����T�Ck�)t��VNh�������^���	-(�}��)��DF?AjuE:u���3`'x��e�u������	���n�}��	����_0GW��\:trtsutu�n�����`U�=P���#���:����]����10
(�Q�ww�����.�Q;5Mew�.��k6��������n�t4�j����7�p�����>�����N�-b�	��]a��!#�w�Nu�����fo�D�8��e�E�*�pq!�����2���uL�]9a�/
:)�T����`uM�,�n��'w]��sc.��re�]��t�����\�L����F!�B�� d��H�����_s�}�����7���B�/�X�o34��^��6\�fh�5��Pm�����jp�� }:Wm�?�@/���+��Y�Q~�R��(�5k#��4F�E�[�"�Rn��b��!�Wp�k�@A|BG��Kl��Kh\��%�6�|��c�e��l��C��)�/�
N������
�]��qO�+�
��Ai��D],_�Z��
x1����th:���]�&Jk�9���^�;_���]���/�
��]\��@_�
����]d+�/W�/�.v�#M����<t��i0N_����.�V�/�5�C�.���/"����Nie��"��<���:Z�/����������/���C��V7�����\���,���u��Q���X��xS��i���/B���}�Z�E�e���xm����f��;�t��l��dV�,��������d��+�](#]��JVW�����"�;���U$��@��*���5���Q/2���.���3���QF��C:��!+<�]�!���d���]�Bz�g���\B����SK*G�
� ���*<l��
�-b�X���n�H}���1�q������;3�}?98���
S$t���\#M�}���a�]Fj��M;�t�m2�gW�F�W���z�/*�V����<���co��]V<��G�b���
����k~Q�b��]����b��(�]�:j��c�����KG���t�mk�w�s��_|�X]-}*G9�O���Q��.j��3���^�].����>����+g�/q����%�b�h�>���T��3|"�$���ldi���|'�����a���Y�(`�I��$d��^�4��a��q�����]}�F�����>�#���������W��~���6����+������\%�����`���$��O��x����Va��4�x�%H����`d3��+�	_��.�J>X�A�zT�r���s)�o����`/�w�rCkc/K�`�%����������'z����e������s@~������������Y��^�2�X�,r_v��� �>��`���j������p��J����������Z0�)�`��~����� ������5�������0r�@����A��AB������ !L$H6i���G���aK�Op�sP����{��B��X������L}���R2s�.��)U%`���0��`�c���C+_/!��G��(�Q�h����� ���l�$A�XaG�n�;E_��?����B79����TR|�������@�#�}x�����(lTd��^�����F����������-���|3�V���|�8����T��hf>��&�O/z��`I�Jo���^����^���1���l�
1AA�������l�1�(nl���������y������p�ey�@�G�����'����c�(Jl\���}`K�h���o������WQ�=���uD�w�����Q=�a'����=�p������aG-T�19�p��%������]B;�#}���
��|�f*;�@���fV��z�v����������������c�>������F�����wO���^�$���i��T�1D�4�U0�e�v�x��7�p���H�T�wna�����Z���9p����$�Y+2�;V:��T8g:w�s��{�0��q�	o����m�z�`��N�w�N:����ww�����M]��"��d;��&���u��;�w�l�N����#/e)�_{����H��`7ag�}G����'���s���^�������������)�t���Am�RJ���J�Zt�������n��E�T���	Fd��J�8�����E��3w�����R����q  �R����q� T��2��8�$��\��;007�����b
�D(U--�]&0���f)Eq�
�F(��}��� �P�J�p��H(�J]���<	�U*7�g��P�J���en	U��|�1'@L(+���;�/b�	U�o���PJ��n�}B���yW<P(�,n�Q(����_[
u�
��)T�3K��;�T�3�0X��7���5+T��$��3l��#1����s3�8��_�
#45Alexander Korotkov
aekorotkov@gmail.com
In reply to: Yura Sokolov (#44)
Re: Get rid of WALBufMappingLock

On Mon, Mar 31, 2025 at 1:42 PM Yura Sokolov <y.sokolov@postgrespro.ru> wrote:

14.03.2025 17:30, Tomas Vondra wrote:

Hi,

I've briefly looked at this patch this week, and done a bit of testing.
I don't have any comments about the correctness - it does seem correct
to me and I haven't noticed any crashes/issues, but I'm not familiar
with the WALBufMappingLock enough to have insightful opinions.

I have however decided to do a bit of benchmarking, to better understand
the possible benefits of the change. I happen to have access to an Azure
machine with 2x AMD EPYC 9V33X (176 cores in total), and NVMe SSD that
can do ~1.5GB/s.

The benchmark script (attached) uses the workload mentioned by Andres
some time ago [1]

SELECT pg_logical_emit_message(true, 'test', repeat('0', $SIZE));

with clients (1..196) and sizes 8K, 64K and 1024K. The aggregated
results look like this (this is throughput):

| 8 | 64 | 1024
clients | master patched | master patched | master patched
---------------------------------------------------------------------
1 | 11864 12035 | 7419 7345 | 968 940
4 | 26311 26919 | 12414 12308 | 1304 1293
8 | 38742 39651 | 14316 14539 | 1348 1348
16 | 57299 59917 | 15405 15871 | 1304 1279
32 | 74857 82598 | 17589 17126 | 1233 1233
48 | 87596 95495 | 18616 18160 | 1199 1227
64 | 89982 97715 | 19033 18910 | 1196 1221
96 | 92853 103448 | 19694 19706 | 1190 1210
128 | 95392 103324 | 20085 19873 | 1188 1213
160 | 94933 102236 | 20227 20323 | 1180 1214
196 | 95933 103341 | 20448 20513 | 1188 1199

To put this into a perspective, this throughput relative to master:

clients | 8 64 1024
----------------------------------
1 | 101% 99% 97%
4 | 102% 99% 99%
8 | 102% 102% 100%
16 | 105% 103% 98%
32 | 110% 97% 100%
48 | 109% 98% 102%
64 | 109% 99% 102%
96 | 111% 100% 102%
128 | 108% 99% 102%
160 | 108% 100% 103%
196 | 108% 100% 101%

That does not seem like a huge improvement :-( Yes, there's 1-10%
speedup for the small (8K) size, but for larger chunks it's a wash.

Looking at the pgbench progress, I noticed stuff like this:

...
progress: 13.0 s, 103575.2 tps, lat 0.309 ms stddev 0.071, 0 failed
progress: 14.0 s, 102685.2 tps, lat 0.312 ms stddev 0.072, 0 failed
progress: 15.0 s, 102853.9 tps, lat 0.311 ms stddev 0.072, 0 failed
progress: 16.0 s, 103146.0 tps, lat 0.310 ms stddev 0.075, 0 failed
progress: 17.0 s, 57168.1 tps, lat 0.560 ms stddev 0.153, 0 failed
progress: 18.0 s, 50495.9 tps, lat 0.634 ms stddev 0.060, 0 failed
progress: 19.0 s, 50927.0 tps, lat 0.628 ms stddev 0.066, 0 failed
progress: 20.0 s, 50986.7 tps, lat 0.628 ms stddev 0.062, 0 failed
progress: 21.0 s, 50652.3 tps, lat 0.632 ms stddev 0.061, 0 failed
progress: 22.0 s, 63792.9 tps, lat 0.502 ms stddev 0.168, 0 failed
progress: 23.0 s, 103109.9 tps, lat 0.310 ms stddev 0.072, 0 failed
progress: 24.0 s, 103503.8 tps, lat 0.309 ms stddev 0.071, 0 failed
progress: 25.0 s, 101984.2 tps, lat 0.314 ms stddev 0.073, 0 failed
progress: 26.0 s, 102923.1 tps, lat 0.311 ms stddev 0.072, 0 failed
progress: 27.0 s, 103973.1 tps, lat 0.308 ms stddev 0.072, 0 failed
...

i.e. it fluctuates a lot. I suspected this is due to the SSD doing funny
things (it's a virtual SSD, I'm not sure what model is that behind the
curtains). So I decided to try running the benchmark on tmpfs, to get
the storage out of the way and get the "best case" results.

This makes the pgbench progress perfectly "smooth" (no jumps like in the
output above), and the comparison looks like this:

| 8 | 64 | 1024
clients | master patched | master patched | master patched
---------|---------------------|--------------------|----------------
1 | 32449 32032 | 19289 20344 | 3108 3081
4 | 68779 69256 | 24585 29912 | 2915 3449
8 | 79787 100655 | 28217 39217 | 3182 4086
16 | 113024 148968 | 42969 62083 | 5134 5712
32 | 125884 170678 | 44256 71183 | 4910 5447
48 | 125571 166695 | 44693 76411 | 4717 5215
64 | 122096 160470 | 42749 83754 | 4631 5103
96 | 120170 154145 | 42696 86529 | 4556 5020
128 | 119204 152977 | 40880 88163 | 4529 5047
160 | 116081 152708 | 42263 88066 | 4512 5000
196 | 115364 152455 | 40765 88602 | 4505 4952

and the comparison to master:

clients 8 64 1024
-----------------------------------------
1 99% 105% 99%
4 101% 122% 118%
8 126% 139% 128%
16 132% 144% 111%
32 136% 161% 111%
48 133% 171% 111%
64 131% 196% 110%
96 128% 203% 110%
128 128% 216% 111%
160 132% 208% 111%
196 132% 217% 110%

Yes, with tmpfs the impact looks much more significant. For 8K the
speedup is ~1.3x, for 64K it's up to ~2x, for 1M it's ~1.1x.

That being said, I wonder how big is the impact for practical workloads.
ISTM this workload is pretty narrow / extreme, it'd be much easier if we
had an example of a more realistic workload, benefiting from this. Of
course, it may be the case that there are multiple related bottlenecks,
and we'd need to fix all of them - in which case it'd be silly to block
the improvements on the grounds that it alone does not help.

Another thought is that this is testing the "good case". Can anyone
think of a workload that would be made worse by the patch?

I've made similar benchmark on system with two Xeon Gold 5220R with two
Samsung SSD 970 PRO 1TB mirrored by md.

Configuration changes:
wal_sync_method = open_datasync
full_page_writes = off
synchronous_commit = off
checkpoint_timeout = 1d
max_connections = 1000
max_wal_size = 4GB
min_wal_size = 640MB

I variated wal segment size (16MB and 64MB), wal_buffers (128kB, 16MB and
1GB) and record size (1kB, 8kB and 64kB).

(I didn't bench 1MB record size, since I don't believe it is critical for
performance).

Here's results for 64MB segment size and 1GB wal_buffers:

+---------+---------+------------+--------------+----------+
| recsize | clients | master_tps | nowalbuf_tps | rel_perf |
+---------+---------+------------+--------------+----------+
| 1       | 1       | 47991.0    | 46995.0      | 0.98     |
| 1       | 4       | 171930.0   | 171166.0     | 1.0      |
| 1       | 16      | 491240.0   | 485132.0     | 0.99     |
| 1       | 64      | 514590.0   | 515534.0     | 1.0      |
| 1       | 128     | 547222.0   | 543543.0     | 0.99     |
| 1       | 256     | 543353.0   | 540802.0     | 1.0      |
| 8       | 1       | 40976.0    | 41603.0      | 1.02     |
| 8       | 4       | 89003.0    | 92008.0      | 1.03     |
| 8       | 16      | 90457.0    | 92282.0      | 1.02     |
| 8       | 64      | 89293.0    | 92022.0      | 1.03     |
| 8       | 128     | 92687.0    | 92768.0      | 1.0      |
| 8       | 256     | 91874.0    | 91665.0      | 1.0      |
| 64      | 1       | 11829.0    | 12031.0      | 1.02     |
| 64      | 4       | 11959.0    | 12832.0      | 1.07     |
| 64      | 16      | 11331.0    | 13417.0      | 1.18     |
| 64      | 64      | 11108.0    | 13588.0      | 1.22     |
| 64      | 128     | 11089.0    | 13648.0      | 1.23     |
| 64      | 256     | 10381.0    | 13542.0      | 1.3      |
+---------+---------+------------+--------------+----------+

Numbers for all configurations in attached 'improvements.out' . It shows,
removing WALBufMappingLock almost always doesn't harm performance and
usually gives measurable gain.

(Numbers are average from 4 middle runs out of 6. i.e. I threw minimum and
maximum tps from 6 runs and took average from remaining).

Also sqlite database is attached with all results. It also contains results
for patch "Several attempts to lock WALInsertLock" (named "attempts") and
cumulative patch ("nowalbuf-attempts").
Suprisingly, "Several attempts" causes measurable impact in some
configurations with hundreds of clients. So, there're more bottlenecks ahead ))

Yes, it is still not "real-world" benchmark. But it at least shows patch is
harmless.

Thank you for your experiments. Your results shows up to 30% speedups
on real hardware, not tmpfs. While this is still a corner case, I
think this is quite a results for a pretty local optimization. On
small connection number there are some cases above and below 1.0. I
think this due to statistical error. If we would calculate average
tps ratio across different experiments, for low number of clients it's
still above 1.0.

sqlite> select clients, avg(ratio) from (select walseg, walbuf,
recsize, clients, (avg(tps) filter (where branch =
'nowalbuf'))/(avg(tps) filter (where branch = 'master')) as ratio from
results where branch in ('master', 'nowalbuf') group by walseg,
walbuf, recsize, clients) x group by clients;
1|1.00546614169766
4|1.00782085856889
16|1.02257892337757
64|1.04400167838906
128|1.04134006876033
256|1.04627949500578

I'm going to push the first patch ("nowalbuf") if no objections. I
think the second one ("Several attempts") still needs more work, as
there are regressions.

------
Regards,
Alexander Korotkov
Supabase

#46Noah Misch
noah@leadboat.com
In reply to: Alexander Korotkov (#45)
1 attachment(s)
Re: Get rid of WALBufMappingLock

On Mon, Mar 31, 2025 at 09:18:30PM +0300, Alexander Korotkov wrote:

I'm going to push the first patch ("nowalbuf") if no objections.

I completed a post-commit review of this patch. I think the patch is correct.

I found some of the variable names and comments challenging, so I made the
attached local edits in the course of the review. I don't know whether my
choices are better or just tailored to my biases. If someone thinks this
would improve the tree, I can polish this and commit it. Absent that,
consider these notes to myself and to other post-commit reviewers.

Attachments:

AdvanceXLInsertBuffer-code-review-v0.patchtext/plain; charset=us-asciiDownload
From: Noah Misch <noah@leadboat.com>

Cosmetic code review of AdvanceXLInsertBuffer().

This mostly changes material new in commit
bc22dc0e0ddc2dcb6043a732415019cc6b6bf683, but some of changes involve older
code.

==== Changes that avoid work -- move to separate patch(es)

Remove "upto = NewPageBeginPtr - 1;" so the "opportunistic" case doesn't
wait.

When the pg_atomic_compare_exchange_u64(&XLogCtl->InitializedEnd, ...)
loop advanced NewPageBeginPtr, we'd forget that progress for the next
iteration through the outer loop.  Use NewPageBeginPtr to control both
outer and inner loops, so we don't waste a compare-exchange in the outer
loop.

==== Code cosmetics

xlog.c has a standard that variables with names containing "End" refer
to the first byte position lacking the property in question.  Rename
some variables already having such last+1 semantics to include word
"End".

Prefer "ptr1 >= ptr2" to "ptr1 + XLOG_BLCKSZ > ptr" where both work.

Move variables to shorter-lived scope.  Rename variables.  Edit comments.

diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a..dd832d1 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -468,37 +468,45 @@ typedef struct XLogCtlData
 	pg_atomic_uint64 logFlushResult;	/* last byte + 1 flushed */
 
 	/*
-	 * First initialized page in the cache (first byte position).
+	 * First byte of the first page containing post-recovery data.  (If one
+	 * page contains both pre-recovery and post-recovery data, it's that page.
+	 * This still points to the page's first byte, even though that particular
+	 * byte is pre-recovery.)  Does not change after startup.  Treat bytes
+	 * below this as infeasible to write out to pg_wal.
 	 */
-	XLogRecPtr	InitializedFrom;
+	XLogRecPtr	PreRecoveryPagesEnd;
 
 	/*
-	 * Latest reserved for initialization page in the cache (last byte
-	 * position + 1).
+	 * First byte that no ongoing AdvanceXLInsertBuffer() has reserved.
 	 *
-	 * To change the identity of a buffer, you need to advance
-	 * InitializeReserved first.  To change the identity of a buffer that's
-	 * still dirty, the old page needs to be written out first, and for that
-	 * you need WALWriteLock, and you need to ensure that there are no
-	 * in-progress insertions to the page by calling
+	 * Every byte greater than or equal to this has not started
+	 * initialization.  Before changing the xlblocks for a buffer (the
+	 * buffer's identity), you first need to advance InitializeReservedEnd to
+	 * a value >= the buffer's next future xlblocks value.  To change the
+	 * identity of a buffer that's still dirty, the old page needs to be
+	 * written out first, and for that you need WALWriteLock, and you need to
+	 * ensure that there are no in-progress insertions to the page by calling
 	 * WaitXLogInsertionsToFinish().
 	 */
-	pg_atomic_uint64 InitializeReserved;
+	pg_atomic_uint64 InitializeReservedEnd;
 
 	/*
-	 * Latest initialized page in the cache (last byte position + 1).
+	 * First byte in a possibly-uninitialized page.
 	 *
-	 * InitializedUpTo is updated after the buffer initialization.  After
-	 * update, waiters got notification using InitializedUpToCondVar.
+	 * Every byte less than this is in a page with both "xlblocks" and the
+	 * "pages" bytes initialized.  Bytes greater than or equal to this may or
+	 * may be in pages lacking one of those initializations, since updating
+	 * this is the last step of initialization.  After update, waiters get
+	 * notification from InitializedEndCondVar.
 	 */
-	pg_atomic_uint64 InitializedUpTo;
-	ConditionVariable InitializedUpToCondVar;
+	pg_atomic_uint64 InitializedEnd;
+	ConditionVariable InitializedEndCondVar;
 
 	/*
 	 * These values do not change after startup, although the pointed-to pages
 	 * and xlblocks values certainly do.  xlblocks values are changed
 	 * lock-free according to the check for the xlog write position and are
-	 * accompanied by changes of InitializeReserved and InitializedUpTo.
+	 * accompanied by changes of InitializeReservedEnd and InitializedEnd.
 	 */
 	char	   *pages;			/* buffers for unwritten XLOG pages */
 	pg_atomic_uint64 *xlblocks; /* 1st byte ptr-s + XLOG_BLCKSZ */
@@ -1996,40 +2004,44 @@ static void
 AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 {
 	XLogCtlInsert *Insert = &XLogCtl->Insert;
-	int			nextidx;
-	XLogRecPtr	OldPageRqstPtr;
-	XLogwrtRqst WriteRqst;
-	XLogRecPtr	NewPageEndPtr = InvalidXLogRecPtr;
 	XLogRecPtr	NewPageBeginPtr;
-	XLogPageHeader NewPage;
-	XLogRecPtr	ReservedPtr;
 	int			npages pg_attribute_unused() = 0;
 
 	/*
-	 * We must run the loop below inside the critical section as we expect
-	 * XLogCtl->InitializedUpTo to eventually keep up.  The most of callers
-	 * already run inside the critical section. Except for WAL writer, which
-	 * passed 'opportunistic == true', and therefore we don't perform
-	 * operations that could error out.
-	 *
-	 * Start an explicit critical section anyway though.
+	 * Once we raise InitializeReservedEnd, we incur a duty to continue this
+	 * initialization work until we've updated xlblocks.  Non-opportunistic
+	 * callers already run inside a critical section.  Coincidentally, the
+	 * opportunistic caller (walwriter) prefers no pg_wal I/O here, so we
+	 * don't perform error-throwing operations for it.  Non-walwriter callers
+	 * accept PANIC on pg_wal I/O failure.
 	 */
 	Assert(CritSectionCount > 0 || opportunistic);
 	START_CRIT_SECTION();
 
 	/*--
-	 * Loop till we get all the pages in WAL buffer before 'upto' reserved for
-	 * initialization.  Multiple process can initialize different buffers with
-	 * this loop in parallel as following.
+	 * For !opportunistic callers, loop till we initialize all the pages in
+	 * WAL buffers before 'upto'.  For "opportunistic" caller walwriter, we
+	 * loop through all pages already written per the local LogwrtResult.Write
+	 * cache.  The idea is that walwriter writes out a batch of pages, then
+	 * initializes each future page that will occupy a buffer walwriter wrote
+	 * out.
 	 *
-	 * 1. Reserve page for initialization using XLogCtl->InitializeReserved.
+	 * Multiple process can initialize different buffers with this loop in
+	 * parallel as following.
+	 *
+	 * 1. Reserve page for initialization using XLogCtl->InitializeReservedEnd.
 	 * 2. Initialize the reserved page.
-	 * 3. Attempt to advance XLogCtl->InitializedUpTo,
+	 * 3. Attempt to advance XLogCtl->InitializedEnd,
 	 */
-	ReservedPtr = pg_atomic_read_u64(&XLogCtl->InitializeReserved);
-	while (upto >= ReservedPtr || opportunistic)
+	NewPageBeginPtr = pg_atomic_read_u64(&XLogCtl->InitializeReservedEnd);
+	while (upto >= NewPageBeginPtr || opportunistic)
 	{
-		Assert(ReservedPtr % XLOG_BLCKSZ == 0);
+		int			nextidx;
+		XLogRecPtr	OldPageEndPtr;
+		XLogPageHeader NewPage;
+		XLogRecPtr	NewPageEndPtr;
+
+		Assert(NewPageBeginPtr % XLOG_BLCKSZ == 0);
 
 		/*
 		 * Get ending-offset of the buffer page we need to replace.
@@ -2038,18 +2050,28 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * must wait to be written. If it was written, xlblocks will have this
 		 * position (or uninitialized)
 		 */
-		if (ReservedPtr + XLOG_BLCKSZ > XLogCtl->InitializedFrom + XLOG_BLCKSZ * XLOGbuffers)
-			OldPageRqstPtr = ReservedPtr + XLOG_BLCKSZ - (XLogRecPtr) XLOG_BLCKSZ * XLOGbuffers;
+		if (NewPageBeginPtr >= XLogCtl->PreRecoveryPagesEnd + XLOG_BLCKSZ * XLOGbuffers)
+		{
+			XLogRecPtr	OldPageBeginPtr = NewPageBeginPtr - (XLogRecPtr) XLOG_BLCKSZ * XLOGbuffers;
+
+			OldPageEndPtr = OldPageBeginPtr + XLOG_BLCKSZ;
+		}
 		else
-			OldPageRqstPtr = InvalidXLogRecPtr;
+		{
+			/*
+			 * Total WAL writes since recovery have been less than
+			 * wal_buffers.  There's no old page to replace.
+			 */
+			OldPageEndPtr = InvalidXLogRecPtr;
+		}
 
-		if (LogwrtResult.Write < OldPageRqstPtr && opportunistic)
+		if (LogwrtResult.Write < OldPageEndPtr && opportunistic)
 		{
 			/*
 			 * If we just want to pre-initialize as much as we can without
 			 * flushing, give up now.
 			 */
-			upto = ReservedPtr - 1;
+			Assert(upto == InvalidXLogRecPtr);
 			break;
 		}
 
@@ -2057,29 +2079,29 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Attempt to reserve the page for initialization.  Failure means that
 		 * this page got reserved by another process.
 		 */
-		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReserved,
-											&ReservedPtr,
-											ReservedPtr + XLOG_BLCKSZ))
+		if (!pg_atomic_compare_exchange_u64(&XLogCtl->InitializeReservedEnd,
+											&NewPageBeginPtr,
+											NewPageBeginPtr + XLOG_BLCKSZ))
 			continue;
 
 		/*
-		 * Wait till page gets correctly initialized up to OldPageRqstPtr.
+		 * Wait till page gets correctly initialized up to OldPageEndPtr.
 		 */
-		nextidx = XLogRecPtrToBufIdx(ReservedPtr);
-		while (pg_atomic_read_u64(&XLogCtl->InitializedUpTo) < OldPageRqstPtr)
-			ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+		nextidx = XLogRecPtrToBufIdx(NewPageBeginPtr);
+		while (pg_atomic_read_u64(&XLogCtl->InitializedEnd) < OldPageEndPtr)
+			ConditionVariableSleep(&XLogCtl->InitializedEndCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
 		ConditionVariableCancelSleep();
-		Assert(pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) == OldPageRqstPtr);
+		Assert(pg_atomic_read_u64(&XLogCtl->xlblocks[nextidx]) == OldPageEndPtr);
 
 		/* Fall through if it's already written out. */
-		if (LogwrtResult.Write < OldPageRqstPtr)
+		if (LogwrtResult.Write < OldPageEndPtr)
 		{
 			/* Nope, got work to do. */
 
 			/* Advance shared memory write request position */
 			SpinLockAcquire(&XLogCtl->info_lck);
-			if (XLogCtl->LogwrtRqst.Write < OldPageRqstPtr)
-				XLogCtl->LogwrtRqst.Write = OldPageRqstPtr;
+			if (XLogCtl->LogwrtRqst.Write < OldPageEndPtr)
+				XLogCtl->LogwrtRqst.Write = OldPageEndPtr;
 			SpinLockRelease(&XLogCtl->info_lck);
 
 			/*
@@ -2087,23 +2109,25 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			 * need to write it or if someone else already did.
 			 */
 			RefreshXLogWriteResult(LogwrtResult);
-			if (LogwrtResult.Write < OldPageRqstPtr)
+			if (LogwrtResult.Write < OldPageEndPtr)
 			{
-				WaitXLogInsertionsToFinish(OldPageRqstPtr);
+				WaitXLogInsertionsToFinish(OldPageEndPtr);
 
 				LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
 
 				RefreshXLogWriteResult(LogwrtResult);
-				if (LogwrtResult.Write >= OldPageRqstPtr)
+				if (LogwrtResult.Write >= OldPageEndPtr)
 				{
 					/* OK, someone wrote it already */
 					LWLockRelease(WALWriteLock);
 				}
 				else
 				{
+					XLogwrtRqst WriteRqst;
+
 					/* Have to write it ourselves */
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_START();
-					WriteRqst.Write = OldPageRqstPtr;
+					WriteRqst.Write = OldPageEndPtr;
 					WriteRqst.Flush = 0;
 					XLogWrite(WriteRqst, tli, false);
 					LWLockRelease(WALWriteLock);
@@ -2117,8 +2141,6 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 * Now the next buffer slot is free and we can set it up to be the
 		 * next output page.
 		 */
-		NewPageBeginPtr = ReservedPtr;
-		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
 
 		NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
@@ -2178,14 +2200,13 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 
 		/*
 		 * Make sure the initialization of the page becomes visible to others
-		 * before the xlblocks update. GetXLogBuffer() reads xlblocks without
-		 * holding a lock.
+		 * before the xlblocks update.
 		 */
 		pg_write_barrier();
 
 		/*-----
 		 * Update the value of XLogCtl->xlblocks[nextidx] and try to advance
-		 * XLogCtl->InitializedUpTo in a lock-less manner.
+		 * XLogCtl->InitializedEnd in a lock-less manner.
 		 *
 		 * First, let's provide a formal proof of the algorithm.  Let it be 'n'
 		 * process with the following variables in shared memory:
@@ -2228,21 +2249,22 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 		 *    where v gets stuck.  Q.E.D.
 		 *
 		 * To apply this proof to the code below, we assume
-		 * XLogCtl->InitializedUpTo will play the role of v with XLOG_BLCKSZ
+		 * XLogCtl->InitializedEnd will play the role of v with XLOG_BLCKSZ
 		 * granularity.  We also assume setting XLogCtl->xlblocks[nextidx] to
 		 * NewPageEndPtr to play the role of setting f[i] to true.  Also, note
 		 * that processes can't concurrently map different xlog locations to
 		 * the same nextidx because we previously requested that
-		 * XLogCtl->InitializedUpTo >= OldPageRqstPtr.  So, a xlog buffer can
+		 * XLogCtl->InitializedEnd >= OldPageEndPtr.  So, a xlog buffer can
 		 * be taken for initialization only once the previous initialization
-		 * takes effect on XLogCtl->InitializedUpTo.
+		 * takes effect on XLogCtl->InitializedEnd.
 		 */
 
+		NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
 		pg_atomic_write_u64(&XLogCtl->xlblocks[nextidx], NewPageEndPtr);
 
 		pg_write_barrier();
 
-		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedUpTo, &NewPageBeginPtr, NewPageEndPtr))
+		while (pg_atomic_compare_exchange_u64(&XLogCtl->InitializedEnd, &NewPageBeginPtr, NewPageEndPtr))
 		{
 			NewPageBeginPtr = NewPageEndPtr;
 			NewPageEndPtr = NewPageBeginPtr + XLOG_BLCKSZ;
@@ -2254,10 +2276,10 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 			{
 				/*
 				 * Page at nextidx wasn't initialized yet, so we can't move
-				 * InitializedUpto further. It will be moved by backend which
+				 * InitializedEnd further. It will be moved by backend which
 				 * will initialize nextidx.
 				 */
-				ConditionVariableBroadcast(&XLogCtl->InitializedUpToCondVar);
+				ConditionVariableBroadcast(&XLogCtl->InitializedEndCondVar);
 				break;
 			}
 		}
@@ -2272,8 +2294,8 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 	 * initialization.  However, some pages might be reserved by concurrent
 	 * processes.  Wait till they finish initialization.
 	 */
-	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedUpTo))
-		ConditionVariableSleep(&XLogCtl->InitializedUpToCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
+	while (upto >= pg_atomic_read_u64(&XLogCtl->InitializedEnd))
+		ConditionVariableSleep(&XLogCtl->InitializedEndCondVar, WAIT_EVENT_WAL_BUFFER_INIT);
 	ConditionVariableCancelSleep();
 
 	pg_read_barrier();
@@ -5205,9 +5227,9 @@ XLOGShmemInit(void)
 	pg_atomic_init_u64(&XLogCtl->logFlushResult, InvalidXLogRecPtr);
 	pg_atomic_init_u64(&XLogCtl->unloggedLSN, InvalidXLogRecPtr);
 
-	pg_atomic_init_u64(&XLogCtl->InitializeReserved, InvalidXLogRecPtr);
-	pg_atomic_init_u64(&XLogCtl->InitializedUpTo, InvalidXLogRecPtr);
-	ConditionVariableInit(&XLogCtl->InitializedUpToCondVar);
+	pg_atomic_init_u64(&XLogCtl->InitializeReservedEnd, InvalidXLogRecPtr);
+	pg_atomic_init_u64(&XLogCtl->InitializedEnd, InvalidXLogRecPtr);
+	ConditionVariableInit(&XLogCtl->InitializedEndCondVar);
 }
 
 /*
@@ -6227,20 +6249,19 @@ StartupXLOG(void)
 		memset(page + len, 0, XLOG_BLCKSZ - len);
 
 		pg_atomic_write_u64(&XLogCtl->xlblocks[firstIdx], endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
-		XLogCtl->InitializedFrom = endOfRecoveryInfo->lastPageBeginPtr;
+		pg_atomic_write_u64(&XLogCtl->InitializedEnd, endOfRecoveryInfo->lastPageBeginPtr + XLOG_BLCKSZ);
+		XLogCtl->PreRecoveryPagesEnd = endOfRecoveryInfo->lastPageBeginPtr;
 	}
 	else
 	{
 		/*
-		 * There is no partial block to copy. Just set InitializedUpTo, and
-		 * let the first attempt to insert a log record to initialize the next
-		 * buffer.
+		 * There is no partial block to copy. Just set InitializedEnd, and let
+		 * the first log record initialize the next buffer.
 		 */
-		pg_atomic_write_u64(&XLogCtl->InitializedUpTo, EndOfLog);
-		XLogCtl->InitializedFrom = EndOfLog;
+		pg_atomic_write_u64(&XLogCtl->InitializedEnd, EndOfLog);
+		XLogCtl->PreRecoveryPagesEnd = EndOfLog;
 	}
-	pg_atomic_write_u64(&XLogCtl->InitializeReserved, pg_atomic_read_u64(&XLogCtl->InitializedUpTo));
+	pg_atomic_write_u64(&XLogCtl->InitializeReservedEnd, pg_atomic_read_u64(&XLogCtl->InitializedEnd));
 
 	/*
 	 * Update local and shared status.  This is OK to do without any locks
#47Alexander Korotkov
aekorotkov@gmail.com
In reply to: Noah Misch (#46)
Re: Get rid of WALBufMappingLock

On Sat, Jul 5, 2025 at 7:16 AM Noah Misch <noah@leadboat.com> wrote:

On Mon, Mar 31, 2025 at 09:18:30PM +0300, Alexander Korotkov wrote:

I'm going to push the first patch ("nowalbuf") if no objections.

I completed a post-commit review of this patch. I think the patch is correct.

Great. I'm glad to hear that.

I found some of the variable names and comments challenging, so I made the
attached local edits in the course of the review. I don't know whether my
choices are better or just tailored to my biases. If someone thinks this
would improve the tree, I can polish this and commit it. Absent that,
consider these notes to myself and to other post-commit reviewers.

I took a look to these edits. They look good to me.

------
Regards,
Alexander Korotkov
Supabase