From 886a3ea87408e628bea08a9c77116343616ad032 Mon Sep 17 00:00:00 2001 From: Dmitrii Dolgov <9erthalion6@gmail.com> Date: Sun, 6 Apr 2025 16:47:16 +0200 Subject: [PATCH v4 6/8] Allow to resize shared memory without restart Add assing hook for shared_buffers to resize shared memory using space, introduced in the previous commits without requiring PostgreSQL restart. Essentially the implementation is based on two mechanisms: a ProcSignalBarrier is used to make sure all processes are starting the resize procedure simultaneously, and a global Barrier is used to coordinate after that and make sure all finished processes are waiting for others that are in progress. The resize process looks like this: * The GUC assign hook sets a flag to let the Postmaster know that resize was requested. * Postmaster verifies the flag in the event loop, and starts the resize by emitting a ProcSignal barrier. * All processes, that participate in ProcSignal mechanism, begin to process ProcSignal barrier. First a process waits until all processes have confirmed they received the message and can start simultaneously. * Every process recalculates shared memory size based on the new NBuffers and extend it using mremap. One elected process signals the postmaster to do the same. * When finished, every process waits on a global ShmemControl barrier, untill all others are finished as well. This way we ensure three stages with clear boundaries: before the resize, when all processes use old NBuffers; during the resize, when processes have mix of old and new NBuffers, and wait until it's done; after the resize, when all processes use new NBuffers. * After all processes are using new value, one of them will initialize new shared structures (buffer blocks, descriptors, etc) as needed and broadcast new value of NBuffers via ShmemControl in shared memory. Other backends are waiting for this operation to finish as well. Then the barrier is lifted and everything goes as usual. Since resizing takes time, we need to take into account that during that time: - New backends can be spawned. They will check status of the barrier early during the bootstrap, and wait until everything is over to work with the new NBuffers value. - Old backends can exit before attempting to resize. Synchronization used between backends relies on ProcSignalBarrier and waits for all participants received the message at the beginning to gather all existing backends. - Some backends might be blocked and not responsing either before or after receiving the message. In the first case such backend still have ProcSignalSlot and should be waited for, in the second case shared barrier will make sure we still waiting for those backends. In any case there is an unbounded wait. - Backends might join barrier in disjoint groups with some time in between. That means that relying only on the shared dynamic barrier is not enough -- it will only synchronize resize procedure withing those groups. That's why we wait first for all participants of ProcSignal mechanism who received the message. Here is how it looks like after raising shared_buffers from 128 MB to 512 MB and calling pg_reload_conf(): -- 128 MB 7f90cde00000-7f90d4fa6000 /dev/zero (deleted) 7f90d4fa6000-7f914de00000 7f914de00000-7f915cfa8000 /dev/zero (deleted) ^ buffers mapping, ~241 MB 7f915cfa8000-7f944de00000 7f944de00000-7f94550a8000 /dev/zero (deleted) 7f94550a8000-7f94cde00000 7f94cde00000-7f94d4fe8000 /dev/zero (deleted) 7f94d4fe8000-7f954de00000 7f954de00000-7f9554ff6000 /dev/zero (deleted) 7f9554ff6000-7f958de00000 7f958de00000-7f959508a000 /dev/zero (deleted) 7f959508a000-7f95cde00000 -- 512 MB 7f90cde00000-7f90d5126000 /dev/zero (deleted) 7f90d5126000-7f914de00000 7f914de00000-7f9175128000 /dev/zero (deleted) ^ buffers mapping, ~627 MB 7f9175128000-7f944de00000 7f944de00000-7f9455528000 /dev/zero (deleted) 7f9455528000-7f94cde00000 7f94cde00000-7f94d5228000 /dev/zero (deleted) 7f94d5228000-7f954de00000 7f954de00000-7f9555266000 /dev/zero (deleted) 7f9555266000-7f958de00000 7f958de00000-7f95954aa000 /dev/zero (deleted) 7f95954aa000-7f95cde00000 The implementation supports only increasing of shared_buffers. For decreasing the value a similar procedure is needed. But the buffer blocks with data have to be drained first, so that the actual data set fits into the new smaller space. From experiment it turns out that shared mappings have to be extended separately for each process that uses them. Another rough edge is that a backend blocked on ReadCommand will not apply shared_buffers change until it receives something. Note, that mremap is Linux specific, thus the implementation not very portable. Authors: Dmitrii Dolgov, Ashutosh Bapat --- src/backend/port/sysv_shmem.c | 413 ++++++++++++++++++ src/backend/postmaster/postmaster.c | 18 + src/backend/storage/buffer/buf_init.c | 75 ++-- src/backend/storage/ipc/ipci.c | 18 +- src/backend/storage/ipc/procsignal.c | 46 ++ src/backend/storage/ipc/shmem.c | 23 +- src/backend/tcop/postgres.c | 10 + .../utils/activity/wait_event_names.txt | 3 + src/backend/utils/misc/guc_tables.c | 4 +- src/include/miscadmin.h | 1 + src/include/storage/bufmgr.h | 2 +- src/include/storage/ipc.h | 3 + src/include/storage/lwlocklist.h | 1 + src/include/storage/pg_shmem.h | 26 ++ src/include/storage/pmsignal.h | 1 + src/include/storage/procsignal.h | 1 + src/tools/pgindent/typedefs.list | 1 + 17 files changed, 603 insertions(+), 43 deletions(-) diff --git a/src/backend/port/sysv_shmem.c b/src/backend/port/sysv_shmem.c index f46d9d5d9cd..a3437973784 100644 --- a/src/backend/port/sysv_shmem.c +++ b/src/backend/port/sysv_shmem.c @@ -30,13 +30,19 @@ #include "miscadmin.h" #include "port/pg_bitutils.h" #include "portability/mem.h" +#include "storage/bufmgr.h" #include "storage/dsm.h" #include "storage/fd.h" #include "storage/ipc.h" +#include "storage/lwlock.h" #include "storage/pg_shmem.h" +#include "storage/pmsignal.h" +#include "storage/procsignal.h" +#include "storage/shmem.h" #include "utils/guc.h" #include "utils/guc_hooks.h" #include "utils/pidfile.h" +#include "utils/wait_event.h" /* @@ -105,6 +111,13 @@ typedef struct AnonymousMapping static AnonymousMapping Mappings[ANON_MAPPINGS]; +/* Flag telling postmaster that resize is needed */ +volatile bool pending_pm_shmem_resize = false; + +/* Keeps track of the previous NBuffers value */ +static int NBuffersOld = -1; +static int NBuffersPending = -1; + /* Keeps track of used mapping segments */ static int next_free_segment = 0; @@ -176,6 +189,49 @@ static Size reserved_offset = 0; */ static bool huge_pages_on = false; +/* + * Flag telling that we have prepared the memory layout to be resizable. If + * false after all shared memory segments creation, it means we failed to setup + * needed layout and falled back to the regular non-resizable approach. + */ +static bool shmem_resizable = false; + +/* + * Currently broadcasted value of NBuffers in shared memory. + * + * Most of the time this value is going to be equal to NBuffers. But if + * postmaster is resizing shared memory and a new backend was created + * at the same time, there is a possibility for the new backend to inherit the + * old NBuffers value, but miss the resize signal if ProcSignal infrastructure + * was not initialized yet. Consider this situation: + * + * Postmaster ------> New Backend + * | | + * | Launch + * | | + * | Inherit NBuffers + * | | + * Resize NBuffers | + * | | + * Emit Barrier | + * | Init ProcSignal + * | | + * Finish resize | + * | | + * New NBuffers Old NBuffers + * + * In this case the backend is not yet ready to receive a signal from + * EmitProcSignalBarrier, and will be ignored. The same happens if ProcSignal + * is initialized even later, after the resizing was finished. + * + * To address resulting inconsistency, postmaster broadcasts the current + * NBuffers value via shared memory. Every new backend has to verify this value + * before it will access the buffer pool: if it differs from its own value, + * this indicates a shared memory resize has happened and the backend has to + * first synchronize with rest of the pack. + */ +ShmemControl *ShmemCtrl = NULL; + static void *InternalIpcMemoryCreate(IpcMemoryKey memKey, Size size); static void IpcMemoryDetach(int status, Datum shmaddr); static void IpcMemoryDelete(int status, Datum shmId); @@ -769,6 +825,7 @@ CreateAnonymousSegment(AnonymousMapping *mapping, Pointer base) { Size total_reserved = (Size) MaxAvailableMemory * BLCKSZ; + shmem_resizable = true; reserved_offset += total_reserved * SHMEM_RESIZE_RATIO[next_free_segment]; } @@ -964,6 +1021,315 @@ AnonymousShmemDetach(int status, Datum arg) } } +/* + * Resize all shared memory segments based on the current NBuffers value, which + * is is applied from NBuffersPending. The actual segment resizing is done via + * mremap, which will fail if is not sufficient space to expand the mapping. + * When finished, based on the new and old values initialize new buffer blocks + * if any. + * + * If reinitializing took place, as the last step this function does buffers + * reinitialization as well and broadcasts the new value of NSharedBuffers. All + * of that needs to be done only by one backend, the first one that managed to + * grab the ShmemResizeLock. + */ +bool +AnonymousShmemResize(void) +{ + int numSemas; + bool reinit = false; + void *ptr = MAP_FAILED; + NBuffers = NBuffersPending; + + elog(DEBUG1, "Resize shmem from %d to %d", NBuffersOld, NBuffers); + + /* + * XXX: Where to reset the flag is still an open question. E.g. do we + * consider a no-op when NBuffers is equal to NBuffersOld a genuine resize + * and reset the flag? + */ + pending_pm_shmem_resize = false; + + /* + * XXX: Currently only increasing of shared_buffers is supported. For + * decreasing something similar has to be done, but buffer blocks with + * data have to be drained first. + */ + if(NBuffersOld > NBuffers) + return false; + + for(int i = 0; i < next_free_segment; i++) + { + /* Note that CalculateShmemSize indirectly depends on NBuffers */ + Size new_size = CalculateShmemSize(&numSemas, i); + AnonymousMapping *m = &Mappings[i]; + + if (m->shmem == NULL) + continue; + + if (m->shmem_size == new_size) + continue; + + /* Clean up some reserved space to resize into */ + if (munmap(m->shmem + m->shmem_size, new_size - m->shmem_size) == -1) + ereport(FATAL, + (errcode(ERRCODE_SYSTEM_ERROR), + errmsg("could not unmap %zu from reserved shared memory %p: %m", + new_size - m->shmem_size, m->shmem))); + + /* Claim the unused space */ + elog(DEBUG1, "segment[%s]: remap from %zu to %zu at address %p", + MappingName(m->shmem_segment), m->shmem_size, + new_size, m->shmem); + + ptr = mremap(m->shmem, m->shmem_size, new_size, 0); + if (ptr == MAP_FAILED) + ereport(FATAL, + (errcode(ERRCODE_SYSTEM_ERROR), + errmsg("could not resize shared memory segment %s [%p] to %d (%zu): %m", + MappingName(m->shmem_segment), m->shmem, NBuffers, + new_size))); + + reinit = true; + m->shmem_size = new_size; + } + + if (reinit) + { + if(IsUnderPostmaster && + LWLockConditionalAcquire(ShmemResizeLock, LW_EXCLUSIVE)) + { + /* + * If the new NBuffers was already broadcasted, the buffer pool was + * already initialized before. + * + * Since we're not on a hot path, we use lwlocks and do not need to + * involve memory barrier. + */ + if(pg_atomic_read_u32(&ShmemCtrl->NSharedBuffers) != NBuffers) + { + /* + * Allow the first backend that managed to get the lock to + * reinitialize the new portion of buffer pool. Every other + * process will wait on the shared barrier for that to finish, + * since it's a part of the SHMEM_RESIZE_DONE phase. + * + * Note that it's enough when only one backend will do that, + * even the ShmemInitStruct part. The reason is that resized + * shared memory will maintain the same addresses, meaning that + * all the pointers are still valid, and we only need to update + * structures size in the ShmemIndex once -- any other backend + * will pick up this shared structure from the index. + * + * XXX: This is the right place for buffer eviction as well. + */ + BufferManagerShmemInit(NBuffersOld); + + /* If all fine, broadcast the new value */ + pg_atomic_write_u32(&ShmemCtrl->NSharedBuffers, NBuffers); + } + + LWLockRelease(ShmemResizeLock); + } + } + + return true; +} + +/* + * We are asked to resize shared memory. Wait for all ProcSignal participants + * to join the barrier, then do the resize and wait on the barrier until all + * participating finish resizing as well -- otherwise we face danger of + * inconsistency between backends. + * + * XXX: If a backend is blocked on ReadCommand in PostgresMain, it will not + * proceed with AnonymousShmemResize after receiving SIGHUP, until something + * will be sent. + */ +bool +ProcessBarrierShmemResize(Barrier *barrier) +{ + elog(DEBUG1, "Handle a barrier for shmem resizing from %d to %d, %d", + NBuffersOld, NBuffersPending, pending_pm_shmem_resize); + + /* Wait until we have seen the new NBuffers value */ + if (!pending_pm_shmem_resize) + return false; + + /* + * First thing to do after attaching to the barrier is to wait for others. + * We can't simply use BarrierArriveAndWait, because backends might arrive + * here in disjoint groups, e.g. first two backends, pause, then second two + * backends. If the resize is quick enough that can lead to a situation + * when the first group is already finished before the second has appeared, + * and the barrier will only synchonize withing those groups. + */ + if (BarrierAttach(barrier) == SHMEM_RESIZE_REQUESTED) + WaitForProcSignalBarrierReceived( + pg_atomic_read_u64(&ShmemCtrl->Generation)); + + /* + * Now start the procedure, and elect one backend to ping postmaster to do + * the same. + * + * XXX: If we need to be able to abort resizing, this has to be done later, + * after the SHMEM_RESIZE_DONE. + */ + if (BarrierArriveAndWait(barrier, WAIT_EVENT_SHMEM_RESIZE_START)) + { + Assert(IsUnderPostmaster); + SendPostmasterSignal(PMSIGNAL_SHMEM_RESIZE); + } + + AnonymousShmemResize(); + + /* The second phase means the resize has finished, SHMEM_RESIZE_DONE */ + BarrierArriveAndWait(barrier, WAIT_EVENT_SHMEM_RESIZE_DONE); + + BarrierDetach(barrier); + return true; +} + +/* + * GUC assign hook for shared_buffers. It's recommended for an assign hook to + * be as minimal as possible, thus we just request shared memory resize and + * remember the previous value. + */ +void +assign_shared_buffers(int newval, void *extra, bool *pending) +{ + elog(DEBUG1, "Received SIGHUP for shmem resizing"); + + /* Request shared memory resize only when it was initialized */ + if (next_free_segment != 0) + { + elog(DEBUG1, "Set pending signal"); + pending_pm_shmem_resize = true; + *pending = true; + NBuffersPending = newval; + } + + NBuffersOld = NBuffers; +} + +/* + * Test if we have somehow missed a shmem resize signal and NBuffers value + * differs from NSharedBuffers. If yes, catchup and do resize. + */ +void +AdjustShmemSize(void) +{ + uint32 NSharedBuffers = pg_atomic_read_u32(&ShmemCtrl->NSharedBuffers); + + if (NSharedBuffers != NBuffers) + { + /* + * If the broadcasted shared_buffers is different from the one we see, + * it could be that the backend has missed a resize signal. To avoid + * any inconsistency, adjust the shared mappings, before having a + * chance to access the buffer pool. + */ + ereport(LOG, + (errmsg("shared_buffers has been changed from %d to %d, " + "resize shared memory", + NBuffers, NSharedBuffers))); + NBuffers = NSharedBuffers; + AnonymousShmemResize(); + } +} + +/* + * Start resizing procedure, making sure all existing processes will have + * consistent view of shared memory size. Must be called only in postmaster. + */ +void +CoordinateShmemResize(void) +{ + elog(DEBUG1, "Coordinating shmem resize from %d to %d", + NBuffersOld, NBuffers); + Assert(!IsUnderPostmaster); + + /* + * We use dynamic barrier to help dealing with backends that were spawned + * during the resize. + */ + BarrierInit(&ShmemCtrl->Barrier, 0); + + /* + * If the value did not change, or shared memory segments are not + * initialized yet, skip the resize. + */ + if (NBuffersPending == NBuffersOld || next_free_segment == 0) + { + elog(DEBUG1, "Skip resizing, new %d, old %d, free segment %d", + NBuffers, NBuffersOld, next_free_segment); + return; + } + + /* + * Shared memory resize requires some coordination done by postmaster, + * and consists of three phases: + * + * - Before the resize all existing backends have the same old NBuffers. + * - When resize is in progress, backends are expected to have a + * mixture of old a new values. They're not allowed to touch buffer + * pool during this time frame. + * - After resize has been finished, all existing backends, that can access + * the buffer pool, are expected to have the same new value of NBuffers. + * + * Those phases are ensured by joining the shared barrier associated with + * the procedure. Since resizing takes time, we need to take into account + * that during that time: + * + * - New backends can be spawned. They will check status of the barrier + * early during the bootstrap, and wait until everything is over to work + * with the new NBuffers value. + * + * - Old backends can exit before attempting to resize. Synchronization + * used between backends relies on ProcSignalBarrier and waits for all + * participants received the message at the beginning to gather all + * existing backends. + * + * - Some backends might be blocked and not responsing either before or + * after receiving the message. In the first case such backend still + * have ProcSignalSlot and should be waited for, in the second case + * shared barrier will make sure we still waiting for those backends. In + * any case there is an unbounded wait. + * + * - Backends might join barrier in disjoint groups with some time in + * between. That means that relying only on the shared dynamic barrier is + * not enough -- it will only synchronize resize procedure withing those + * groups. That's why we wait first for all participants of ProcSignal + * mechanism who received the message. + */ + elog(DEBUG1, "Emit a barrier for shmem resizing"); + pg_atomic_init_u64(&ShmemCtrl->Generation, + EmitProcSignalBarrier(PROCSIGNAL_BARRIER_SHMEM_RESIZE)); + + /* To order everything after setting Generation value */ + pg_memory_barrier(); + + /* + * After that postmaster waits for PMSIGNAL_SHMEM_RESIZE as a sign that all + * the rest of the pack has started the procedure and it can resize shared + * memory as well. + * + * Normally we would call WaitForProcSignalBarrier here to wait until every + * backend has reported on the ProcSignalBarrier. But for shared memory + * resize we don't need this, as every participating backend will + * synchronize on the ProcSignal barrier. In fact even if we would like to + * wait here, it wouldn't be possible -- we're in the postmaster, without + * any waiting infrastructure available. + * + * If at some point it will turn out that waiting is essential, we would + * need to consider some alternatives. E.g. it could be a designated + * coordination process, which is not a postmaster. Another option would be + * to introduce a CoordinateShmemResize lock and allow only one process to + * take it (this probably would have to be something different than + * LWLocks, since they block interrupts, and coordination relies on them). + */ +} + /* * PGSharedMemoryCreate * @@ -1271,3 +1637,50 @@ PGSharedMemoryDetach(void) } } } + +void +WaitOnShmemBarrier() +{ + Barrier *barrier = &ShmemCtrl->Barrier; + + /* Nothing to do if resizing is not started */ + if (BarrierPhase(barrier) < SHMEM_RESIZE_START) + return; + + BarrierAttach(barrier); + + /* Otherwise wait through all available phases */ + while (BarrierPhase(barrier) < SHMEM_RESIZE_DONE) + { + ereport(LOG, (errmsg("ProcSignal barrier is in phase %d, waiting", + BarrierPhase(barrier)))); + + BarrierArriveAndWait(barrier, 0); + } + + BarrierDetach(barrier); +} + +void +ShmemControlInit(void) +{ + bool foundShmemCtrl; + + ShmemCtrl = (ShmemControl *) + ShmemInitStruct("Shmem Control", sizeof(ShmemControl), + &foundShmemCtrl); + + if (!foundShmemCtrl) + { + /* + * The barrier is missing here, it will be initialized right before + * starting the resizing process as a convenient way to reset it. + */ + + /* Initialize with the currently known value */ + pg_atomic_init_u32(&ShmemCtrl->NSharedBuffers, NBuffers); + + /* shmem_resizable should be initialized by now */ + ShmemCtrl->Resizable = shmem_resizable; + } +} diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c index 3fe45de5da0..196f233fe0e 100644 --- a/src/backend/postmaster/postmaster.c +++ b/src/backend/postmaster/postmaster.c @@ -425,6 +425,7 @@ static void process_pm_pmsignal(void); static void process_pm_child_exit(void); static void process_pm_reload_request(void); static void process_pm_shutdown_request(void); +static void process_pm_shmem_resize(void); static void dummy_handler(SIGNAL_ARGS); static void CleanupBackend(PMChild *bp, int exitstatus); static void HandleChildCrash(int pid, int exitstatus, const char *procname); @@ -1693,6 +1694,9 @@ ServerLoop(void) if (pending_pm_pmsignal) process_pm_pmsignal(); + if (pending_pm_shmem_resize) + process_pm_shmem_resize(); + if (events[i].events & WL_SOCKET_ACCEPT) { ClientSocket s; @@ -2038,6 +2042,17 @@ process_pm_reload_request(void) } } +static void +process_pm_shmem_resize(void) +{ + /* + * Failure to resize is considered to be fatal and will not be + * retried, which means we can disable pending flag right here. + */ + pending_pm_shmem_resize = false; + CoordinateShmemResize(); +} + /* * pg_ctl uses SIGTERM, SIGINT and SIGQUIT to request different types of * shutdown. @@ -3851,6 +3866,9 @@ process_pm_pmsignal(void) request_state_update = true; } + if (CheckPostmasterSignal(PMSIGNAL_SHMEM_RESIZE)) + AnonymousShmemResize(); + /* * Try to advance postmaster's state machine, if a child requests it. */ diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c index bd68b69ee98..ac844b114bd 100644 --- a/src/backend/storage/buffer/buf_init.c +++ b/src/backend/storage/buffer/buf_init.c @@ -17,6 +17,7 @@ #include "storage/aio.h" #include "storage/buf_internals.h" #include "storage/bufmgr.h" +#include "storage/pg_shmem.h" BufferDescPadded *BufferDescriptors; char *BufferBlocks; @@ -24,7 +25,6 @@ ConditionVariableMinimallyPadded *BufferIOCVArray; WritebackContext BackendWritebackContext; CkptSortItem *CkptBufferIds; - /* * Data Structures: * buffers live in a freelist and a lookup data structure. @@ -62,18 +62,28 @@ CkptSortItem *CkptBufferIds; * Initialize shared buffer pool * * This is called once during shared-memory initialization (either in the - * postmaster, or in a standalone backend). Size of data structures initialized - * here depends on NBuffers, and to be able to change NBuffers without a - * restart we store each structure into a separate shared memory segment, which - * could be resized on demand. + * postmaster, or in a standalone backend) or during shared-memory resize. Size + * of data structures initialized here depends on NBuffers, and to be able to + * change NBuffers without a restart we store each structure into a separate + * shared memory segment, which could be resized on demand. + * + * FirstBufferToInit tells where to start initializing buffers. For + * initialization it always will be zero, but when resizing shared-memory it + * indicates the number of already initialized buffers. + * + * No locks are taking in this function, it is the caller responsibility to + * make sure only one backend can work with new buffers. */ void -BufferManagerShmemInit(void) +BufferManagerShmemInit(int FirstBufferToInit) { bool foundBufs, foundDescs, foundIOCV, foundBufCkpt; + int i; + elog(DEBUG1, "BufferManagerShmemInit from %d to %d", + FirstBufferToInit, NBuffers); /* Align descriptors to a cacheline boundary. */ BufferDescriptors = (BufferDescPadded *) @@ -110,43 +120,44 @@ BufferManagerShmemInit(void) { /* should find all of these, or none of them */ Assert(foundDescs && foundBufs && foundIOCV && foundBufCkpt); - /* note: this path is only taken in EXEC_BACKEND case */ - } - else - { - int i; - /* - * Initialize all the buffer headers. + * note: this path is only taken in EXEC_BACKEND case when initializing + * shared memory, or in all cases when resizing shared memory. */ - for (i = 0; i < NBuffers; i++) - { - BufferDesc *buf = GetBufferDescriptor(i); + } - ClearBufferTag(&buf->tag); +#ifndef EXEC_BACKEND + /* + * Initialize all the buffer headers. + */ + for (i = FirstBufferToInit; i < NBuffers; i++) + { + BufferDesc *buf = GetBufferDescriptor(i); - pg_atomic_init_u32(&buf->state, 0); - buf->wait_backend_pgprocno = INVALID_PROC_NUMBER; + ClearBufferTag(&buf->tag); - buf->buf_id = i; + pg_atomic_init_u32(&buf->state, 0); + buf->wait_backend_pgprocno = INVALID_PROC_NUMBER; - pgaio_wref_clear(&buf->io_wref); + buf->buf_id = i; - /* - * Initially link all the buffers together as unused. Subsequent - * management of this list is done by freelist.c. - */ - buf->freeNext = i + 1; + pgaio_wref_clear(&buf->io_wref); - LWLockInitialize(BufferDescriptorGetContentLock(buf), - LWTRANCHE_BUFFER_CONTENT); + /* + * Initially link all the buffers together as unused. Subsequent + * management of this list is done by freelist.c. + */ + buf->freeNext = i + 1; - ConditionVariableInit(BufferDescriptorGetIOCV(buf)); - } + LWLockInitialize(BufferDescriptorGetContentLock(buf), + LWTRANCHE_BUFFER_CONTENT); - /* Correct last entry of linked list */ - GetBufferDescriptor(NBuffers - 1)->freeNext = FREENEXT_END_OF_LIST; + ConditionVariableInit(BufferDescriptorGetIOCV(buf)); } +#endif + + /* Correct last entry of linked list */ + GetBufferDescriptor(NBuffers - 1)->freeNext = FREENEXT_END_OF_LIST; /* Init other shared buffer-management stuff */ StrategyInitialize(!foundDescs); diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c index 9d00b80b4f8..abeb91e24fd 100644 --- a/src/backend/storage/ipc/ipci.c +++ b/src/backend/storage/ipc/ipci.c @@ -84,6 +84,9 @@ RequestAddinShmemSpace(Size size) * * If num_semaphores is not NULL, it will be set to the number of semaphores * required. + * + * XXX: Calculation for non main shared memory segments are incorrect, it + * includes more than needed for buffers only. */ Size CalculateShmemSize(int *num_semaphores, int shmem_segment) @@ -151,6 +154,14 @@ CalculateShmemSize(int *num_semaphores, int shmem_segment) size = add_size(size, SlotSyncShmemSize()); size = add_size(size, AioShmemSize()); + /* + * XXX: For some reason slightly more memory is needed for larger + * shared_buffers, but this size is enough for any large value I've tested + * with. Is it a mistake in how slots are split, or there was a hidden + * inconsistency in shmem calculation? + */ + size = add_size(size, 1024 * 1024 * 100); + /* include additional requested shmem from preload libraries */ size = add_size(size, total_addin_request); @@ -298,7 +309,7 @@ CreateOrAttachShmemStructs(void) CommitTsShmemInit(); SUBTRANSShmemInit(); MultiXactShmemInit(); - BufferManagerShmemInit(); + BufferManagerShmemInit(0); /* * Set up lock manager @@ -310,6 +321,11 @@ CreateOrAttachShmemStructs(void) */ PredicateLockShmemInit(); + /* + * Set up shared memory resize manager + */ + ShmemControlInit(); + /* * Set up process table */ diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c index 8e313ad9bf8..35c42f260a8 100644 --- a/src/backend/storage/ipc/procsignal.c +++ b/src/backend/storage/ipc/procsignal.c @@ -27,6 +27,7 @@ #include "storage/condition_variable.h" #include "storage/ipc.h" #include "storage/latch.h" +#include "storage/pg_shmem.h" #include "storage/shmem.h" #include "storage/sinval.h" #include "storage/smgr.h" @@ -112,6 +113,10 @@ static bool CheckProcSignal(ProcSignalReason reason); static void CleanupProcSignalState(int status, Datum arg); static void ResetProcSignalBarrierBits(uint32 flags); +#ifdef DEBUG_SHMEM_RESIZE +bool delay_proc_signal_init = false; +#endif + /* * ProcSignalShmemSize * Compute space needed for ProcSignal's shared memory @@ -175,6 +180,43 @@ ProcSignalInit(char *cancel_key, int cancel_key_len) uint32 old_pss_pid; Assert(cancel_key_len >= 0 && cancel_key_len <= MAX_CANCEL_KEY_LENGTH); + +#ifdef DEBUG_SHMEM_RESIZE + /* + * Introduced for debugging purposes. You can change the variable at + * runtime using gdb, then start new backends with delayed ProcSignal + * initialization. Simple pg_usleep wont work here due to SIGHUP interrupt + * needed for testing. Taken from pg_sleep; + */ + if (delay_proc_signal_init) + { +#define GetNowFloat() ((float8) GetCurrentTimestamp() / 1000000.0) + float8 endtime = GetNowFloat() + 5; + + for (;;) + { + float8 delay; + long delay_ms; + + CHECK_FOR_INTERRUPTS(); + + delay = endtime - GetNowFloat(); + if (delay >= 600.0) + delay_ms = 600000; + else if (delay > 0.0) + delay_ms = (long) (delay * 1000.0); + else + break; + + (void) WaitLatch(MyLatch, + WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH, + delay_ms, + WAIT_EVENT_PG_SLEEP); + ResetLatch(MyLatch); + } + } +#endif + if (MyProcNumber < 0) elog(ERROR, "MyProcNumber not set"); if (MyProcNumber >= NumProcSignalSlots) @@ -614,6 +656,10 @@ ProcessProcSignalBarrier(void) case PROCSIGNAL_BARRIER_SMGRRELEASE: processed = ProcessBarrierSmgrRelease(); break; + case PROCSIGNAL_BARRIER_SHMEM_RESIZE: + processed = ProcessBarrierShmemResize( + &ShmemCtrl->Barrier); + break; } /* diff --git a/src/backend/storage/ipc/shmem.c b/src/backend/storage/ipc/shmem.c index 389abc82519..0fd421f004e 100644 --- a/src/backend/storage/ipc/shmem.c +++ b/src/backend/storage/ipc/shmem.c @@ -493,17 +493,26 @@ ShmemInitStructInSegment(const char *name, Size size, bool *foundPtr, { /* * Structure is in the shmem index so someone else has allocated it - * already. The size better be the same as the size we are trying to - * initialize to, or there is a name conflict (or worse). + * already. Verify the structure's size: + * - If it's the same, we've found the expected structure. + * - If it's different, we're resizing the expected structure. + * + * XXX: There is an implicit assumption this can only happen in + * "resizable" segments, where only one shared structure is allowed. + * This has to be implemented more cleanly. */ if (result->size != size) { - LWLockRelease(ShmemIndexLock); - ereport(ERROR, - (errmsg("ShmemIndex entry size is wrong for data structure" - " \"%s\": expected %zu, actual %zu", - name, size, result->size))); + Size delta = size - result->size; + + result->size = size; + + /* Reflect size change in the shared segment */ + SpinLockAcquire(Segments[shmem_segment].ShmemLock); + Segments[shmem_segment].ShmemSegHdr->freeoffset += delta; + SpinLockRelease(Segments[shmem_segment].ShmemLock); } + structPtr = result->location; } else diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c index b1fba850f02..58f1a05fd2a 100644 --- a/src/backend/tcop/postgres.c +++ b/src/backend/tcop/postgres.c @@ -62,6 +62,7 @@ #include "rewrite/rewriteHandler.h" #include "storage/bufmgr.h" #include "storage/ipc.h" +#include "storage/pg_shmem.h" #include "storage/pmsignal.h" #include "storage/proc.h" #include "storage/procsignal.h" @@ -4311,6 +4312,15 @@ PostgresMain(const char *dbname, const char *username) */ BeginReportingGUCOptions(); + /* Verify the shared barrier, if it's still active: join and wait. */ + WaitOnShmemBarrier(); + + /* + * After waiting on the barrier above we guaranteed to have NSharedBuffers + * broadcasted, so we can use it in the function below. + */ + AdjustShmemSize(); + /* * Also set up handler to log session end; we have to wait till now to be * sure Log_disconnections has its final value. diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt index 8bce14c38fd..e0ba8384fdd 100644 --- a/src/backend/utils/activity/wait_event_names.txt +++ b/src/backend/utils/activity/wait_event_names.txt @@ -155,6 +155,8 @@ REPLICATION_ORIGIN_DROP "Waiting for a replication origin to become inactive so REPLICATION_SLOT_DROP "Waiting for a replication slot to become inactive so it can be dropped." RESTORE_COMMAND "Waiting for to complete." SAFE_SNAPSHOT "Waiting to obtain a valid snapshot for a READ ONLY DEFERRABLE transaction." +SHMEM_RESIZE_START "Waiting for other backends to start resizing shared memory." +SHMEM_RESIZE_DONE "Waiting for other backends to finish resizing shared memory." 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." @@ -351,6 +353,7 @@ DSMRegistry "Waiting to read or update the dynamic shared memory registry." InjectionPoint "Waiting to read or update information related to injection points." SerialControl "Waiting to read or update shared pg_serial state." AioWorkerSubmissionQueue "Waiting to access AIO worker submission queue." +ShmemResize "Waiting to resize shared memory." # # END OF PREDEFINED LWLOCKS (DO NOT CHANGE THIS LINE) diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c index dede37f7905..1e70853ccdb 100644 --- a/src/backend/utils/misc/guc_tables.c +++ b/src/backend/utils/misc/guc_tables.c @@ -2354,14 +2354,14 @@ struct config_int ConfigureNamesInt[] = * checking for overflow, so we mustn't allow more than INT_MAX / 2. */ { - {"shared_buffers", PGC_POSTMASTER, RESOURCES_MEM, + {"shared_buffers", PGC_SIGHUP, RESOURCES_MEM, gettext_noop("Sets the number of shared memory buffers used by the server."), NULL, GUC_UNIT_BLOCKS }, &NBuffers, 16384, 16, INT_MAX / 2, - NULL, NULL, NULL + NULL, assign_shared_buffers, NULL }, { diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h index 0d8528b2875..405d0a7e65d 100644 --- a/src/include/miscadmin.h +++ b/src/include/miscadmin.h @@ -173,6 +173,7 @@ extern PGDLLIMPORT char *DataDir; extern PGDLLIMPORT int data_directory_mode; extern PGDLLIMPORT int NBuffers; +extern PGDLLIMPORT int MaxAvailableMemory; extern PGDLLIMPORT int MaxBackends; extern PGDLLIMPORT int MaxConnections; extern PGDLLIMPORT int max_worker_processes; diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h index 1977001e533..52633dd7537 100644 --- a/src/include/storage/bufmgr.h +++ b/src/include/storage/bufmgr.h @@ -307,7 +307,7 @@ extern void LimitAdditionalLocalPins(uint32 *additional_pins); extern bool EvictUnpinnedBuffer(Buffer buf); /* in buf_init.c */ -extern void BufferManagerShmemInit(void); +extern void BufferManagerShmemInit(int); extern Size BufferManagerShmemSize(int); /* in localbuf.c */ diff --git a/src/include/storage/ipc.h b/src/include/storage/ipc.h index 6ebda479ced..bb7ae4d33b3 100644 --- a/src/include/storage/ipc.h +++ b/src/include/storage/ipc.h @@ -64,6 +64,7 @@ typedef void (*shmem_startup_hook_type) (void); /* ipc.c */ extern PGDLLIMPORT bool proc_exit_inprogress; extern PGDLLIMPORT bool shmem_exit_inprogress; +extern PGDLLIMPORT volatile bool pending_pm_shmem_resize; pg_noreturn extern void proc_exit(int code); extern void shmem_exit(int code); @@ -83,5 +84,7 @@ extern void CreateSharedMemoryAndSemaphores(void); extern void AttachSharedMemoryStructs(void); #endif extern void InitializeShmemGUCs(void); +extern void CoordinateShmemResize(void); +extern bool AnonymousShmemResize(void); #endif /* IPC_H */ diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h index a9681738146..558da6fdd55 100644 --- a/src/include/storage/lwlocklist.h +++ b/src/include/storage/lwlocklist.h @@ -84,3 +84,4 @@ PG_LWLOCK(50, DSMRegistry) PG_LWLOCK(51, InjectionPoint) PG_LWLOCK(52, SerialControl) PG_LWLOCK(53, AioWorkerSubmissionQueue) +PG_LWLOCK(54, ShmemResize) diff --git a/src/include/storage/pg_shmem.h b/src/include/storage/pg_shmem.h index c5009a1cd73..2e47b222cbb 100644 --- a/src/include/storage/pg_shmem.h +++ b/src/include/storage/pg_shmem.h @@ -24,6 +24,7 @@ #ifndef PG_SHMEM_H #define PG_SHMEM_H +#include "storage/barrier.h" #include "storage/dsm_impl.h" #include "storage/spin.h" @@ -56,6 +57,25 @@ typedef struct ShmemSegment extern PGDLLIMPORT ShmemSegment Segments[ANON_MAPPINGS]; +/* + * ShmemControl is shared between backends and helps to coordinate shared + * memory resize. + */ +typedef struct +{ + pg_atomic_uint32 NSharedBuffers; + Barrier Barrier; + pg_atomic_uint64 Generation; + bool Resizable; +} ShmemControl; + +extern PGDLLIMPORT ShmemControl *ShmemCtrl; + +/* The phases for shared memory resizing, used by for ProcSignal barrier. */ +#define SHMEM_RESIZE_REQUESTED 0 +#define SHMEM_RESIZE_START 1 +#define SHMEM_RESIZE_DONE 2 + /* GUC variables */ extern PGDLLIMPORT int shared_memory_type; extern PGDLLIMPORT int huge_pages; @@ -107,6 +127,12 @@ extern void PGSharedMemoryDetach(void); extern void GetHugePageSize(Size *hugepagesize, int *mmap_flags); void *ReserveAnonymousMemory(Size reserve_size); +bool ProcessBarrierShmemResize(Barrier *barrier); +void assign_shared_buffers(int newval, void *extra, bool *pending); +void AdjustShmemSize(void); +extern void WaitOnShmemBarrier(void); +extern void ShmemControlInit(void); + /* * To be able to dynamically resize largest parts of the data stored in shared * memory, we split it into multiple shared memory mappings segments. Each diff --git a/src/include/storage/pmsignal.h b/src/include/storage/pmsignal.h index 67fa9ac06e1..27bc6a81191 100644 --- a/src/include/storage/pmsignal.h +++ b/src/include/storage/pmsignal.h @@ -42,6 +42,7 @@ typedef enum PMSIGNAL_START_WALRECEIVER, /* start a walreceiver */ PMSIGNAL_ADVANCE_STATE_MACHINE, /* advance postmaster's state machine */ PMSIGNAL_XLOG_IS_SHUTDOWN, /* ShutdownXLOG() completed */ + PMSIGNAL_SHMEM_RESIZE, /* resize shared memory */ } PMSignalReason; #define NUM_PMSIGNALS (PMSIGNAL_XLOG_IS_SHUTDOWN+1) diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h index defd8b66a19..522b8de1e02 100644 --- a/src/include/storage/procsignal.h +++ b/src/include/storage/procsignal.h @@ -54,6 +54,7 @@ typedef enum typedef enum { PROCSIGNAL_BARRIER_SMGRRELEASE, /* ask smgr to close files */ + PROCSIGNAL_BARRIER_SHMEM_RESIZE, /* ask backends to resize shared memory */ } ProcSignalBarrierType; /* diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index 1a30437ad96..6755b302858 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -2738,6 +2738,7 @@ ShellTypeInfo ShippableCacheEntry ShippableCacheKey ShmemIndexEnt +ShmemControl ShutdownForeignScan_function ShutdownInformation ShutdownMode -- 2.45.1