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] http://postgr.es/m/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.

-------
regards
Yura Sokolov aka funny-falcon
From 236b69ae1f524c7e8488da7244966e631324a0e3 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.soko...@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

From bc52c5ceec65bee210982c3df6f292f36db8ddd5 Mon Sep 17 00:00:00 2001
From: Yura Sokolov <y.soko...@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

Reply via email to