2009-07-31 20:26:23 +00:00
|
|
|
/*-------------------------------------------------------------------------
|
|
|
|
*
|
|
|
|
* procsignal.c
|
2020-06-07 15:06:51 +02:00
|
|
|
* Routines for interprocess signaling
|
2009-07-31 20:26:23 +00:00
|
|
|
*
|
|
|
|
*
|
2022-01-07 19:04:57 -05:00
|
|
|
* Portions Copyright (c) 1996-2022, PostgreSQL Global Development Group
|
2009-07-31 20:26:23 +00:00
|
|
|
* Portions Copyright (c) 1994, Regents of the University of California
|
|
|
|
*
|
|
|
|
* IDENTIFICATION
|
2010-09-20 22:08:53 +02:00
|
|
|
* src/backend/storage/ipc/procsignal.c
|
2009-07-31 20:26:23 +00:00
|
|
|
*
|
|
|
|
*-------------------------------------------------------------------------
|
|
|
|
*/
|
|
|
|
#include "postgres.h"
|
|
|
|
|
|
|
|
#include <signal.h>
|
|
|
|
#include <unistd.h>
|
|
|
|
|
Create an infrastructure for parallel computation in PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Noah
Misch, Simon Riggs, Euler Taveira, and Jim Nasby.
2015-04-30 15:02:14 -04:00
|
|
|
#include "access/parallel.h"
|
2021-01-18 12:09:52 -05:00
|
|
|
#include "port/pg_bitutils.h"
|
2009-07-31 20:26:23 +00:00
|
|
|
#include "commands/async.h"
|
|
|
|
#include "miscadmin.h"
|
2019-12-19 14:56:20 -05:00
|
|
|
#include "pgstat.h"
|
Prevent possibility of panics during shutdown checkpoint.
When the checkpointer writes the shutdown checkpoint, it checks
afterwards whether any WAL has been written since it started and
throws a PANIC if so. At that point, only walsenders are still
active, so one might think this could not happen, but walsenders can
also generate WAL, for instance in BASE_BACKUP and logical decoding
related commands (e.g. via hint bits). So they can trigger this panic
if such a command is run while the shutdown checkpoint is being
written.
To fix this, divide the walsender shutdown into two phases. First,
checkpointer, itself triggered by postmaster, sends a
PROCSIG_WALSND_INIT_STOPPING signal to all walsenders. If the backend
is idle or runs an SQL query this causes the backend to shutdown, if
logical replication is in progress all existing WAL records are
processed followed by a shutdown. Otherwise this causes the walsender
to switch to the "stopping" state. In this state, the walsender will
reject any further replication commands. The checkpointer begins the
shutdown checkpoint once all walsenders are confirmed as
stopping. When the shutdown checkpoint finishes, the postmaster sends
us SIGUSR2. This instructs walsender to send any outstanding WAL,
including the shutdown checkpoint record, wait for it to be replicated
to the standby, and then exit.
Author: Andres Freund, based on an earlier patch by Michael Paquier
Reported-By: Fujii Masao, Andres Freund
Reviewed-By: Michael Paquier
Discussion: https://postgr.es/m/20170602002912.tqlwn4gymzlxpvs2@alap3.anarazel.de
Backpatch: 9.4, where logical decoding was introduced
2017-06-05 18:53:41 -07:00
|
|
|
#include "replication/walsender.h"
|
2021-03-01 16:26:22 +13:00
|
|
|
#include "storage/condition_variable.h"
|
2009-07-31 20:26:23 +00:00
|
|
|
#include "storage/ipc.h"
|
2019-11-12 08:30:16 +05:30
|
|
|
#include "storage/latch.h"
|
2013-08-28 14:08:13 -04:00
|
|
|
#include "storage/proc.h"
|
2012-09-20 11:03:04 -03:00
|
|
|
#include "storage/shmem.h"
|
2022-02-12 10:21:23 +13:00
|
|
|
#include "storage/smgr.h"
|
2009-07-31 20:26:23 +00:00
|
|
|
#include "storage/sinval.h"
|
2010-01-16 10:05:59 +00:00
|
|
|
#include "tcop/tcopprot.h"
|
Add function to log the memory contexts of specified backend process.
Commit 3e98c0bafb added pg_backend_memory_contexts view to display
the memory contexts of the backend process. However its target process
is limited to the backend that is accessing to the view. So this is
not so convenient when investigating the local memory bloat of other
backend process. To improve this situation, this commit adds
pg_log_backend_memory_contexts() function that requests to log
the memory contexts of the specified backend process.
This information can be also collected by calling
MemoryContextStats(TopMemoryContext) via a debugger. But
this technique cannot be used in some environments because no debugger
is available there. So, pg_log_backend_memory_contexts() allows us to
see the memory contexts of specified backend more easily.
Only superusers are allowed to request to log the memory contexts
because allowing any users to issue this request at an unbounded rate
would cause lots of log messages and which can lead to denial of service.
On receipt of the request, at the next CHECK_FOR_INTERRUPTS(),
the target backend logs its memory contexts at LOG_SERVER_ONLY level,
so that these memory contexts will appear in the server log but not
be sent to the client. It logs one message per memory context.
Because if it buffers all memory contexts into StringInfo to log them
as one message, which may require the buffer to be enlarged very much
and lead to OOM error since there can be a large number of memory
contexts in a backend.
When a backend process is consuming huge memory, logging all its
memory contexts might overrun available disk space. To prevent this,
now this patch limits the number of child contexts to log per parent
to 100. As with MemoryContextStats(), it supposes that practical cases
where the log gets long will typically be huge numbers of siblings
under the same parent context; while the additional debugging value
from seeing details about individual siblings beyond 100 will not be large.
There was another proposed patch to add the function to return
the memory contexts of specified backend as the result sets,
instead of logging them, in the discussion. However that patch is
not included in this commit because it had several issues to address.
Thanks to Tatsuhito Kasahara, Andres Freund, Tom Lane, Tomas Vondra,
Michael Paquier, Kyotaro Horiguchi and Zhihong Yu for the discussion.
Bump catalog version.
Author: Atsushi Torikoshi
Reviewed-by: Kyotaro Horiguchi, Zhihong Yu, Fujii Masao
Discussion: https://postgr.es/m/0271f440ac77f2a4180e0e56ebd944d1@oss.nttdata.com
2021-04-06 13:44:15 +09:00
|
|
|
#include "utils/memutils.h"
|
2009-07-31 20:26:23 +00:00
|
|
|
|
|
|
|
/*
|
2020-06-07 15:06:51 +02:00
|
|
|
* The SIGUSR1 signal is multiplexed to support signaling multiple event
|
2009-07-31 20:26:23 +00:00
|
|
|
* types. The specific reason is communicated via flags in shared memory.
|
|
|
|
* We keep a boolean flag for each possible "reason", so that different
|
|
|
|
* reasons can be signaled to a process concurrently. (However, if the same
|
|
|
|
* reason is signaled more than once nearly simultaneously, the process may
|
|
|
|
* observe it only once.)
|
|
|
|
*
|
|
|
|
* Each process that wants to receive signals registers its process ID
|
|
|
|
* in the ProcSignalSlots array. The array is indexed by backend ID to make
|
|
|
|
* slot allocation simple, and to avoid having to search the array when you
|
2020-06-07 15:06:51 +02:00
|
|
|
* know the backend ID of the process you're signaling. (We do support
|
|
|
|
* signaling without backend ID, but it's a bit less efficient.)
|
2009-07-31 20:26:23 +00:00
|
|
|
*
|
|
|
|
* The flags are actually declared as "volatile sig_atomic_t" for maximum
|
|
|
|
* portability. This should ensure that loads and stores of the flag
|
|
|
|
* values are atomic, allowing us to dispense with any explicit locking.
|
2019-12-19 14:56:20 -05:00
|
|
|
*
|
|
|
|
* pss_signalFlags are intended to be set in cases where we don't need to
|
|
|
|
* keep track of whether or not the target process has handled the signal,
|
|
|
|
* but sometimes we need confirmation, as when making a global state change
|
|
|
|
* that cannot be considered complete until all backends have taken notice
|
|
|
|
* of it. For such use cases, we set a bit in pss_barrierCheckMask and then
|
|
|
|
* increment the current "barrier generation"; when the new barrier generation
|
|
|
|
* (or greater) appears in the pss_barrierGeneration flag of every process,
|
|
|
|
* we know that the message has been received everywhere.
|
2009-07-31 20:26:23 +00:00
|
|
|
*/
|
|
|
|
typedef struct
|
|
|
|
{
|
2021-03-01 16:26:22 +13:00
|
|
|
volatile pid_t pss_pid;
|
|
|
|
volatile sig_atomic_t pss_signalFlags[NUM_PROCSIGNALS];
|
2019-12-19 14:56:20 -05:00
|
|
|
pg_atomic_uint64 pss_barrierGeneration;
|
|
|
|
pg_atomic_uint32 pss_barrierCheckMask;
|
2021-03-01 16:26:22 +13:00
|
|
|
ConditionVariable pss_barrierCV;
|
2009-07-31 20:26:23 +00:00
|
|
|
} ProcSignalSlot;
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/*
|
|
|
|
* Information that is global to the entire ProcSignal system can be stored
|
|
|
|
* here.
|
|
|
|
*
|
|
|
|
* psh_barrierGeneration is the highest barrier generation in existence.
|
|
|
|
*/
|
|
|
|
typedef struct
|
|
|
|
{
|
|
|
|
pg_atomic_uint64 psh_barrierGeneration;
|
|
|
|
ProcSignalSlot psh_slot[FLEXIBLE_ARRAY_MEMBER];
|
|
|
|
} ProcSignalHeader;
|
|
|
|
|
2022-04-12 14:45:23 -04:00
|
|
|
/*
|
|
|
|
* We reserve a slot for each possible BackendId, plus one for each
|
|
|
|
* possible auxiliary process type. (This scheme assumes there is not
|
|
|
|
* more than one of any auxiliary process type at a time.)
|
|
|
|
*/
|
|
|
|
#define NumProcSignalSlots (MaxBackends + NUM_AUXPROCTYPES)
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/* Check whether the relevant type bit is set in the flags. */
|
|
|
|
#define BARRIER_SHOULD_CHECK(flags, type) \
|
|
|
|
(((flags) & (((uint32) 1) << (uint32) (type))) != 0)
|
|
|
|
|
2021-01-18 12:09:52 -05:00
|
|
|
/* Clear the relevant type bit from the flags. */
|
|
|
|
#define BARRIER_CLEAR_BIT(flags, type) \
|
|
|
|
((flags) &= ~(((uint32) 1) << (uint32) (type)))
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
static ProcSignalHeader *ProcSignal = NULL;
|
2021-03-01 16:26:22 +13:00
|
|
|
static ProcSignalSlot *MyProcSignalSlot = NULL;
|
2009-07-31 20:26:23 +00:00
|
|
|
|
|
|
|
static bool CheckProcSignal(ProcSignalReason reason);
|
|
|
|
static void CleanupProcSignalState(int status, Datum arg);
|
2021-01-18 12:09:52 -05:00
|
|
|
static void ResetProcSignalBarrierBits(uint32 flags);
|
2009-07-31 20:26:23 +00:00
|
|
|
|
|
|
|
/*
|
2010-08-30 06:33:22 +00:00
|
|
|
* ProcSignalShmemSize
|
2021-11-09 12:56:34 +09:00
|
|
|
* Compute space needed for ProcSignal's shared memory
|
2009-07-31 20:26:23 +00:00
|
|
|
*/
|
|
|
|
Size
|
|
|
|
ProcSignalShmemSize(void)
|
|
|
|
{
|
2019-12-19 14:56:20 -05:00
|
|
|
Size size;
|
|
|
|
|
2022-04-12 14:45:23 -04:00
|
|
|
size = mul_size(NumProcSignalSlots, sizeof(ProcSignalSlot));
|
2019-12-19 14:56:20 -05:00
|
|
|
size = add_size(size, offsetof(ProcSignalHeader, psh_slot));
|
|
|
|
return size;
|
2009-07-31 20:26:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* ProcSignalShmemInit
|
2021-11-09 12:56:34 +09:00
|
|
|
* Allocate and initialize ProcSignal's shared memory
|
2009-07-31 20:26:23 +00:00
|
|
|
*/
|
|
|
|
void
|
|
|
|
ProcSignalShmemInit(void)
|
|
|
|
{
|
|
|
|
Size size = ProcSignalShmemSize();
|
|
|
|
bool found;
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
ProcSignal = (ProcSignalHeader *)
|
|
|
|
ShmemInitStruct("ProcSignal", size, &found);
|
2009-07-31 20:26:23 +00:00
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/* If we're first, initialize. */
|
2009-07-31 20:26:23 +00:00
|
|
|
if (!found)
|
2019-12-19 14:56:20 -05:00
|
|
|
{
|
|
|
|
int i;
|
|
|
|
|
|
|
|
pg_atomic_init_u64(&ProcSignal->psh_barrierGeneration, 0);
|
|
|
|
|
2022-04-12 14:45:23 -04:00
|
|
|
for (i = 0; i < NumProcSignalSlots; ++i)
|
2019-12-19 14:56:20 -05:00
|
|
|
{
|
|
|
|
ProcSignalSlot *slot = &ProcSignal->psh_slot[i];
|
|
|
|
|
|
|
|
slot->pss_pid = 0;
|
|
|
|
MemSet(slot->pss_signalFlags, 0, sizeof(slot->pss_signalFlags));
|
|
|
|
pg_atomic_init_u64(&slot->pss_barrierGeneration, PG_UINT64_MAX);
|
|
|
|
pg_atomic_init_u32(&slot->pss_barrierCheckMask, 0);
|
2021-03-01 16:26:22 +13:00
|
|
|
ConditionVariableInit(&slot->pss_barrierCV);
|
2019-12-19 14:56:20 -05:00
|
|
|
}
|
|
|
|
}
|
2009-07-31 20:26:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* ProcSignalInit
|
2021-11-09 12:56:34 +09:00
|
|
|
* Register the current process in the ProcSignal array
|
2009-07-31 20:26:23 +00:00
|
|
|
*
|
|
|
|
* The passed index should be my BackendId if the process has one,
|
|
|
|
* or MaxBackends + aux process type if not.
|
|
|
|
*/
|
|
|
|
void
|
|
|
|
ProcSignalInit(int pss_idx)
|
|
|
|
{
|
2021-03-01 16:26:22 +13:00
|
|
|
ProcSignalSlot *slot;
|
2019-12-19 14:56:20 -05:00
|
|
|
uint64 barrier_generation;
|
2009-07-31 20:26:23 +00:00
|
|
|
|
2022-04-12 14:45:23 -04:00
|
|
|
Assert(pss_idx >= 1 && pss_idx <= NumProcSignalSlots);
|
2009-07-31 20:26:23 +00:00
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
slot = &ProcSignal->psh_slot[pss_idx - 1];
|
2009-07-31 20:26:23 +00:00
|
|
|
|
|
|
|
/* sanity check */
|
|
|
|
if (slot->pss_pid != 0)
|
|
|
|
elog(LOG, "process %d taking over ProcSignal slot %d, but it's not empty",
|
|
|
|
MyProcPid, pss_idx);
|
|
|
|
|
|
|
|
/* Clear out any leftover signal reasons */
|
|
|
|
MemSet(slot->pss_signalFlags, 0, NUM_PROCSIGNALS * sizeof(sig_atomic_t));
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/*
|
|
|
|
* Initialize barrier state. Since we're a brand-new process, there
|
|
|
|
* shouldn't be any leftover backend-private state that needs to be
|
|
|
|
* updated. Therefore, we can broadcast the latest barrier generation and
|
|
|
|
* disregard any previously-set check bits.
|
|
|
|
*
|
|
|
|
* NB: This only works if this initialization happens early enough in the
|
|
|
|
* startup sequence that we haven't yet cached any state that might need
|
|
|
|
* to be invalidated. That's also why we have a memory barrier here, to be
|
|
|
|
* sure that any later reads of memory happen strictly after this.
|
|
|
|
*/
|
|
|
|
pg_atomic_write_u32(&slot->pss_barrierCheckMask, 0);
|
|
|
|
barrier_generation =
|
|
|
|
pg_atomic_read_u64(&ProcSignal->psh_barrierGeneration);
|
|
|
|
pg_atomic_write_u64(&slot->pss_barrierGeneration, barrier_generation);
|
|
|
|
pg_memory_barrier();
|
|
|
|
|
2009-07-31 20:26:23 +00:00
|
|
|
/* Mark slot with my PID */
|
|
|
|
slot->pss_pid = MyProcPid;
|
|
|
|
|
|
|
|
/* Remember slot location for CheckProcSignal */
|
|
|
|
MyProcSignalSlot = slot;
|
|
|
|
|
|
|
|
/* Set up to release the slot on process exit */
|
|
|
|
on_shmem_exit(CleanupProcSignalState, Int32GetDatum(pss_idx));
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* CleanupProcSignalState
|
2019-12-19 14:56:20 -05:00
|
|
|
* Remove current process from ProcSignal mechanism
|
2009-07-31 20:26:23 +00:00
|
|
|
*
|
|
|
|
* This function is called via on_shmem_exit() during backend shutdown.
|
|
|
|
*/
|
|
|
|
static void
|
|
|
|
CleanupProcSignalState(int status, Datum arg)
|
|
|
|
{
|
|
|
|
int pss_idx = DatumGetInt32(arg);
|
2021-03-01 16:26:22 +13:00
|
|
|
ProcSignalSlot *slot;
|
2009-07-31 20:26:23 +00:00
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
slot = &ProcSignal->psh_slot[pss_idx - 1];
|
2009-07-31 20:26:23 +00:00
|
|
|
Assert(slot == MyProcSignalSlot);
|
|
|
|
|
2014-01-31 21:31:08 -05:00
|
|
|
/*
|
|
|
|
* Clear MyProcSignalSlot, so that a SIGUSR1 received after this point
|
|
|
|
* won't try to access it after it's no longer ours (and perhaps even
|
|
|
|
* after we've unmapped the shared memory segment).
|
|
|
|
*/
|
|
|
|
MyProcSignalSlot = NULL;
|
|
|
|
|
2009-07-31 20:26:23 +00:00
|
|
|
/* sanity check */
|
|
|
|
if (slot->pss_pid != MyProcPid)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
* don't ERROR here. We're exiting anyway, and don't want to get into
|
|
|
|
* infinite loop trying to exit
|
|
|
|
*/
|
|
|
|
elog(LOG, "process %d releasing ProcSignal slot %d, but it contains %d",
|
|
|
|
MyProcPid, pss_idx, (int) slot->pss_pid);
|
|
|
|
return; /* XXX better to zero the slot anyway? */
|
|
|
|
}
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/*
|
|
|
|
* Make this slot look like it's absorbed all possible barriers, so that
|
|
|
|
* no barrier waits block on it.
|
|
|
|
*/
|
|
|
|
pg_atomic_write_u64(&slot->pss_barrierGeneration, PG_UINT64_MAX);
|
2021-03-01 16:26:22 +13:00
|
|
|
ConditionVariableBroadcast(&slot->pss_barrierCV);
|
2019-12-19 14:56:20 -05:00
|
|
|
|
2009-07-31 20:26:23 +00:00
|
|
|
slot->pss_pid = 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* SendProcSignal
|
|
|
|
* Send a signal to a Postgres process
|
|
|
|
*
|
|
|
|
* Providing backendId is optional, but it will speed up the operation.
|
|
|
|
*
|
|
|
|
* On success (a signal was sent), zero is returned.
|
|
|
|
* On error, -1 is returned, and errno is set (typically to ESRCH or EPERM).
|
|
|
|
*
|
|
|
|
* Not to be confused with ProcSendSignal
|
|
|
|
*/
|
|
|
|
int
|
|
|
|
SendProcSignal(pid_t pid, ProcSignalReason reason, BackendId backendId)
|
|
|
|
{
|
|
|
|
volatile ProcSignalSlot *slot;
|
|
|
|
|
|
|
|
if (backendId != InvalidBackendId)
|
|
|
|
{
|
2019-12-19 14:56:20 -05:00
|
|
|
slot = &ProcSignal->psh_slot[backendId - 1];
|
2009-07-31 20:26:23 +00:00
|
|
|
|
|
|
|
/*
|
|
|
|
* Note: Since there's no locking, it's possible that the target
|
|
|
|
* process detaches from shared memory and exits right after this
|
|
|
|
* test, before we set the flag and send signal. And the signal slot
|
|
|
|
* might even be recycled by a new process, so it's remotely possible
|
|
|
|
* that we set a flag for a wrong process. That's OK, all the signals
|
|
|
|
* are such that no harm is done if they're mistakenly fired.
|
|
|
|
*/
|
|
|
|
if (slot->pss_pid == pid)
|
|
|
|
{
|
|
|
|
/* Atomically set the proper flag */
|
|
|
|
slot->pss_signalFlags[reason] = true;
|
|
|
|
/* Send signal */
|
|
|
|
return kill(pid, SIGUSR1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
* BackendId not provided, so search the array using pid. We search
|
|
|
|
* the array back to front so as to reduce search overhead. Passing
|
|
|
|
* InvalidBackendId means that the target is most likely an auxiliary
|
|
|
|
* process, which will have a slot near the end of the array.
|
|
|
|
*/
|
|
|
|
int i;
|
|
|
|
|
2022-04-12 14:45:23 -04:00
|
|
|
for (i = NumProcSignalSlots - 1; i >= 0; i--)
|
2009-07-31 20:26:23 +00:00
|
|
|
{
|
2019-12-19 14:56:20 -05:00
|
|
|
slot = &ProcSignal->psh_slot[i];
|
2009-07-31 20:26:23 +00:00
|
|
|
|
|
|
|
if (slot->pss_pid == pid)
|
|
|
|
{
|
|
|
|
/* the above note about race conditions applies here too */
|
|
|
|
|
|
|
|
/* Atomically set the proper flag */
|
|
|
|
slot->pss_signalFlags[reason] = true;
|
|
|
|
/* Send signal */
|
|
|
|
return kill(pid, SIGUSR1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
errno = ESRCH;
|
|
|
|
return -1;
|
|
|
|
}
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/*
|
|
|
|
* EmitProcSignalBarrier
|
|
|
|
* Send a signal to every Postgres process
|
|
|
|
*
|
|
|
|
* The return value of this function is the barrier "generation" created
|
|
|
|
* by this operation. This value can be passed to WaitForProcSignalBarrier
|
|
|
|
* to wait until it is known that every participant in the ProcSignal
|
|
|
|
* mechanism has absorbed the signal (or started afterwards).
|
|
|
|
*
|
|
|
|
* Note that it would be a bad idea to use this for anything that happens
|
|
|
|
* frequently, as interrupting every backend could cause a noticeable
|
|
|
|
* performance hit.
|
|
|
|
*
|
|
|
|
* Callers are entitled to assume that this function will not throw ERROR
|
|
|
|
* or FATAL.
|
|
|
|
*/
|
|
|
|
uint64
|
|
|
|
EmitProcSignalBarrier(ProcSignalBarrierType type)
|
|
|
|
{
|
2020-06-15 18:23:10 -07:00
|
|
|
uint32 flagbit = 1 << (uint32) type;
|
2019-12-19 14:56:20 -05:00
|
|
|
uint64 generation;
|
|
|
|
|
|
|
|
/*
|
|
|
|
* Set all the flags.
|
|
|
|
*
|
|
|
|
* Note that pg_atomic_fetch_or_u32 has full barrier semantics, so this is
|
|
|
|
* totally ordered with respect to anything the caller did before, and
|
|
|
|
* anything that we do afterwards. (This is also true of the later call to
|
|
|
|
* pg_atomic_add_fetch_u64.)
|
|
|
|
*/
|
2022-04-12 14:45:23 -04:00
|
|
|
for (int i = 0; i < NumProcSignalSlots; i++)
|
2019-12-19 14:56:20 -05:00
|
|
|
{
|
|
|
|
volatile ProcSignalSlot *slot = &ProcSignal->psh_slot[i];
|
|
|
|
|
|
|
|
pg_atomic_fetch_or_u32(&slot->pss_barrierCheckMask, flagbit);
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* Increment the generation counter.
|
|
|
|
*/
|
|
|
|
generation =
|
|
|
|
pg_atomic_add_fetch_u64(&ProcSignal->psh_barrierGeneration, 1);
|
|
|
|
|
|
|
|
/*
|
|
|
|
* Signal all the processes, so that they update their advertised barrier
|
|
|
|
* generation.
|
|
|
|
*
|
|
|
|
* Concurrency is not a problem here. Backends that have exited don't
|
|
|
|
* matter, and new backends that have joined since we entered this
|
|
|
|
* function must already have current state, since the caller is
|
|
|
|
* responsible for making sure that the relevant state is entirely visible
|
|
|
|
* before calling this function in the first place. We still have to wake
|
|
|
|
* them up - because we can't distinguish between such backends and older
|
|
|
|
* backends that need to update state - but they won't actually need to
|
|
|
|
* change any state.
|
|
|
|
*/
|
2022-04-12 14:45:23 -04:00
|
|
|
for (int i = NumProcSignalSlots - 1; i >= 0; i--)
|
2019-12-19 14:56:20 -05:00
|
|
|
{
|
|
|
|
volatile ProcSignalSlot *slot = &ProcSignal->psh_slot[i];
|
|
|
|
pid_t pid = slot->pss_pid;
|
|
|
|
|
|
|
|
if (pid != 0)
|
2020-06-15 18:23:10 -07:00
|
|
|
{
|
|
|
|
/* see SendProcSignal for details */
|
|
|
|
slot->pss_signalFlags[PROCSIG_BARRIER] = true;
|
2019-12-19 14:56:20 -05:00
|
|
|
kill(pid, SIGUSR1);
|
2020-06-15 18:23:10 -07:00
|
|
|
}
|
2019-12-19 14:56:20 -05:00
|
|
|
}
|
|
|
|
|
|
|
|
return generation;
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* WaitForProcSignalBarrier - wait until it is guaranteed that all changes
|
|
|
|
* requested by a specific call to EmitProcSignalBarrier() have taken effect.
|
|
|
|
*/
|
|
|
|
void
|
|
|
|
WaitForProcSignalBarrier(uint64 generation)
|
|
|
|
{
|
2020-06-15 18:23:10 -07:00
|
|
|
Assert(generation <= pg_atomic_read_u64(&ProcSignal->psh_barrierGeneration));
|
|
|
|
|
2022-04-12 14:45:23 -04:00
|
|
|
for (int i = NumProcSignalSlots - 1; i >= 0; i--)
|
2019-12-19 14:56:20 -05:00
|
|
|
{
|
2021-03-01 16:26:22 +13:00
|
|
|
ProcSignalSlot *slot = &ProcSignal->psh_slot[i];
|
2019-12-19 14:56:20 -05:00
|
|
|
uint64 oldval;
|
|
|
|
|
2021-01-18 12:09:52 -05:00
|
|
|
/*
|
|
|
|
* It's important that we check only pss_barrierGeneration here and
|
|
|
|
* not pss_barrierCheckMask. Bits in pss_barrierCheckMask get cleared
|
|
|
|
* before the barrier is actually absorbed, but pss_barrierGeneration
|
|
|
|
* is updated only afterward.
|
|
|
|
*/
|
2019-12-19 14:56:20 -05:00
|
|
|
oldval = pg_atomic_read_u64(&slot->pss_barrierGeneration);
|
|
|
|
while (oldval < generation)
|
|
|
|
{
|
2021-03-01 16:26:22 +13:00
|
|
|
ConditionVariableSleep(&slot->pss_barrierCV,
|
|
|
|
WAIT_EVENT_PROC_SIGNAL_BARRIER);
|
2019-12-19 14:56:20 -05:00
|
|
|
oldval = pg_atomic_read_u64(&slot->pss_barrierGeneration);
|
|
|
|
}
|
2021-03-01 16:26:22 +13:00
|
|
|
ConditionVariableCancelSleep();
|
2019-12-19 14:56:20 -05:00
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* The caller is probably calling this function because it wants to read
|
|
|
|
* the shared state or perform further writes to shared state once all
|
|
|
|
* backends are known to have absorbed the barrier. However, the read of
|
|
|
|
* pss_barrierGeneration was performed unlocked; insert a memory barrier
|
|
|
|
* to separate it from whatever follows.
|
|
|
|
*/
|
|
|
|
pg_memory_barrier();
|
|
|
|
}
|
|
|
|
|
2020-06-15 18:23:10 -07:00
|
|
|
/*
|
|
|
|
* Handle receipt of an interrupt indicating a global barrier event.
|
|
|
|
*
|
|
|
|
* All the actual work is deferred to ProcessProcSignalBarrier(), because we
|
|
|
|
* cannot safely access the barrier generation inside the signal handler as
|
|
|
|
* 64bit atomics might use spinlock based emulation, even for reads. As this
|
|
|
|
* routine only gets called when PROCSIG_BARRIER is sent that won't cause a
|
2020-09-06 19:26:55 +02:00
|
|
|
* lot of unnecessary work.
|
2020-06-15 18:23:10 -07:00
|
|
|
*/
|
|
|
|
static void
|
|
|
|
HandleProcSignalBarrierInterrupt(void)
|
|
|
|
{
|
|
|
|
InterruptPending = true;
|
|
|
|
ProcSignalBarrierPending = true;
|
|
|
|
/* latch will be set by procsignal_sigusr1_handler */
|
|
|
|
}
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/*
|
|
|
|
* Perform global barrier related interrupt checking.
|
|
|
|
*
|
2020-06-07 15:06:51 +02:00
|
|
|
* Any backend that participates in ProcSignal signaling must arrange to
|
2019-12-19 14:56:20 -05:00
|
|
|
* call this function periodically. It is called from CHECK_FOR_INTERRUPTS(),
|
|
|
|
* which is enough for normal backends, but not necessarily for all types of
|
|
|
|
* background processes.
|
|
|
|
*/
|
|
|
|
void
|
|
|
|
ProcessProcSignalBarrier(void)
|
|
|
|
{
|
2020-06-15 18:23:10 -07:00
|
|
|
uint64 local_gen;
|
|
|
|
uint64 shared_gen;
|
2021-01-18 12:09:52 -05:00
|
|
|
volatile uint32 flags;
|
2019-12-19 14:56:20 -05:00
|
|
|
|
2020-06-15 18:23:10 -07:00
|
|
|
Assert(MyProcSignalSlot);
|
|
|
|
|
2019-12-19 14:56:20 -05:00
|
|
|
/* Exit quickly if there's no work to do. */
|
|
|
|
if (!ProcSignalBarrierPending)
|
|
|
|
return;
|
|
|
|
ProcSignalBarrierPending = false;
|
|
|
|
|
|
|
|
/*
|
2020-06-15 18:23:10 -07:00
|
|
|
* It's not unlikely to process multiple barriers at once, before the
|
|
|
|
* signals for all the barriers have arrived. To avoid unnecessary work in
|
|
|
|
* response to subsequent signals, exit early if we already have processed
|
|
|
|
* all of them.
|
|
|
|
*/
|
|
|
|
local_gen = pg_atomic_read_u64(&MyProcSignalSlot->pss_barrierGeneration);
|
|
|
|
shared_gen = pg_atomic_read_u64(&ProcSignal->psh_barrierGeneration);
|
|
|
|
|
|
|
|
Assert(local_gen <= shared_gen);
|
|
|
|
|
|
|
|
if (local_gen == shared_gen)
|
|
|
|
return;
|
|
|
|
|
|
|
|
/*
|
|
|
|
* Get and clear the flags that are set for this backend. Note that
|
|
|
|
* pg_atomic_exchange_u32 is a full barrier, so we're guaranteed that the
|
|
|
|
* read of the barrier generation above happens before we atomically
|
|
|
|
* extract the flags, and that any subsequent state changes happen
|
|
|
|
* afterward.
|
2021-01-18 12:09:52 -05:00
|
|
|
*
|
|
|
|
* NB: In order to avoid race conditions, we must zero
|
|
|
|
* pss_barrierCheckMask first and only afterwards try to do barrier
|
|
|
|
* processing. If we did it in the other order, someone could send us
|
|
|
|
* another barrier of some type right after we called the
|
|
|
|
* barrier-processing function but before we cleared the bit. We would
|
|
|
|
* have no way of knowing that the bit needs to stay set in that case, so
|
|
|
|
* the need to call the barrier-processing function again would just get
|
|
|
|
* forgotten. So instead, we tentatively clear all the bits and then put
|
|
|
|
* back any for which we don't manage to successfully absorb the barrier.
|
2019-12-19 14:56:20 -05:00
|
|
|
*/
|
|
|
|
flags = pg_atomic_exchange_u32(&MyProcSignalSlot->pss_barrierCheckMask, 0);
|
|
|
|
|
|
|
|
/*
|
2021-01-18 12:09:52 -05:00
|
|
|
* If there are no flags set, then we can skip doing any real work.
|
|
|
|
* Otherwise, establish a PG_TRY block, so that we don't lose track of
|
|
|
|
* which types of barrier processing are needed if an ERROR occurs.
|
2019-12-19 14:56:20 -05:00
|
|
|
*/
|
2021-01-18 12:09:52 -05:00
|
|
|
if (flags != 0)
|
|
|
|
{
|
|
|
|
bool success = true;
|
|
|
|
|
|
|
|
PG_TRY();
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
* Process each type of barrier. The barrier-processing functions
|
|
|
|
* should normally return true, but may return false if the
|
|
|
|
* barrier can't be absorbed at the current time. This should be
|
|
|
|
* rare, because it's pretty expensive. Every single
|
|
|
|
* CHECK_FOR_INTERRUPTS() will return here until we manage to
|
|
|
|
* absorb the barrier, and that cost will add up in a hurry.
|
|
|
|
*
|
|
|
|
* NB: It ought to be OK to call the barrier-processing functions
|
|
|
|
* unconditionally, but it's more efficient to call only the ones
|
|
|
|
* that might need us to do something based on the flags.
|
|
|
|
*/
|
|
|
|
while (flags != 0)
|
|
|
|
{
|
|
|
|
ProcSignalBarrierType type;
|
|
|
|
bool processed = true;
|
|
|
|
|
|
|
|
type = (ProcSignalBarrierType) pg_rightmost_one_pos32(flags);
|
|
|
|
switch (type)
|
|
|
|
{
|
2022-02-12 10:21:23 +13:00
|
|
|
case PROCSIGNAL_BARRIER_SMGRRELEASE:
|
|
|
|
processed = ProcessBarrierSmgrRelease();
|
2021-01-18 12:09:52 -05:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* To avoid an infinite loop, we must always unset the bit in
|
|
|
|
* flags.
|
|
|
|
*/
|
|
|
|
BARRIER_CLEAR_BIT(flags, type);
|
|
|
|
|
|
|
|
/*
|
|
|
|
* If we failed to process the barrier, reset the shared bit
|
|
|
|
* so we try again later, and set a flag so that we don't bump
|
|
|
|
* our generation.
|
|
|
|
*/
|
|
|
|
if (!processed)
|
|
|
|
{
|
|
|
|
ResetProcSignalBarrierBits(((uint32) 1) << type);
|
|
|
|
success = false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
PG_CATCH();
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
* If an ERROR occurred, we'll need to try again later to handle
|
|
|
|
* that barrier type and any others that haven't been handled yet
|
|
|
|
* or weren't successfully absorbed.
|
|
|
|
*/
|
|
|
|
ResetProcSignalBarrierBits(flags);
|
|
|
|
PG_RE_THROW();
|
|
|
|
}
|
|
|
|
PG_END_TRY();
|
|
|
|
|
|
|
|
/*
|
|
|
|
* If some barrier types were not successfully absorbed, we will have
|
|
|
|
* to try again later.
|
|
|
|
*/
|
|
|
|
if (!success)
|
|
|
|
return;
|
|
|
|
}
|
2019-12-19 14:56:20 -05:00
|
|
|
|
|
|
|
/*
|
|
|
|
* State changes related to all types of barriers that might have been
|
|
|
|
* emitted have now been handled, so we can update our notion of the
|
|
|
|
* generation to the one we observed before beginning the updates. If
|
|
|
|
* things have changed further, it'll get fixed up when this function is
|
|
|
|
* next called.
|
|
|
|
*/
|
2020-06-15 18:23:10 -07:00
|
|
|
pg_atomic_write_u64(&MyProcSignalSlot->pss_barrierGeneration, shared_gen);
|
2021-03-01 16:26:22 +13:00
|
|
|
ConditionVariableBroadcast(&MyProcSignalSlot->pss_barrierCV);
|
2019-12-19 14:56:20 -05:00
|
|
|
}
|
|
|
|
|
2021-01-18 12:09:52 -05:00
|
|
|
/*
|
|
|
|
* If it turns out that we couldn't absorb one or more barrier types, either
|
|
|
|
* because the barrier-processing functions returned false or due to an error,
|
|
|
|
* arrange for processing to be retried later.
|
|
|
|
*/
|
2019-12-19 14:56:20 -05:00
|
|
|
static void
|
2021-01-18 12:09:52 -05:00
|
|
|
ResetProcSignalBarrierBits(uint32 flags)
|
|
|
|
{
|
|
|
|
pg_atomic_fetch_or_u32(&MyProcSignalSlot->pss_barrierCheckMask, flags);
|
|
|
|
ProcSignalBarrierPending = true;
|
|
|
|
InterruptPending = true;
|
|
|
|
}
|
|
|
|
|
2009-07-31 20:26:23 +00:00
|
|
|
/*
|
|
|
|
* CheckProcSignal - check to see if a particular reason has been
|
|
|
|
* signaled, and clear the signal flag. Should be called after receiving
|
|
|
|
* SIGUSR1.
|
|
|
|
*/
|
|
|
|
static bool
|
|
|
|
CheckProcSignal(ProcSignalReason reason)
|
|
|
|
{
|
|
|
|
volatile ProcSignalSlot *slot = MyProcSignalSlot;
|
|
|
|
|
|
|
|
if (slot != NULL)
|
|
|
|
{
|
|
|
|
/* Careful here --- don't clear flag if we haven't seen it set */
|
|
|
|
if (slot->pss_signalFlags[reason])
|
|
|
|
{
|
|
|
|
slot->pss_signalFlags[reason] = false;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
* procsignal_sigusr1_handler - handle SIGUSR1 signal.
|
|
|
|
*/
|
|
|
|
void
|
|
|
|
procsignal_sigusr1_handler(SIGNAL_ARGS)
|
|
|
|
{
|
|
|
|
int save_errno = errno;
|
|
|
|
|
|
|
|
if (CheckProcSignal(PROCSIG_CATCHUP_INTERRUPT))
|
|
|
|
HandleCatchupInterrupt();
|
|
|
|
|
|
|
|
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
|
|
|
|
HandleNotifyInterrupt();
|
|
|
|
|
Create an infrastructure for parallel computation in PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Noah
Misch, Simon Riggs, Euler Taveira, and Jim Nasby.
2015-04-30 15:02:14 -04:00
|
|
|
if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
|
|
|
|
HandleParallelMessageInterrupt();
|
|
|
|
|
Prevent possibility of panics during shutdown checkpoint.
When the checkpointer writes the shutdown checkpoint, it checks
afterwards whether any WAL has been written since it started and
throws a PANIC if so. At that point, only walsenders are still
active, so one might think this could not happen, but walsenders can
also generate WAL, for instance in BASE_BACKUP and logical decoding
related commands (e.g. via hint bits). So they can trigger this panic
if such a command is run while the shutdown checkpoint is being
written.
To fix this, divide the walsender shutdown into two phases. First,
checkpointer, itself triggered by postmaster, sends a
PROCSIG_WALSND_INIT_STOPPING signal to all walsenders. If the backend
is idle or runs an SQL query this causes the backend to shutdown, if
logical replication is in progress all existing WAL records are
processed followed by a shutdown. Otherwise this causes the walsender
to switch to the "stopping" state. In this state, the walsender will
reject any further replication commands. The checkpointer begins the
shutdown checkpoint once all walsenders are confirmed as
stopping. When the shutdown checkpoint finishes, the postmaster sends
us SIGUSR2. This instructs walsender to send any outstanding WAL,
including the shutdown checkpoint record, wait for it to be replicated
to the standby, and then exit.
Author: Andres Freund, based on an earlier patch by Michael Paquier
Reported-By: Fujii Masao, Andres Freund
Reviewed-By: Michael Paquier
Discussion: https://postgr.es/m/20170602002912.tqlwn4gymzlxpvs2@alap3.anarazel.de
Backpatch: 9.4, where logical decoding was introduced
2017-06-05 18:53:41 -07:00
|
|
|
if (CheckProcSignal(PROCSIG_WALSND_INIT_STOPPING))
|
|
|
|
HandleWalSndInitStopping();
|
|
|
|
|
2020-06-15 18:23:10 -07:00
|
|
|
if (CheckProcSignal(PROCSIG_BARRIER))
|
|
|
|
HandleProcSignalBarrierInterrupt();
|
|
|
|
|
Add function to log the memory contexts of specified backend process.
Commit 3e98c0bafb added pg_backend_memory_contexts view to display
the memory contexts of the backend process. However its target process
is limited to the backend that is accessing to the view. So this is
not so convenient when investigating the local memory bloat of other
backend process. To improve this situation, this commit adds
pg_log_backend_memory_contexts() function that requests to log
the memory contexts of the specified backend process.
This information can be also collected by calling
MemoryContextStats(TopMemoryContext) via a debugger. But
this technique cannot be used in some environments because no debugger
is available there. So, pg_log_backend_memory_contexts() allows us to
see the memory contexts of specified backend more easily.
Only superusers are allowed to request to log the memory contexts
because allowing any users to issue this request at an unbounded rate
would cause lots of log messages and which can lead to denial of service.
On receipt of the request, at the next CHECK_FOR_INTERRUPTS(),
the target backend logs its memory contexts at LOG_SERVER_ONLY level,
so that these memory contexts will appear in the server log but not
be sent to the client. It logs one message per memory context.
Because if it buffers all memory contexts into StringInfo to log them
as one message, which may require the buffer to be enlarged very much
and lead to OOM error since there can be a large number of memory
contexts in a backend.
When a backend process is consuming huge memory, logging all its
memory contexts might overrun available disk space. To prevent this,
now this patch limits the number of child contexts to log per parent
to 100. As with MemoryContextStats(), it supposes that practical cases
where the log gets long will typically be huge numbers of siblings
under the same parent context; while the additional debugging value
from seeing details about individual siblings beyond 100 will not be large.
There was another proposed patch to add the function to return
the memory contexts of specified backend as the result sets,
instead of logging them, in the discussion. However that patch is
not included in this commit because it had several issues to address.
Thanks to Tatsuhito Kasahara, Andres Freund, Tom Lane, Tomas Vondra,
Michael Paquier, Kyotaro Horiguchi and Zhihong Yu for the discussion.
Bump catalog version.
Author: Atsushi Torikoshi
Reviewed-by: Kyotaro Horiguchi, Zhihong Yu, Fujii Masao
Discussion: https://postgr.es/m/0271f440ac77f2a4180e0e56ebd944d1@oss.nttdata.com
2021-04-06 13:44:15 +09:00
|
|
|
if (CheckProcSignal(PROCSIG_LOG_MEMORY_CONTEXT))
|
|
|
|
HandleLogMemoryContextInterrupt();
|
|
|
|
|
2010-01-16 10:05:59 +00:00
|
|
|
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
|
|
|
|
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
|
|
|
|
|
|
|
|
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_TABLESPACE))
|
|
|
|
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_TABLESPACE);
|
|
|
|
|
|
|
|
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_LOCK))
|
|
|
|
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_LOCK);
|
|
|
|
|
|
|
|
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_SNAPSHOT))
|
|
|
|
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_SNAPSHOT);
|
|
|
|
|
2010-02-13 01:32:20 +00:00
|
|
|
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_STARTUP_DEADLOCK))
|
|
|
|
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_STARTUP_DEADLOCK);
|
|
|
|
|
2010-01-23 16:37:12 +00:00
|
|
|
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_BUFFERPIN))
|
|
|
|
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_BUFFERPIN);
|
|
|
|
|
2015-10-09 14:31:04 -04:00
|
|
|
SetLatch(MyLatch);
|
2013-08-28 14:08:13 -04:00
|
|
|
|
2009-07-31 20:26:23 +00:00
|
|
|
errno = save_errno;
|
|
|
|
}
|