Assign a child slot to every postmaster child process

Previously, only backends, autovacuum workers, and background workers
had an entry in the PMChildFlags array. With this commit, all
postmaster child processes, including all the aux processes, have an
entry. Dead-end backends still don't get an entry, though, and other
processes that don't touch shared memory will never mark their
PMChildFlags entry as active.

We now maintain separate freelists for different kinds of child
processes. That ensures that there are always slots available for
autovacuum and background workers. Previously, pre-authentication
backends could prevent autovacuum or background workers from starting
up, by using up all the slots.

The code to manage the slots in the postmaster process is in a new
pmchild.c source file. Because postmaster.c is just so large.
Assigning pmsignal slot numbers is now pmchild.c's responsibility.
This replaces the PMChildInUse array in pmsignal.c.

Some of the comments in postmaster.c still talked about the "stats
process", but that was removed in commit 5891c7a8ed. Fix those while
we're at it.

Reviewed-by: Andres Freund <andres@anarazel.de>
Discussion: https://www.postgresql.org/message-id/a102f15f-eac4-4ff2-af02-f9ff209ec66f@iki.fi
This commit is contained in:
Heikki Linnakangas 2024-11-14 16:12:28 +02:00
parent bb861414fe
commit a78af04270
15 changed files with 786 additions and 595 deletions

View File

@ -31,6 +31,7 @@
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "pg_getopt.h"
#include "postmaster/postmaster.h"
#include "storage/bufpage.h"
#include "storage/ipc.h"
#include "storage/proc.h"
@ -309,6 +310,13 @@ BootstrapModeMain(int argc, char *argv[], bool check_only)
InitializeMaxBackends();
/*
* Even though bootstrapping runs in single-process mode, initialize
* postmaster child slots array so that --check can detect running out of
* shared memory or other resources if max_connections is set too high.
*/
InitPostmasterChildSlots();
InitializeFastPathLocks();
CreateSharedMemoryAndSemaphores();

View File

@ -22,6 +22,7 @@ OBJS = \
interrupt.o \
launch_backend.o \
pgarch.o \
pmchild.o \
postmaster.o \
startup.o \
syslogger.o \

View File

@ -118,6 +118,7 @@ typedef struct
bool query_id_enabled;
int max_safe_fds;
int MaxBackends;
int num_pmchild_slots;
#ifdef WIN32
HANDLE PostmasterHandle;
HANDLE initial_signal_pipe;
@ -735,6 +736,7 @@ save_backend_variables(BackendParameters *param, ClientSocket *client_sock,
param->max_safe_fds = max_safe_fds;
param->MaxBackends = MaxBackends;
param->num_pmchild_slots = num_pmchild_slots;
#ifdef WIN32
param->PostmasterHandle = PostmasterHandle;
@ -994,6 +996,7 @@ restore_backend_variables(BackendParameters *param)
max_safe_fds = param->max_safe_fds;
MaxBackends = param->MaxBackends;
num_pmchild_slots = param->num_pmchild_slots;
#ifdef WIN32
PostmasterHandle = param->PostmasterHandle;

View File

@ -10,6 +10,7 @@ backend_sources += files(
'interrupt.c',
'launch_backend.c',
'pgarch.c',
'pmchild.c',
'postmaster.c',
'startup.c',
'syslogger.c',

View File

@ -0,0 +1,285 @@
/*-------------------------------------------------------------------------
*
* pmchild.c
* Functions for keeping track of postmaster child processes.
*
* Postmaster keeps track of all child processes so that when a process exits,
* it knows what kind of a process it was and can clean up accordingly. Every
* child process is allocated a PMChild struct from a fixed pool of structs.
* The size of the pool is determined by various settings that configure how
* many worker processes and backend connections are allowed, i.e.
* autovacuum_max_workers, max_worker_processes, max_wal_senders, and
* max_connections.
*
* Dead-end backends are handled slightly differently. There is no limit
* on the number of dead-end backends, and they do not need unique IDs, so
* their PMChild structs are allocated dynamically, not from a pool.
*
* The structures and functions in this file are private to the postmaster
* process. But note that there is an array in shared memory, managed by
* pmsignal.c, that mirrors this.
*
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* src/backend/postmaster/pmchild.c
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
#include "replication/walsender.h"
#include "storage/pmsignal.h"
#include "storage/proc.h"
/*
* Freelists for different kinds of child processes. We maintain separate
* pools for each, so that for example launching a lot of regular backends
* cannot prevent autovacuum or an aux process from launching.
*/
typedef struct PMChildPool
{
int size; /* number of PMChild slots reserved for this
* kind of processes */
int first_slotno; /* first slot belonging to this pool */
dlist_head freelist; /* currently unused PMChild entries */
} PMChildPool;
static PMChildPool pmchild_pools[BACKEND_NUM_TYPES];
NON_EXEC_STATIC int num_pmchild_slots = 0;
/*
* List of active child processes. This includes dead-end children.
*/
dlist_head ActiveChildList;
/*
* MaxLivePostmasterChildren
*
* This reports the number of postmaster child processes that can be active.
* It includes all children except for dead-end children. This allows the
* array in shared memory (PMChildFlags) to have a fixed maximum size.
*/
int
MaxLivePostmasterChildren(void)
{
if (num_pmchild_slots == 0)
elog(ERROR, "PM child array not initialized yet");
return num_pmchild_slots;
}
/*
* Initialize at postmaster startup
*
* Note: This is not called on crash restart. We rely on PMChild entries to
* remain valid through the restart process. This is important because the
* syslogger survives through the crash restart process, so we must not
* invalidate its PMChild slot.
*/
void
InitPostmasterChildSlots(void)
{
int slotno;
PMChild *slots;
/*
* We allow more connections here than we can have backends because some
* might still be authenticating; they might fail auth, or some existing
* backend might exit before the auth cycle is completed. The exact
* MaxConnections limit is enforced when a new backend tries to join the
* PGPROC array.
*
* WAL senders start out as regular backends, so they share the same pool.
*/
pmchild_pools[B_BACKEND].size = 2 * (MaxConnections + max_wal_senders);
pmchild_pools[B_AUTOVAC_WORKER].size = autovacuum_max_workers;
pmchild_pools[B_BG_WORKER].size = max_worker_processes;
/*
* There can be only one of each of these running at a time. They each
* get their own pool of just one entry.
*/
pmchild_pools[B_AUTOVAC_LAUNCHER].size = 1;
pmchild_pools[B_SLOTSYNC_WORKER].size = 1;
pmchild_pools[B_ARCHIVER].size = 1;
pmchild_pools[B_BG_WRITER].size = 1;
pmchild_pools[B_CHECKPOINTER].size = 1;
pmchild_pools[B_STARTUP].size = 1;
pmchild_pools[B_WAL_RECEIVER].size = 1;
pmchild_pools[B_WAL_SUMMARIZER].size = 1;
pmchild_pools[B_WAL_WRITER].size = 1;
pmchild_pools[B_LOGGER].size = 1;
/* The rest of the pmchild_pools are left at zero size */
/* Count the total number of slots */
num_pmchild_slots = 0;
for (int i = 0; i < BACKEND_NUM_TYPES; i++)
num_pmchild_slots += pmchild_pools[i].size;
/* Initialize them */
slots = palloc(num_pmchild_slots * sizeof(PMChild));
slotno = 0;
for (int btype = 0; btype < BACKEND_NUM_TYPES; btype++)
{
pmchild_pools[btype].first_slotno = slotno + 1;
dlist_init(&pmchild_pools[btype].freelist);
for (int j = 0; j < pmchild_pools[btype].size; j++)
{
slots[slotno].pid = 0;
slots[slotno].child_slot = slotno + 1;
slots[slotno].bkend_type = B_INVALID;
slots[slotno].rw = NULL;
slots[slotno].bgworker_notify = false;
dlist_push_tail(&pmchild_pools[btype].freelist, &slots[slotno].elem);
slotno++;
}
}
Assert(slotno == num_pmchild_slots);
/* Initialize other structures */
dlist_init(&ActiveChildList);
}
/*
* Allocate a PMChild entry for a postmaster child process of given type.
*
* The entry is taken from the right pool for the type.
*
* pmchild->child_slot in the returned struct is unique among all active child
* processes.
*/
PMChild *
AssignPostmasterChildSlot(BackendType btype)
{
dlist_head *freelist;
PMChild *pmchild;
if (pmchild_pools[btype].size == 0)
elog(ERROR, "cannot allocate a PMChild slot for backend type %d", btype);
freelist = &pmchild_pools[btype].freelist;
if (dlist_is_empty(freelist))
return NULL;
pmchild = dlist_container(PMChild, elem, dlist_pop_head_node(freelist));
pmchild->pid = 0;
pmchild->bkend_type = btype;
pmchild->rw = NULL;
pmchild->bgworker_notify = true;
/*
* pmchild->child_slot for each entry was initialized when the array of
* slots was allocated. Sanity check it.
*/
if (!(pmchild->child_slot >= pmchild_pools[btype].first_slotno &&
pmchild->child_slot < pmchild_pools[btype].first_slotno + pmchild_pools[btype].size))
{
elog(ERROR, "pmchild freelist for backend type %d is corrupt",
pmchild->bkend_type);
}
dlist_push_head(&ActiveChildList, &pmchild->elem);
/* Update the status in the shared memory array */
MarkPostmasterChildSlotAssigned(pmchild->child_slot);
elog(DEBUG2, "assigned pm child slot %d for %s",
pmchild->child_slot, PostmasterChildName(btype));
return pmchild;
}
/*
* Allocate a PMChild struct for a dead-end backend. Dead-end children are
* not assigned a child_slot number. The struct is palloc'd; returns NULL if
* out of memory.
*/
PMChild *
AllocDeadEndChild(void)
{
PMChild *pmchild;
elog(DEBUG2, "allocating dead-end child");
pmchild = (PMChild *) palloc_extended(sizeof(PMChild), MCXT_ALLOC_NO_OOM);
if (pmchild)
{
pmchild->pid = 0;
pmchild->child_slot = 0;
pmchild->bkend_type = B_DEAD_END_BACKEND;
pmchild->rw = NULL;
pmchild->bgworker_notify = false;
dlist_push_head(&ActiveChildList, &pmchild->elem);
}
return pmchild;
}
/*
* Release a PMChild slot, after the child process has exited.
*
* Returns true if the child detached cleanly from shared memory, false
* otherwise (see MarkPostmasterChildSlotUnassigned).
*/
bool
ReleasePostmasterChildSlot(PMChild *pmchild)
{
dlist_delete(&pmchild->elem);
if (pmchild->bkend_type == B_DEAD_END_BACKEND)
{
elog(DEBUG2, "releasing dead-end backend");
pfree(pmchild);
return true;
}
else
{
PMChildPool *pool;
elog(DEBUG2, "releasing pm child slot %d", pmchild->child_slot);
/* WAL senders start out as regular backends, and share the pool */
if (pmchild->bkend_type == B_WAL_SENDER)
pool = &pmchild_pools[B_BACKEND];
else
pool = &pmchild_pools[pmchild->bkend_type];
/* sanity check that we return the entry to the right pool */
if (!(pmchild->child_slot >= pool->first_slotno &&
pmchild->child_slot < pool->first_slotno + pool->size))
{
elog(ERROR, "pmchild freelist for backend type %d is corrupt",
pmchild->bkend_type);
}
dlist_push_head(&pool->freelist, &pmchild->elem);
return MarkPostmasterChildSlotUnassigned(pmchild->child_slot);
}
}
/*
* Find the PMChild entry of a running child process by PID.
*/
PMChild *
FindPostmasterChildByPid(int pid)
{
dlist_iter iter;
dlist_foreach(iter, &ActiveChildList)
{
PMChild *bp = dlist_container(PMChild, elem, iter.cur);
if (bp->pid == pid)
return bp;
}
return NULL;
}

File diff suppressed because it is too large Load Diff

View File

@ -590,7 +590,7 @@ SysLoggerMain(char *startup_data, size_t startup_data_len)
* Postmaster subroutine to start a syslogger subprocess.
*/
int
SysLogger_Start(void)
SysLogger_Start(int child_slot)
{
pid_t sysloggerPid;
char *filename;
@ -598,8 +598,7 @@ SysLogger_Start(void)
SysloggerStartupData startup_data;
#endif /* EXEC_BACKEND */
if (!Logging_collector)
return 0;
Assert(Logging_collector);
/*
* If first time through, create the pipe which will receive stderr
@ -695,6 +694,7 @@ SysLogger_Start(void)
pfree(filename);
}
MyPMChildSlot = child_slot;
#ifdef EXEC_BACKEND
startup_data.syslogFile = syslogger_fdget(syslogFile);
startup_data.csvlogFile = syslogger_fdget(csvlogFile);

View File

@ -47,11 +47,11 @@
* exited without performing proper shutdown. The per-child-process flags
* have three possible states: UNUSED, ASSIGNED, ACTIVE. An UNUSED slot is
* available for assignment. An ASSIGNED slot is associated with a postmaster
* child process, but either the process has not touched shared memory yet,
* or it has successfully cleaned up after itself. A ACTIVE slot means the
* process is actively using shared memory. The slots are assigned to
* child processes at random, and postmaster.c is responsible for tracking
* which one goes with which PID.
* child process, but either the process has not touched shared memory yet, or
* it has successfully cleaned up after itself. An ACTIVE slot means the
* process is actively using shared memory. The slots are assigned to child
* processes by postmaster, and pmchild.c is responsible for tracking which
* one goes with which PID.
*
* Actually there is a fourth state, WALSENDER. This is just like ACTIVE,
* but carries the extra information that the child is a WAL sender.
@ -84,13 +84,11 @@ struct PMSignalData
NON_EXEC_STATIC volatile PMSignalData *PMSignalState = NULL;
/*
* These static variables are valid only in the postmaster. We keep a
* duplicative private array so that we can trust its state even if some
* failing child has clobbered the PMSignalData struct in shared memory.
* Local copy of PMSignalState->num_child_flags, only valid in the
* postmaster. Postmaster keeps a local copy so that it doesn't need to
* trust the value in shared memory.
*/
static int num_child_inuse; /* # of entries in PMChildInUse[] */
static int next_child_inuse; /* next slot to try to assign */
static bool *PMChildInUse; /* true if i'th flag slot is assigned */
static int num_child_flags;
/*
* Signal handler to be notified if postmaster dies.
@ -155,25 +153,8 @@ PMSignalShmemInit(void)
{
/* initialize all flags to zeroes */
MemSet(unvolatize(PMSignalData *, PMSignalState), 0, PMSignalShmemSize());
num_child_inuse = MaxLivePostmasterChildren();
PMSignalState->num_child_flags = num_child_inuse;
/*
* Also allocate postmaster's private PMChildInUse[] array. We
* might've already done that in a previous shared-memory creation
* cycle, in which case free the old array to avoid a leak. (Do it
* like this to support the possibility that MaxLivePostmasterChildren
* changed.) In a standalone backend, we do not need this.
*/
if (PostmasterContext != NULL)
{
if (PMChildInUse)
pfree(PMChildInUse);
PMChildInUse = (bool *)
MemoryContextAllocZero(PostmasterContext,
num_child_inuse * sizeof(bool));
}
next_child_inuse = 0;
num_child_flags = MaxLivePostmasterChildren();
PMSignalState->num_child_flags = num_child_flags;
}
}
@ -239,56 +220,37 @@ GetQuitSignalReason(void)
/*
* AssignPostmasterChildSlot - select an unused slot for a new postmaster
* child process, and set its state to ASSIGNED. Returns a slot number
* (one to N).
* MarkPostmasterChildSlotAssigned - mark the given slot as ASSIGNED for a
* new postmaster child process.
*
* Only the postmaster is allowed to execute this routine, so we need no
* special locking.
*/
int
AssignPostmasterChildSlot(void)
void
MarkPostmasterChildSlotAssigned(int slot)
{
int slot = next_child_inuse;
int n;
Assert(slot > 0 && slot <= num_child_flags);
slot--;
/*
* Scan for a free slot. Notice that we trust nothing about the contents
* of PMSignalState, but use only postmaster-local data for this decision.
* We track the last slot assigned so as not to waste time repeatedly
* rescanning low-numbered slots.
*/
for (n = num_child_inuse; n > 0; n--)
{
if (--slot < 0)
slot = num_child_inuse - 1;
if (!PMChildInUse[slot])
{
PMChildInUse[slot] = true;
PMSignalState->PMChildFlags[slot] = PM_CHILD_ASSIGNED;
next_child_inuse = slot;
return slot + 1;
}
}
if (PMSignalState->PMChildFlags[slot] != PM_CHILD_UNUSED)
elog(FATAL, "postmaster child slot is already in use");
/* Out of slots ... should never happen, else postmaster.c messed up */
elog(FATAL, "no free slots in PMChildFlags array");
return 0; /* keep compiler quiet */
PMSignalState->PMChildFlags[slot] = PM_CHILD_ASSIGNED;
}
/*
* ReleasePostmasterChildSlot - release a slot after death of a postmaster
* child process. This must be called in the postmaster process.
* MarkPostmasterChildSlotUnassigned - release a slot after death of a
* postmaster child process. This must be called in the postmaster process.
*
* Returns true if the slot had been in ASSIGNED state (the expected case),
* false otherwise (implying that the child failed to clean itself up).
*/
bool
ReleasePostmasterChildSlot(int slot)
MarkPostmasterChildSlotUnassigned(int slot)
{
bool result;
Assert(slot > 0 && slot <= num_child_inuse);
Assert(slot > 0 && slot <= num_child_flags);
slot--;
/*
@ -298,7 +260,6 @@ ReleasePostmasterChildSlot(int slot)
*/
result = (PMSignalState->PMChildFlags[slot] == PM_CHILD_ASSIGNED);
PMSignalState->PMChildFlags[slot] = PM_CHILD_UNUSED;
PMChildInUse[slot] = false;
return result;
}
@ -309,7 +270,7 @@ ReleasePostmasterChildSlot(int slot)
bool
IsPostmasterChildWalSender(int slot)
{
Assert(slot > 0 && slot <= num_child_inuse);
Assert(slot > 0 && slot <= num_child_flags);
slot--;
if (PMSignalState->PMChildFlags[slot] == PM_CHILD_WALSENDER)

View File

@ -353,14 +353,9 @@ InitProcess(void)
/*
* Before we start accessing the shared memory in a serious way, mark
* ourselves as an active postmaster child; this is so that the postmaster
* can detect it if we exit without cleaning up. (XXX autovac launcher
* currently doesn't participate in this; it probably should.)
*
* Slot sync worker also does not participate in it, see comments atop
* 'struct bkend' in postmaster.c.
* can detect it if we exit without cleaning up.
*/
if (IsUnderPostmaster && !AmAutoVacuumLauncherProcess() &&
!AmLogicalSlotSyncWorkerProcess())
if (IsUnderPostmaster)
RegisterPostmasterChildActive();
/* Decide which list should supply our PGPROC. */
@ -578,6 +573,9 @@ InitAuxiliaryProcess(void)
if (MyProc != NULL)
elog(ERROR, "you already exist");
if (IsUnderPostmaster)
RegisterPostmasterChildActive();
/*
* We use the ProcStructLock to protect assignment and releasing of
* AuxiliaryProcs entries.

View File

@ -4189,6 +4189,12 @@ PostgresSingleUserMain(int argc, char *argv[],
/* Initialize MaxBackends */
InitializeMaxBackends();
/*
* We don't need postmaster child slots in single-user mode, but
* initialize them anyway to avoid having special handling.
*/
InitPostmasterChildSlots();
/* Initialize size of fast-path lock cache. */
InitializeFastPathLocks();

View File

@ -27,7 +27,7 @@
* List of background workers, private to postmaster.
*
* All workers that are currently running will also have an entry in
* BackendList.
* ActiveChildList.
*/
typedef struct RegisteredBgWorker
{

View File

@ -13,8 +13,44 @@
#ifndef _POSTMASTER_H
#define _POSTMASTER_H
#include "lib/ilist.h"
#include "miscadmin.h"
/*
* A struct representing an active postmaster child process. This is used
* mainly to keep track of how many children we have and send them appropriate
* signals when necessary. All postmaster child processes are assigned a
* PMChild entry. That includes "normal" client sessions, but also autovacuum
* workers, walsenders, background workers, and aux processes. (Note that at
* the time of launch, walsenders are labeled B_BACKEND; we relabel them to
* B_WAL_SENDER upon noticing they've changed their PMChildFlags entry. Hence
* that check must be done before any operation that needs to distinguish
* walsenders from normal backends.)
*
* "dead-end" children are also allocated a PMChild entry: these are children
* launched just for the purpose of sending a friendly rejection message to a
* would-be client. We must track them because they are attached to shared
* memory, but we know they will never become live backends.
*
* child_slot is an identifier that is unique across all running child
* processes. It is used as an index into the PMChildFlags array. dead-end
* children are not assigned a child_slot and have child_slot == 0 (valid
* child_slot ids start from 1).
*/
typedef struct
{
pid_t pid; /* process id of backend */
int child_slot; /* PMChildSlot for this backend, if any */
BackendType bkend_type; /* child process flavor, see above */
struct RegisteredBgWorker *rw; /* bgworker info, if this is a bgworker */
bool bgworker_notify; /* gets bgworker start/stop notifications */
dlist_node elem; /* list link in ActiveChildList */
} PMChild;
#ifdef EXEC_BACKEND
extern int num_pmchild_slots;
#endif
/* GUC options */
extern PGDLLIMPORT bool EnableSSL;
extern PGDLLIMPORT int SuperuserReservedConnections;
@ -80,6 +116,15 @@ const char *PostmasterChildName(BackendType child_type);
extern void SubPostmasterMain(int argc, char *argv[]) pg_attribute_noreturn();
#endif
/* defined in pmchild.c */
extern dlist_head ActiveChildList;
extern void InitPostmasterChildSlots(void);
extern PMChild *AssignPostmasterChildSlot(BackendType btype);
extern PMChild *AllocDeadEndChild(void);
extern bool ReleasePostmasterChildSlot(PMChild *pmchild);
extern PMChild *FindPostmasterChildByPid(int pid);
/*
* Note: MAX_BACKENDS is limited to 2^18-1 because that's the width reserved
* for buffer references in buf_internals.h. This limitation could be lifted

View File

@ -86,7 +86,7 @@ extern PGDLLIMPORT HANDLE syslogPipe[2];
#endif
extern int SysLogger_Start(void);
extern int SysLogger_Start(int child_slot);
extern void write_syslogger_file(const char *buffer, int count, int destination);

View File

@ -70,8 +70,8 @@ extern void SendPostmasterSignal(PMSignalReason reason);
extern bool CheckPostmasterSignal(PMSignalReason reason);
extern void SetQuitSignalReason(QuitSignalReason reason);
extern QuitSignalReason GetQuitSignalReason(void);
extern int AssignPostmasterChildSlot(void);
extern bool ReleasePostmasterChildSlot(int slot);
extern void MarkPostmasterChildSlotAssigned(int slot);
extern bool MarkPostmasterChildSlotUnassigned(int slot);
extern bool IsPostmasterChildWalSender(int slot);
extern void RegisterPostmasterChildActive(void);
extern void MarkPostmasterChildWalSender(void);

View File

@ -230,7 +230,6 @@ BTWriteState
BUF_MEM
BYTE
BY_HANDLE_FILE_INFORMATION
Backend
BackendParameters
BackendStartupData
BackendState
@ -1932,6 +1931,8 @@ PLyTransformToOb
PLyTupleToOb
PLyUnicode_FromStringAndSize_t
PLy_elog_impl_t
PMChild
PMChildPool
PMINIDUMP_CALLBACK_INFORMATION
PMINIDUMP_EXCEPTION_INFORMATION
PMINIDUMP_USER_STREAM_INFORMATION