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