postgresql/src/backend/storage/lmgr/proc.c

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

1964 lines
61 KiB
C
Raw Normal View History

/*-------------------------------------------------------------------------
*
* proc.c
* routines to manage per-process shared memory data structure
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
*
* IDENTIFICATION
2010-09-20 22:08:53 +02:00
* src/backend/storage/lmgr/proc.c
*
*-------------------------------------------------------------------------
*/
/*
* Interface (a):
* ProcSleep(), ProcWakeup(),
*
* Waiting for a lock causes the backend to be put to sleep. Whoever releases
* the lock wakes the process up again (and gives it an error code so it knows
* whether it was awoken on an error condition).
*
* Interface (b):
*
* ProcReleaseLocks -- frees the locks associated with current transaction
*
* ProcKill -- destroys the shared memory state (and locks)
* associated with the process.
*/
#include "postgres.h"
#include <signal.h>
#include <unistd.h>
#include <sys/time.h>
#include "access/transam.h"
#include "access/twophase.h"
#include "access/xlogutils.h"
#include "commands/waitlsn.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "postmaster/autovacuum.h"
#include "replication/slotsync.h"
#include "replication/syncrep.h"
#include "storage/condition_variable.h"
#include "storage/ipc.h"
#include "storage/lmgr.h"
#include "storage/pmsignal.h"
#include "storage/proc.h"
#include "storage/procarray.h"
#include "storage/procsignal.h"
#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/timeout.h"
#include "utils/timestamp.h"
/* GUC variables */
int DeadlockTimeout = 1000;
int StatementTimeout = 0;
int LockTimeout = 0;
int IdleInTransactionSessionTimeout = 0;
int TransactionTimeout = 0;
int IdleSessionTimeout = 0;
bool log_lock_waits = false;
/* Pointer to this process's PGPROC struct, if any */
PGPROC *MyProc = NULL;
/*
* This spinlock protects the freelist of recycled PGPROC structures.
* We cannot use an LWLock because the LWLock manager depends on already
* having a PGPROC and a wait semaphore! But these structures are touched
* relatively infrequently (only at backend startup or shutdown) and not for
* very long, so a spinlock is okay.
*/
NON_EXEC_STATIC slock_t *ProcStructLock = NULL;
/* Pointers to shared-memory structures */
PROC_HDR *ProcGlobal = NULL;
NON_EXEC_STATIC PGPROC *AuxiliaryProcs = NULL;
PGPROC *PreparedXactProcs = NULL;
/* If we are waiting for a lock, this points to the associated LOCALLOCK */
static LOCALLOCK *lockAwaited = NULL;
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
static DeadLockState deadlock_state = DS_NOT_YET_CHECKED;
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
/* Is a deadlock check pending? */
static volatile sig_atomic_t got_deadlock_timeout;
static void RemoveProcFromArray(int code, Datum arg);
static void ProcKill(int code, Datum arg);
static void AuxiliaryProcKill(int code, Datum arg);
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
static void CheckDeadLock(void);
/*
* Report shared-memory space needed by InitProcGlobal.
*/
Size
ProcGlobalShmemSize(void)
{
Size size = 0;
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
Size TotalProcs =
add_size(MaxBackends, add_size(NUM_AUXILIARY_PROCS, max_prepared_xacts));
/* ProcGlobal */
size = add_size(size, sizeof(PROC_HDR));
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
size = add_size(size, mul_size(TotalProcs, sizeof(PGPROC)));
size = add_size(size, sizeof(slock_t));
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
size = add_size(size, mul_size(TotalProcs, sizeof(*ProcGlobal->xids)));
size = add_size(size, mul_size(TotalProcs, sizeof(*ProcGlobal->subxidStates)));
size = add_size(size, mul_size(TotalProcs, sizeof(*ProcGlobal->statusFlags)));
return size;
}
/*
* Report number of semaphores needed by InitProcGlobal.
*/
int
ProcGlobalSemas(void)
{
/*
* We need a sema per backend (including autovacuum), plus one for each
* auxiliary process.
*/
return MaxBackends + NUM_AUXILIARY_PROCS;
}
/*
* InitProcGlobal -
* Initialize the global process table during postmaster or standalone
* backend startup.
*
* We also create all the per-process semaphores we will need to support
* the requested number of backends. We used to allocate semaphores
* only when backends were actually started up, but that is bad because
* it lets Postgres fail under load --- a lot of Unix systems are
* (mis)configured with small limits on the number of semaphores, and
* running out when trying to start another backend is a common failure.
* So, now we grab enough semaphores to support the desired max number
* of backends immediately at initialization --- if the sysadmin has set
2019-02-12 02:07:56 +01:00
* MaxConnections, max_worker_processes, max_wal_senders, or
* autovacuum_max_workers higher than his kernel will support, he'll
* find out sooner rather than later.
*
* Another reason for creating semaphores here is that the semaphore
* implementation typically requires us to create semaphores in the
* postmaster, not in backends.
*
* Note: this is NOT called by individual backends under a postmaster,
* not even in the EXEC_BACKEND case. The ProcGlobal and AuxiliaryProcs
* pointers must be propagated specially for EXEC_BACKEND operation.
*/
void
InitProcGlobal(void)
{
PGPROC *procs;
int i,
j;
bool found;
uint32 TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS + max_prepared_xacts;
/* Create the ProcGlobal shared structure */
ProcGlobal = (PROC_HDR *)
ShmemInitStruct("Proc Header", sizeof(PROC_HDR), &found);
Assert(!found);
/*
* Initialize the data structures.
*/
ProcGlobal->spins_per_delay = DEFAULT_SPINS_PER_DELAY;
dlist_init(&ProcGlobal->freeProcs);
dlist_init(&ProcGlobal->autovacFreeProcs);
dlist_init(&ProcGlobal->bgworkerFreeProcs);
dlist_init(&ProcGlobal->walsenderFreeProcs);
ProcGlobal->startupBufferPinWaitBufId = -1;
ProcGlobal->walwriterLatch = NULL;
ProcGlobal->checkpointerLatch = NULL;
pg_atomic_init_u32(&ProcGlobal->procArrayGroupFirst, INVALID_PROC_NUMBER);
pg_atomic_init_u32(&ProcGlobal->clogGroupFirst, INVALID_PROC_NUMBER);
/*
* Create and initialize all the PGPROC structures we'll need. There are
Background worker processes Background workers are postmaster subprocesses that run arbitrary user-specified code. They can request shared memory access as well as backend database connections; or they can just use plain libpq frontend database connections. Modules listed in shared_preload_libraries can register background workers in their _PG_init() function; this is early enough that it's not necessary to provide an extra GUC option, because the necessary extra resources can be allocated early on. Modules can install more than one bgworker, if necessary. Care is taken that these extra processes do not interfere with other postmaster tasks: only one such process is started on each ServerLoop iteration. This means a large number of them could be waiting to be started up and postmaster is still able to quickly service external connection requests. Also, shutdown sequence should not be impacted by a worker process that's reasonably well behaved (i.e. promptly responds to termination signals.) The current implementation lets worker processes specify their start time, i.e. at what point in the server startup process they are to be started: right after postmaster start (in which case they mustn't ask for shared memory access), when consistent state has been reached (useful during recovery in a HOT standby server), or when recovery has terminated (i.e. when normal backends are allowed). In case of a bgworker crash, actions to take depend on registration data: if shared memory was requested, then all other connections are taken down (as well as other bgworkers), just like it were a regular backend crashing. The bgworker itself is restarted, too, within a configurable timeframe (which can be configured to be never). More features to add to this framework can be imagined without much effort, and have been discussed, but this seems good enough as a useful unit already. An elementary sample module is supplied. Author: Álvaro Herrera This patch is loosely based on prior patches submitted by KaiGai Kohei, and unsubmitted code by Simon Riggs. Reviewed by: KaiGai Kohei, Markus Wanner, Andres Freund, Heikki Linnakangas, Simon Riggs, Amit Kapila
2012-12-06 18:57:52 +01:00
* five separate consumers: (1) normal backends, (2) autovacuum workers
* and the autovacuum launcher, (3) background workers, (4) auxiliary
* processes, and (5) prepared transactions. Each PGPROC structure is
* dedicated to exactly one of these purposes, and they do not move
* between groups.
*/
procs = (PGPROC *) ShmemAlloc(TotalProcs * sizeof(PGPROC));
Change API of ShmemAlloc() so it throws error rather than returning NULL. A majority of callers seem to have believed that this was the API spec already, because they omitted any check for a NULL result, and hence would crash on an out-of-shared-memory failure. The original proposal was to just add such error checks everywhere, but that does nothing to prevent similar omissions in future. Instead, let's make ShmemAlloc() throw the error (so we can remove the caller-side checks that do exist), and introduce a new function ShmemAllocNoError() that has the previous behavior of returning NULL, for the small number of callers that need that and are prepared to do the right thing. This also lets us remove the rather wishy-washy behavior of printing a WARNING for out-of-shmem, which never made much sense: either the caller has a strategy for dealing with that, or it doesn't. It's not ShmemAlloc's business to decide whether a warning is appropriate. The v10 release notes will need to call this out as a significant source-code change. It's likely that it will be a bug fix for extension callers too, but if not, they'll need to change to using ShmemAllocNoError(). This is nominally a bug fix, but the odds that it's fixing any live bug are actually rather small, because in general the requests being made by the unchecked callers were already accounted for in determining the overall shmem size, so really they ought not fail. Between that and the possible impact on extensions, no back-patch. Discussion: <24843.1472563085@sss.pgh.pa.us>
2016-09-01 16:13:55 +02:00
MemSet(procs, 0, TotalProcs * sizeof(PGPROC));
ProcGlobal->allProcs = procs;
/* XXX allProcCount isn't really all of them; it excludes prepared xacts */
ProcGlobal->allProcCount = MaxBackends + NUM_AUXILIARY_PROCS;
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
/*
* Allocate arrays mirroring PGPROC fields in a dense manner. See
* PROC_HDR.
*
* XXX: It might make sense to increase padding for these arrays, given
* how hotly they are accessed.
*/
ProcGlobal->xids =
(TransactionId *) ShmemAlloc(TotalProcs * sizeof(*ProcGlobal->xids));
MemSet(ProcGlobal->xids, 0, TotalProcs * sizeof(*ProcGlobal->xids));
ProcGlobal->subxidStates = (XidCacheStatus *) ShmemAlloc(TotalProcs * sizeof(*ProcGlobal->subxidStates));
MemSet(ProcGlobal->subxidStates, 0, TotalProcs * sizeof(*ProcGlobal->subxidStates));
ProcGlobal->statusFlags = (uint8 *) ShmemAlloc(TotalProcs * sizeof(*ProcGlobal->statusFlags));
MemSet(ProcGlobal->statusFlags, 0, TotalProcs * sizeof(*ProcGlobal->statusFlags));
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
for (i = 0; i < TotalProcs; i++)
{
PGPROC *proc = &procs[i];
/* Common initialization for all PGPROCs, regardless of type. */
/*
* Set up per-PGPROC semaphore, latch, and fpInfoLock. Prepared xact
* dummy PGPROCs don't need these though - they're never associated
* with a real process
*/
if (i < MaxBackends + NUM_AUXILIARY_PROCS)
{
proc->sem = PGSemaphoreCreate();
InitSharedLatch(&(proc->procLatch));
LWLockInitialize(&(proc->fpInfoLock), LWTRANCHE_LOCK_FASTPATH);
}
/*
Background worker processes Background workers are postmaster subprocesses that run arbitrary user-specified code. They can request shared memory access as well as backend database connections; or they can just use plain libpq frontend database connections. Modules listed in shared_preload_libraries can register background workers in their _PG_init() function; this is early enough that it's not necessary to provide an extra GUC option, because the necessary extra resources can be allocated early on. Modules can install more than one bgworker, if necessary. Care is taken that these extra processes do not interfere with other postmaster tasks: only one such process is started on each ServerLoop iteration. This means a large number of them could be waiting to be started up and postmaster is still able to quickly service external connection requests. Also, shutdown sequence should not be impacted by a worker process that's reasonably well behaved (i.e. promptly responds to termination signals.) The current implementation lets worker processes specify their start time, i.e. at what point in the server startup process they are to be started: right after postmaster start (in which case they mustn't ask for shared memory access), when consistent state has been reached (useful during recovery in a HOT standby server), or when recovery has terminated (i.e. when normal backends are allowed). In case of a bgworker crash, actions to take depend on registration data: if shared memory was requested, then all other connections are taken down (as well as other bgworkers), just like it were a regular backend crashing. The bgworker itself is restarted, too, within a configurable timeframe (which can be configured to be never). More features to add to this framework can be imagined without much effort, and have been discussed, but this seems good enough as a useful unit already. An elementary sample module is supplied. Author: Álvaro Herrera This patch is loosely based on prior patches submitted by KaiGai Kohei, and unsubmitted code by Simon Riggs. Reviewed by: KaiGai Kohei, Markus Wanner, Andres Freund, Heikki Linnakangas, Simon Riggs, Amit Kapila
2012-12-06 18:57:52 +01:00
* Newly created PGPROCs for normal backends, autovacuum and bgworkers
* must be queued up on the appropriate free list. Because there can
* only ever be a small, fixed number of auxiliary processes, no free
* list is used in that case; InitAuxiliaryProcess() instead uses a
* linear search. PGPROCs for prepared transactions are added to a
* free list by TwoPhaseShmemInit().
*/
if (i < MaxConnections)
{
/* PGPROC for normal backend, add to freeProcs list */
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
dlist_push_tail(&ProcGlobal->freeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->freeProcs;
}
Background worker processes Background workers are postmaster subprocesses that run arbitrary user-specified code. They can request shared memory access as well as backend database connections; or they can just use plain libpq frontend database connections. Modules listed in shared_preload_libraries can register background workers in their _PG_init() function; this is early enough that it's not necessary to provide an extra GUC option, because the necessary extra resources can be allocated early on. Modules can install more than one bgworker, if necessary. Care is taken that these extra processes do not interfere with other postmaster tasks: only one such process is started on each ServerLoop iteration. This means a large number of them could be waiting to be started up and postmaster is still able to quickly service external connection requests. Also, shutdown sequence should not be impacted by a worker process that's reasonably well behaved (i.e. promptly responds to termination signals.) The current implementation lets worker processes specify their start time, i.e. at what point in the server startup process they are to be started: right after postmaster start (in which case they mustn't ask for shared memory access), when consistent state has been reached (useful during recovery in a HOT standby server), or when recovery has terminated (i.e. when normal backends are allowed). In case of a bgworker crash, actions to take depend on registration data: if shared memory was requested, then all other connections are taken down (as well as other bgworkers), just like it were a regular backend crashing. The bgworker itself is restarted, too, within a configurable timeframe (which can be configured to be never). More features to add to this framework can be imagined without much effort, and have been discussed, but this seems good enough as a useful unit already. An elementary sample module is supplied. Author: Álvaro Herrera This patch is loosely based on prior patches submitted by KaiGai Kohei, and unsubmitted code by Simon Riggs. Reviewed by: KaiGai Kohei, Markus Wanner, Andres Freund, Heikki Linnakangas, Simon Riggs, Amit Kapila
2012-12-06 18:57:52 +01:00
else if (i < MaxConnections + autovacuum_max_workers + 1)
{
/* PGPROC for AV launcher/worker, add to autovacFreeProcs list */
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
dlist_push_tail(&ProcGlobal->autovacFreeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->autovacFreeProcs;
}
2019-02-12 02:07:56 +01:00
else if (i < MaxConnections + autovacuum_max_workers + 1 + max_worker_processes)
Background worker processes Background workers are postmaster subprocesses that run arbitrary user-specified code. They can request shared memory access as well as backend database connections; or they can just use plain libpq frontend database connections. Modules listed in shared_preload_libraries can register background workers in their _PG_init() function; this is early enough that it's not necessary to provide an extra GUC option, because the necessary extra resources can be allocated early on. Modules can install more than one bgworker, if necessary. Care is taken that these extra processes do not interfere with other postmaster tasks: only one such process is started on each ServerLoop iteration. This means a large number of them could be waiting to be started up and postmaster is still able to quickly service external connection requests. Also, shutdown sequence should not be impacted by a worker process that's reasonably well behaved (i.e. promptly responds to termination signals.) The current implementation lets worker processes specify their start time, i.e. at what point in the server startup process they are to be started: right after postmaster start (in which case they mustn't ask for shared memory access), when consistent state has been reached (useful during recovery in a HOT standby server), or when recovery has terminated (i.e. when normal backends are allowed). In case of a bgworker crash, actions to take depend on registration data: if shared memory was requested, then all other connections are taken down (as well as other bgworkers), just like it were a regular backend crashing. The bgworker itself is restarted, too, within a configurable timeframe (which can be configured to be never). More features to add to this framework can be imagined without much effort, and have been discussed, but this seems good enough as a useful unit already. An elementary sample module is supplied. Author: Álvaro Herrera This patch is loosely based on prior patches submitted by KaiGai Kohei, and unsubmitted code by Simon Riggs. Reviewed by: KaiGai Kohei, Markus Wanner, Andres Freund, Heikki Linnakangas, Simon Riggs, Amit Kapila
2012-12-06 18:57:52 +01:00
{
/* PGPROC for bgworker, add to bgworkerFreeProcs list */
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
dlist_push_tail(&ProcGlobal->bgworkerFreeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->bgworkerFreeProcs;
Background worker processes Background workers are postmaster subprocesses that run arbitrary user-specified code. They can request shared memory access as well as backend database connections; or they can just use plain libpq frontend database connections. Modules listed in shared_preload_libraries can register background workers in their _PG_init() function; this is early enough that it's not necessary to provide an extra GUC option, because the necessary extra resources can be allocated early on. Modules can install more than one bgworker, if necessary. Care is taken that these extra processes do not interfere with other postmaster tasks: only one such process is started on each ServerLoop iteration. This means a large number of them could be waiting to be started up and postmaster is still able to quickly service external connection requests. Also, shutdown sequence should not be impacted by a worker process that's reasonably well behaved (i.e. promptly responds to termination signals.) The current implementation lets worker processes specify their start time, i.e. at what point in the server startup process they are to be started: right after postmaster start (in which case they mustn't ask for shared memory access), when consistent state has been reached (useful during recovery in a HOT standby server), or when recovery has terminated (i.e. when normal backends are allowed). In case of a bgworker crash, actions to take depend on registration data: if shared memory was requested, then all other connections are taken down (as well as other bgworkers), just like it were a regular backend crashing. The bgworker itself is restarted, too, within a configurable timeframe (which can be configured to be never). More features to add to this framework can be imagined without much effort, and have been discussed, but this seems good enough as a useful unit already. An elementary sample module is supplied. Author: Álvaro Herrera This patch is loosely based on prior patches submitted by KaiGai Kohei, and unsubmitted code by Simon Riggs. Reviewed by: KaiGai Kohei, Markus Wanner, Andres Freund, Heikki Linnakangas, Simon Riggs, Amit Kapila
2012-12-06 18:57:52 +01:00
}
else if (i < MaxBackends)
2019-02-12 02:07:56 +01:00
{
/* PGPROC for walsender, add to walsenderFreeProcs list */
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
dlist_push_tail(&ProcGlobal->walsenderFreeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->walsenderFreeProcs;
2019-02-12 02:07:56 +01:00
}
/* Initialize myProcLocks[] shared memory queues. */
for (j = 0; j < NUM_LOCK_PARTITIONS; j++)
dlist_init(&(proc->myProcLocks[j]));
/* Initialize lockGroupMembers list. */
dlist_init(&proc->lockGroupMembers);
/*
* Initialize the atomic variables, otherwise, it won't be safe to
* access them for backends that aren't currently in use.
*/
pg_atomic_init_u32(&(proc->procArrayGroupNext), INVALID_PROC_NUMBER);
pg_atomic_init_u32(&(proc->clogGroupNext), INVALID_PROC_NUMBER);
pg_atomic_init_u64(&(proc->waitStart), 0);
}
/*
* Save pointers to the blocks of PGPROC structures reserved for auxiliary
* processes and prepared transactions.
*/
AuxiliaryProcs = &procs[MaxBackends];
PreparedXactProcs = &procs[MaxBackends + NUM_AUXILIARY_PROCS];
/* Create ProcStructLock spinlock, too */
ProcStructLock = (slock_t *) ShmemAlloc(sizeof(slock_t));
SpinLockInit(ProcStructLock);
}
/*
* InitProcess -- initialize a per-process PGPROC entry for this backend
*/
void
InitProcess(void)
{
dlist_head *procgloballist;
/*
* ProcGlobal should be set up already (if we are a backend, we inherit
* this by fork() or EXEC_BACKEND mechanism from the postmaster).
*/
if (ProcGlobal == NULL)
elog(PANIC, "proc header uninitialized");
if (MyProc != NULL)
elog(ERROR, "you already exist");
/* Decide which list should supply our PGPROC. */
if (AmAutoVacuumLauncherProcess() || AmAutoVacuumWorkerProcess())
procgloballist = &ProcGlobal->autovacFreeProcs;
else if (AmBackgroundWorkerProcess())
procgloballist = &ProcGlobal->bgworkerFreeProcs;
else if (AmWalSenderProcess())
2019-02-12 02:07:56 +01:00
procgloballist = &ProcGlobal->walsenderFreeProcs;
else
procgloballist = &ProcGlobal->freeProcs;
/*
* Try to get a proc struct from the appropriate free list. If this
* fails, we must be out of PGPROC structures (not to mention semaphores).
*
* While we are holding the ProcStructLock, also copy the current shared
* estimate of spins_per_delay to local storage.
*/
SpinLockAcquire(ProcStructLock);
set_spins_per_delay(ProcGlobal->spins_per_delay);
if (!dlist_is_empty(procgloballist))
{
MyProc = (PGPROC *) dlist_pop_head_node(procgloballist);
SpinLockRelease(ProcStructLock);
}
else
{
/*
* If we reach here, all the PGPROCs are in use. This is one of the
* possible places to detect "too many backends", so give the standard
* error message. XXX do we need to give a different failure message
* in the autovacuum case?
*/
SpinLockRelease(ProcStructLock);
if (AmWalSenderProcess())
2019-02-12 02:07:56 +01:00
ereport(FATAL,
(errcode(ERRCODE_TOO_MANY_CONNECTIONS),
errmsg("number of requested standby connections exceeds max_wal_senders (currently %d)",
max_wal_senders)));
ereport(FATAL,
(errcode(ERRCODE_TOO_MANY_CONNECTIONS),
errmsg("sorry, too many clients already")));
}
MyProcNumber = GetNumberFromPGProc(MyProc);
/*
* Cross-check that the PGPROC is of the type we expect; if this were not
* the case, it would get returned to the wrong list.
*/
Assert(MyProc->procgloballist == procgloballist);
/*
* Now that we have a PGPROC, 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.
*/
if (IsUnderPostmaster && !AmAutoVacuumLauncherProcess() &&
!AmLogicalSlotSyncWorkerProcess())
MarkPostmasterChildActive();
/*
* Initialize all fields of MyProc, except for those previously
* initialized by InitProcGlobal.
*/
dlist_node_init(&MyProc->links);
MyProc->waitStatus = PROC_WAIT_STATUS_OK;
MyProc->fpVXIDLock = false;
MyProc->fpLocalTransactionId = InvalidLocalTransactionId;
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
MyProc->pid = MyProcPid;
MyProc->vxid.procNumber = MyProcNumber;
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
MyProc->vxid.lxid = InvalidLocalTransactionId;
/* databaseId and roleId will be filled in later */
MyProc->databaseId = InvalidOid;
MyProc->roleId = InvalidOid;
Make autovacuum more aggressive to remove orphaned temp tables Commit dafa084, added in 10, made the removal of temporary orphaned tables more aggressive. This commit makes an extra step into the aggressiveness by adding a flag in each backend's MyProc which tracks down any temporary namespace currently in use. The flag is set when the namespace gets created and can be reset if the temporary namespace has been created in a transaction or sub-transaction which is aborted. The flag value assignment is assumed to be atomic, so this can be done in a lock-less fashion like other flags already present in PGPROC like databaseId or backendId, still the fact that the temporary namespace and table created are still locked until the transaction creating those commits acts as a barrier for other backends. This new flag gets used by autovacuum to discard more aggressively orphaned tables by additionally checking for the database a backend is connected to as well as its temporary namespace in-use, removing orphaned temporary relations even if a backend reuses the same slot as one which created temporary relations in a past session. The base idea of this patch comes from Robert Haas, has been written in its first version by Tsunakawa Takayuki, then heavily reviewed by me. Author: Tsunakawa Takayuki Reviewed-by: Michael Paquier, Kyotaro Horiguchi, Andres Freund Discussion: https://postgr.es/m/0A3221C70F24FB45833433255569204D1F8A4DC6@G01JPEXMBYT05 Backpatch: 11-, as PGPROC gains a new flag and we don't want silent ABI breakages on already released versions.
2018-08-13 11:49:04 +02:00
MyProc->tempNamespaceId = InvalidOid;
MyProc->isBackgroundWorker = AmBackgroundWorkerProcess();
MyProc->delayChkptFlags = 0;
MyProc->statusFlags = 0;
/* NB -- autovac launcher intentionally does not set IS_AUTOVACUUM */
if (AmAutoVacuumWorkerProcess())
MyProc->statusFlags |= PROC_IS_AUTOVACUUM;
lwlock: Fix quadratic behavior with very long wait lists Until now LWLockDequeueSelf() sequentially searched the list of waiters to see if the current proc is still is on the list of waiters, or has already been removed. In extreme workloads, where the wait lists are very long, this leads to a quadratic behavior. #backends iterating over a list #backends long. Additionally, the likelihood of needing to call LWLockDequeueSelf() in the first place also increases with the increased length of the wait queue, as it becomes more likely that a lock is released while waiting for the wait list lock, which is held for longer during lock release. Due to the exponential back-off in perform_spin_delay() this is surprisingly hard to detect. We should make that easier, e.g. by adding a wait event around the pg_usleep() - but that's a separate patch. The fix is simple - track whether a proc is currently waiting in the wait list or already removed but waiting to be woken up in PGPROC->lwWaiting. In some workloads with a lot of clients contending for a small number of lwlocks (e.g. WALWriteLock), the fix can substantially increase throughput. As the quadratic behavior arguably is a bug, we might want to decide to backpatch this fix in the future. Author: Andres Freund <andres@anarazel.de> Reviewed-by: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> Discussion: https://postgr.es/m/20221027165914.2hofzp4cvutj6gin@awork3.anarazel.de Discussion: https://postgr.es/m/CALj2ACXktNbG=K8Xi7PSqbofTZozavhaxjatVc14iYaLu4Maag@mail.gmail.com
2022-11-20 20:56:32 +01:00
MyProc->lwWaiting = LW_WS_NOT_WAITING;
MyProc->lwWaitMode = 0;
MyProc->waitLock = NULL;
MyProc->waitProcLock = NULL;
pg_atomic_write_u64(&MyProc->waitStart, 0);
#ifdef USE_ASSERT_CHECKING
{
int i;
/* Last process should have released all locks. */
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
Assert(dlist_is_empty(&(MyProc->myProcLocks[i])));
}
#endif
MyProc->recoveryConflictPending = false;
/* Initialize fields for sync rep */
MyProc->waitLSN = 0;
MyProc->syncRepState = SYNC_REP_NOT_WAITING;
dlist_node_init(&MyProc->syncRepLinks);
/* Initialize fields for group XID clearing. */
MyProc->procArrayGroupMember = false;
MyProc->procArrayGroupMemberXid = InvalidTransactionId;
Assert(pg_atomic_read_u32(&MyProc->procArrayGroupNext) == INVALID_PROC_NUMBER);
/* Check that group locking fields are in a proper initial state. */
Assert(MyProc->lockGroupLeader == NULL);
Assert(dlist_is_empty(&MyProc->lockGroupMembers));
/* Initialize wait event information. */
MyProc->wait_event_info = 0;
/* Initialize fields for group transaction status update. */
MyProc->clogGroupMember = false;
MyProc->clogGroupMemberXid = InvalidTransactionId;
MyProc->clogGroupMemberXidStatus = TRANSACTION_STATUS_IN_PROGRESS;
MyProc->clogGroupMemberPage = -1;
MyProc->clogGroupMemberLsn = InvalidXLogRecPtr;
Assert(pg_atomic_read_u32(&MyProc->clogGroupNext) == INVALID_PROC_NUMBER);
/*
* Acquire ownership of the PGPROC's latch, so that we can use WaitLatch
* on it. That allows us to repoint the process latch, which so far
* points to process local one, to the shared one.
*/
OwnLatch(&MyProc->procLatch);
SwitchToSharedLatch();
Improve efficiency of wait event reporting, remove proc.h dependency. pgstat_report_wait_start() and pgstat_report_wait_end() required two conditional branches so far. One to check if MyProc is NULL, the other to check if pgstat_track_activities is set. As wait events are used around comparatively lightweight operations, and are inlined (reducing branch predictor effectiveness), that's not great. The dependency on MyProc has a second disadvantage: Low-level subsystems, like storage/file/fd.c, report wait events, but architecturally it is preferable for them to not depend on inter-process subsystems like proc.h (defining PGPROC). After this change including pgstat.h (nor obviously its sub-components like backend_status.h, wait_event.h, ...) does not pull in IPC related headers anymore. These goals, efficiency and abstraction, are achieved by having pgstat_report_wait_start/end() not interact with MyProc, but instead a new my_wait_event_info variable. At backend startup it points to a local variable, removing the need to check for MyProc being NULL. During process initialization my_wait_event_info is redirected to MyProc->wait_event_info. At shutdown this is reversed. Because wait event reporting now does not need to know about where the wait event is stored, it does not need to know about PGPROC anymore. The removal of the branch for checking pgstat_track_activities is simpler: Don't check anymore. The cost due to the branch are often higher than the store - and even if not, pgstat_track_activities is rarely disabled. The main motivator to commit this work now is that removing the (indirect) pgproc.h include from pgstat.h simplifies a patch to move statistics reporting to shared memory (which still has a chance to get into 14). Author: Andres Freund <andres@anarazel.de> Discussion: https://postgr.es/m/20210402194458.2vu324hkk2djq6ce@alap3.anarazel.de
2021-04-03 20:44:47 +02:00
/* now that we have a proc, report wait events to shared memory */
pgstat_set_wait_event_storage(&MyProc->wait_event_info);
/*
* We might be reusing a semaphore that belonged to a failed process. So
* be careful and reinitialize its value here. (This is not strictly
* necessary anymore, but seems like a good idea for cleanliness.)
*/
Make the different Unix-y semaphore implementations ABI-compatible. Previously, the "sem" field of PGPROC varied in size depending on which kernel semaphore API we were using. That was okay as long as there was only one likely choice per platform, but in the wake of commit ecb0d20a9, that assumption seems rather shaky. It doesn't seem out of the question anymore that an extension compiled against one API choice might be loaded into a postmaster built with another choice. Moreover, this prevents any possibility of selecting the semaphore API at postmaster startup, which might be something we want to do in future. Hence, change PGPROC.sem to be PGSemaphore (i.e. a pointer) for all Unix semaphore APIs, and turn the pointed-to data into an opaque struct whose contents are only known within the responsible modules. For the SysV and unnamed-POSIX APIs, the pointed-to data has to be allocated elsewhere in shared memory, which takes a little bit of rejiggering of the InitShmemAllocation code sequence. (I invented a ShmemAllocUnlocked() function to make that a little cleaner than it used to be. That function is not meant for any uses other than the ones it has now, but it beats having InitShmemAllocation() know explicitly about allocation of space for semaphores and spinlocks.) This change means an extra indirection to access the semaphore data, but since we only touch that when blocking or awakening a process, there shouldn't be any meaningful performance penalty. Moreover, at least for the unnamed-POSIX case on Linux, the sem_t type is quite a bit wider than a pointer, so this reduces sizeof(PGPROC) which seems like a good thing. For the named-POSIX API, there's effectively no change: the PGPROC.sem field was and still is a pointer to something returned by sem_open() in the postmaster's memory space. Document and check the pre-existing limitation that this case can't work in EXEC_BACKEND mode. It did not seem worth unifying the Windows semaphore ABI with the Unix cases, since there's no likelihood of needing ABI compatibility much less runtime switching across those cases. However, we can simplify the Windows code a bit if we define PGSemaphore as being directly a HANDLE, rather than pointer to HANDLE, so let's do that while we're here. (This also ends up being no change in what's physically stored in PGPROC.sem. We're just moving the HANDLE fetch from callees to callers.) It would take a bunch of additional code shuffling to get to the point of actually choosing a semaphore API at postmaster start, but the effects of that would now be localized in the port/XXX_sema.c files, so it seems like fit material for a separate patch. The need for it is unproven as yet, anyhow, whereas the ABI risk to extensions seems real enough. Discussion: https://postgr.es/m/4029.1481413370@sss.pgh.pa.us
2016-12-12 19:32:10 +01:00
PGSemaphoreReset(MyProc->sem);
/*
* Arrange to clean up at backend exit.
*/
on_shmem_exit(ProcKill, 0);
/*
* Now that we have a PGPROC, we could try to acquire locks, so initialize
* local state needed for LWLocks, and the deadlock checker.
*/
InitLWLockAccess();
InitDeadLockChecking();
#ifdef EXEC_BACKEND
/*
* Initialize backend-local pointers to all the shared data structures.
* (We couldn't do this until now because it needs LWLocks.)
*/
if (IsUnderPostmaster)
AttachSharedMemoryStructs();
#endif
}
/*
* InitProcessPhase2 -- make MyProc visible in the shared ProcArray.
*
* This is separate from InitProcess because we can't acquire LWLocks until
* we've created a PGPROC, but in the EXEC_BACKEND case ProcArrayAdd won't
* work until after we've done AttachSharedMemoryStructs.
*/
void
InitProcessPhase2(void)
{
Assert(MyProc != NULL);
/*
* Add our PGPROC to the PGPROC array in shared memory.
*/
ProcArrayAdd(MyProc);
/*
* Arrange to clean that up at backend exit.
*/
on_shmem_exit(RemoveProcFromArray, 0);
}
/*
* InitAuxiliaryProcess -- create a PGPROC entry for an auxiliary process
*
* This is called by bgwriter and similar processes so that they will have a
* MyProc value that's real enough to let them wait for LWLocks. The PGPROC
* and sema that are assigned are one of the extra ones created during
* InitProcGlobal.
*
* Auxiliary processes are presently not expected to wait for real (lockmgr)
* locks, so we need not set up the deadlock checker. They are never added
* to the ProcArray or the sinval messaging mechanism, either. They also
* don't get a VXID assigned, since this is only useful when we actually
* hold lockmgr locks.
Allow read only connections during recovery, known as Hot Standby. Enabled by recovery_connections = on (default) and forcing archive recovery using a recovery.conf. Recovery processing now emulates the original transactions as they are replayed, providing full locking and MVCC behaviour for read only queries. Recovery must enter consistent state before connections are allowed, so there is a delay, typically short, before connections succeed. Replay of recovering transactions can conflict and in some cases deadlock with queries during recovery; these result in query cancellation after max_standby_delay seconds have expired. Infrastructure changes have minor effects on normal running, though introduce four new types of WAL record. New test mode "make standbycheck" allows regression tests of static command behaviour on a standby server while in recovery. Typical and extreme dynamic behaviours have been checked via code inspection and manual testing. Few port specific behaviours have been utilised, though primary testing has been on Linux only so far. This commit is the basic patch. Additional changes will follow in this release to enhance some aspects of behaviour, notably improved handling of conflicts, deadlock detection and query cancellation. Changes to VACUUM FULL are also required. Simon Riggs, with significant and lengthy review by Heikki Linnakangas, including streamlined redesign of snapshot creation and two-phase commit. Important contributions from Florian Pflug, Mark Kirkwood, Merlin Moncure, Greg Stark, Gianni Ciolli, Gabriele Bartolini, Hannu Krosing, Robert Haas, Tatsuo Ishii, Hiroyuki Yamada plus support and feedback from many other community members.
2009-12-19 02:32:45 +01:00
*
* Startup process however uses locks but never waits for them in the
* normal backend sense. Startup process also takes part in sinval messaging
* as a sendOnly process, so never reads messages from sinval queue. So
* Startup process does have a VXID and does show up in pg_locks.
*/
void
InitAuxiliaryProcess(void)
{
PGPROC *auxproc;
int proctype;
/*
* ProcGlobal should be set up already (if we are a backend, we inherit
* this by fork() or EXEC_BACKEND mechanism from the postmaster).
*/
if (ProcGlobal == NULL || AuxiliaryProcs == NULL)
elog(PANIC, "proc header uninitialized");
if (MyProc != NULL)
elog(ERROR, "you already exist");
/*
* We use the ProcStructLock to protect assignment and releasing of
* AuxiliaryProcs entries.
*
* While we are holding the ProcStructLock, also copy the current shared
* estimate of spins_per_delay to local storage.
*/
SpinLockAcquire(ProcStructLock);
set_spins_per_delay(ProcGlobal->spins_per_delay);
/*
* Find a free auxproc ... *big* trouble if there isn't one ...
*/
for (proctype = 0; proctype < NUM_AUXILIARY_PROCS; proctype++)
{
auxproc = &AuxiliaryProcs[proctype];
if (auxproc->pid == 0)
break;
}
if (proctype >= NUM_AUXILIARY_PROCS)
{
SpinLockRelease(ProcStructLock);
elog(FATAL, "all AuxiliaryProcs are in use");
}
/* Mark auxiliary proc as in use by me */
/* use volatile pointer to prevent code rearrangement */
((volatile PGPROC *) auxproc)->pid = MyProcPid;
SpinLockRelease(ProcStructLock);
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
MyProc = auxproc;
MyProcNumber = GetNumberFromPGProc(MyProc);
/*
* Initialize all fields of MyProc, except for those previously
* initialized by InitProcGlobal.
*/
dlist_node_init(&MyProc->links);
MyProc->waitStatus = PROC_WAIT_STATUS_OK;
MyProc->fpVXIDLock = false;
MyProc->fpLocalTransactionId = InvalidLocalTransactionId;
snapshot scalability: Introduce dense array of in-progress xids. The new array contains the xids for all connected backends / in-use PGPROC entries in a dense manner (in contrast to the PGPROC/PGXACT arrays which can have unused entries interspersed). This improves performance because GetSnapshotData() always needs to scan the xids of all live procarray entries and now there's no need to go through the procArray->pgprocnos indirection anymore. As the set of running top-level xids changes rarely, compared to the number of snapshots taken, this substantially increases the likelihood of most data required for a snapshot being in l2 cache. In read-mostly workloads scanning the xids[] array will sufficient to build a snapshot, as most backends will not have an xid assigned. To keep the xid array dense ProcArrayRemove() needs to move entries behind the to-be-removed proc's one further up in the array. Obviously moving array entries cannot happen while a backend sets it xid. I.e. locking needs to prevent that array entries are moved while a backend modifies its xid. To avoid locking ProcArrayLock in GetNewTransactionId() - a fairly hot spot already - ProcArrayAdd() / ProcArrayRemove() now needs to hold XidGenLock in addition to ProcArrayLock. Adding / Removing a procarray entry is not a very frequent operation, even taking 2PC into account. Due to the above, the dense array entries can only be read or modified while holding ProcArrayLock and/or XidGenLock. This prevents a concurrent ProcArrayRemove() from shifting the dense array while it is accessed concurrently. While the new dense array is very good when needing to look at all xids it is less suitable when accessing a single backend's xid. In particular it would be problematic to have to acquire a lock to access a backend's own xid. Therefore a backend's xid is not just stored in the dense array, but also in PGPROC. This also allows a backend to only access the shared xid value when the backend had acquired an xid. The infrastructure added in this commit will be used for the remaining PGXACT fields in subsequent commits. They are kept separate to make review easier. Author: Andres Freund <andres@anarazel.de> Reviewed-By: Robert Haas <robertmhaas@gmail.com> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: David Rowley <dgrowleyml@gmail.com> Discussion: https://postgr.es/m/20200301083601.ews6hz5dduc3w2se@alap3.anarazel.de
2020-08-14 21:15:38 +02:00
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
MyProc->vxid.procNumber = INVALID_PROC_NUMBER;
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
MyProc->vxid.lxid = InvalidLocalTransactionId;
MyProc->databaseId = InvalidOid;
MyProc->roleId = InvalidOid;
Make autovacuum more aggressive to remove orphaned temp tables Commit dafa084, added in 10, made the removal of temporary orphaned tables more aggressive. This commit makes an extra step into the aggressiveness by adding a flag in each backend's MyProc which tracks down any temporary namespace currently in use. The flag is set when the namespace gets created and can be reset if the temporary namespace has been created in a transaction or sub-transaction which is aborted. The flag value assignment is assumed to be atomic, so this can be done in a lock-less fashion like other flags already present in PGPROC like databaseId or backendId, still the fact that the temporary namespace and table created are still locked until the transaction creating those commits acts as a barrier for other backends. This new flag gets used by autovacuum to discard more aggressively orphaned tables by additionally checking for the database a backend is connected to as well as its temporary namespace in-use, removing orphaned temporary relations even if a backend reuses the same slot as one which created temporary relations in a past session. The base idea of this patch comes from Robert Haas, has been written in its first version by Tsunakawa Takayuki, then heavily reviewed by me. Author: Tsunakawa Takayuki Reviewed-by: Michael Paquier, Kyotaro Horiguchi, Andres Freund Discussion: https://postgr.es/m/0A3221C70F24FB45833433255569204D1F8A4DC6@G01JPEXMBYT05 Backpatch: 11-, as PGPROC gains a new flag and we don't want silent ABI breakages on already released versions.
2018-08-13 11:49:04 +02:00
MyProc->tempNamespaceId = InvalidOid;
MyProc->isBackgroundWorker = AmBackgroundWorkerProcess();
MyProc->delayChkptFlags = 0;
MyProc->statusFlags = 0;
lwlock: Fix quadratic behavior with very long wait lists Until now LWLockDequeueSelf() sequentially searched the list of waiters to see if the current proc is still is on the list of waiters, or has already been removed. In extreme workloads, where the wait lists are very long, this leads to a quadratic behavior. #backends iterating over a list #backends long. Additionally, the likelihood of needing to call LWLockDequeueSelf() in the first place also increases with the increased length of the wait queue, as it becomes more likely that a lock is released while waiting for the wait list lock, which is held for longer during lock release. Due to the exponential back-off in perform_spin_delay() this is surprisingly hard to detect. We should make that easier, e.g. by adding a wait event around the pg_usleep() - but that's a separate patch. The fix is simple - track whether a proc is currently waiting in the wait list or already removed but waiting to be woken up in PGPROC->lwWaiting. In some workloads with a lot of clients contending for a small number of lwlocks (e.g. WALWriteLock), the fix can substantially increase throughput. As the quadratic behavior arguably is a bug, we might want to decide to backpatch this fix in the future. Author: Andres Freund <andres@anarazel.de> Reviewed-by: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> Discussion: https://postgr.es/m/20221027165914.2hofzp4cvutj6gin@awork3.anarazel.de Discussion: https://postgr.es/m/CALj2ACXktNbG=K8Xi7PSqbofTZozavhaxjatVc14iYaLu4Maag@mail.gmail.com
2022-11-20 20:56:32 +01:00
MyProc->lwWaiting = LW_WS_NOT_WAITING;
MyProc->lwWaitMode = 0;
MyProc->waitLock = NULL;
MyProc->waitProcLock = NULL;
pg_atomic_write_u64(&MyProc->waitStart, 0);
#ifdef USE_ASSERT_CHECKING
{
int i;
/* Last process should have released all locks. */
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
Assert(dlist_is_empty(&(MyProc->myProcLocks[i])));
}
#endif
/*
* Acquire ownership of the PGPROC's latch, so that we can use WaitLatch
* on it. That allows us to repoint the process latch, which so far
* points to process local one, to the shared one.
*/
OwnLatch(&MyProc->procLatch);
SwitchToSharedLatch();
Improve efficiency of wait event reporting, remove proc.h dependency. pgstat_report_wait_start() and pgstat_report_wait_end() required two conditional branches so far. One to check if MyProc is NULL, the other to check if pgstat_track_activities is set. As wait events are used around comparatively lightweight operations, and are inlined (reducing branch predictor effectiveness), that's not great. The dependency on MyProc has a second disadvantage: Low-level subsystems, like storage/file/fd.c, report wait events, but architecturally it is preferable for them to not depend on inter-process subsystems like proc.h (defining PGPROC). After this change including pgstat.h (nor obviously its sub-components like backend_status.h, wait_event.h, ...) does not pull in IPC related headers anymore. These goals, efficiency and abstraction, are achieved by having pgstat_report_wait_start/end() not interact with MyProc, but instead a new my_wait_event_info variable. At backend startup it points to a local variable, removing the need to check for MyProc being NULL. During process initialization my_wait_event_info is redirected to MyProc->wait_event_info. At shutdown this is reversed. Because wait event reporting now does not need to know about where the wait event is stored, it does not need to know about PGPROC anymore. The removal of the branch for checking pgstat_track_activities is simpler: Don't check anymore. The cost due to the branch are often higher than the store - and even if not, pgstat_track_activities is rarely disabled. The main motivator to commit this work now is that removing the (indirect) pgproc.h include from pgstat.h simplifies a patch to move statistics reporting to shared memory (which still has a chance to get into 14). Author: Andres Freund <andres@anarazel.de> Discussion: https://postgr.es/m/20210402194458.2vu324hkk2djq6ce@alap3.anarazel.de
2021-04-03 20:44:47 +02:00
/* now that we have a proc, report wait events to shared memory */
pgstat_set_wait_event_storage(&MyProc->wait_event_info);
/* Check that group locking fields are in a proper initial state. */
Assert(MyProc->lockGroupLeader == NULL);
Assert(dlist_is_empty(&MyProc->lockGroupMembers));
/*
* We might be reusing a semaphore that belonged to a failed process. So
* be careful and reinitialize its value here. (This is not strictly
* necessary anymore, but seems like a good idea for cleanliness.)
*/
Make the different Unix-y semaphore implementations ABI-compatible. Previously, the "sem" field of PGPROC varied in size depending on which kernel semaphore API we were using. That was okay as long as there was only one likely choice per platform, but in the wake of commit ecb0d20a9, that assumption seems rather shaky. It doesn't seem out of the question anymore that an extension compiled against one API choice might be loaded into a postmaster built with another choice. Moreover, this prevents any possibility of selecting the semaphore API at postmaster startup, which might be something we want to do in future. Hence, change PGPROC.sem to be PGSemaphore (i.e. a pointer) for all Unix semaphore APIs, and turn the pointed-to data into an opaque struct whose contents are only known within the responsible modules. For the SysV and unnamed-POSIX APIs, the pointed-to data has to be allocated elsewhere in shared memory, which takes a little bit of rejiggering of the InitShmemAllocation code sequence. (I invented a ShmemAllocUnlocked() function to make that a little cleaner than it used to be. That function is not meant for any uses other than the ones it has now, but it beats having InitShmemAllocation() know explicitly about allocation of space for semaphores and spinlocks.) This change means an extra indirection to access the semaphore data, but since we only touch that when blocking or awakening a process, there shouldn't be any meaningful performance penalty. Moreover, at least for the unnamed-POSIX case on Linux, the sem_t type is quite a bit wider than a pointer, so this reduces sizeof(PGPROC) which seems like a good thing. For the named-POSIX API, there's effectively no change: the PGPROC.sem field was and still is a pointer to something returned by sem_open() in the postmaster's memory space. Document and check the pre-existing limitation that this case can't work in EXEC_BACKEND mode. It did not seem worth unifying the Windows semaphore ABI with the Unix cases, since there's no likelihood of needing ABI compatibility much less runtime switching across those cases. However, we can simplify the Windows code a bit if we define PGSemaphore as being directly a HANDLE, rather than pointer to HANDLE, so let's do that while we're here. (This also ends up being no change in what's physically stored in PGPROC.sem. We're just moving the HANDLE fetch from callees to callers.) It would take a bunch of additional code shuffling to get to the point of actually choosing a semaphore API at postmaster start, but the effects of that would now be localized in the port/XXX_sema.c files, so it seems like fit material for a separate patch. The need for it is unproven as yet, anyhow, whereas the ABI risk to extensions seems real enough. Discussion: https://postgr.es/m/4029.1481413370@sss.pgh.pa.us
2016-12-12 19:32:10 +01:00
PGSemaphoreReset(MyProc->sem);
/*
* Arrange to clean up at process exit.
*/
on_shmem_exit(AuxiliaryProcKill, Int32GetDatum(proctype));
/*
* Now that we have a PGPROC, we could try to acquire lightweight locks.
* Initialize local state needed for them. (Heavyweight locks cannot be
* acquired in aux processes.)
*/
InitLWLockAccess();
#ifdef EXEC_BACKEND
/*
* Initialize backend-local pointers to all the shared data structures.
* (We couldn't do this until now because it needs LWLocks.)
*/
if (IsUnderPostmaster)
AttachSharedMemoryStructs();
#endif
}
/*
* Used from bufmgr to share the value of the buffer that Startup waits on,
* or to reset the value to "not waiting" (-1). This allows processing
* of recovery conflicts for buffer pins. Set is made before backends look
* at this value, so locking not required, especially since the set is
* an atomic integer set operation.
*/
void
SetStartupBufferPinWaitBufId(int bufid)
{
/* use volatile pointer to prevent code rearrangement */
volatile PROC_HDR *procglobal = ProcGlobal;
procglobal->startupBufferPinWaitBufId = bufid;
}
/*
* Used by backends when they receive a request to check for buffer pin waits.
*/
int
GetStartupBufferPinWaitBufId(void)
{
/* use volatile pointer to prevent code rearrangement */
volatile PROC_HDR *procglobal = ProcGlobal;
return procglobal->startupBufferPinWaitBufId;
}
/*
* Check whether there are at least N free PGPROC objects. If false is
* returned, *nfree will be set to the number of free PGPROC objects.
* Otherwise, *nfree will be set to n.
*
* Note: this is designed on the assumption that N will generally be small.
*/
bool
HaveNFreeProcs(int n, int *nfree)
{
dlist_iter iter;
2005-10-15 04:49:52 +02:00
Assert(n > 0);
Assert(nfree);
SpinLockAcquire(ProcStructLock);
*nfree = 0;
dlist_foreach(iter, &ProcGlobal->freeProcs)
{
(*nfree)++;
if (*nfree == n)
break;
}
SpinLockRelease(ProcStructLock);
return (*nfree == n);
}
Improve control logic for bgwriter hibernation mode. Commit 6d90eaaa89a007e0d365f49d6436f35d2392cfeb added a hibernation mode to the bgwriter to reduce the server's idle-power consumption. However, its interaction with the detailed behavior of BgBufferSync's feedback control loop wasn't very well thought out. That control loop depends primarily on the rate of buffer allocation, not the rate of buffer dirtying, so the hibernation mode has to be designed to operate only when no new buffer allocations are happening. Also, the check for whether the system is effectively idle was not quite right and would fail to detect a constant low level of activity, thus allowing the bgwriter to go into hibernation mode in a way that would let the cycle time vary quite a bit, possibly further confusing the feedback loop. To fix, move the wakeup support from MarkBufferDirty and SetBufferCommitInfoNeedsSave into StrategyGetBuffer, and prevent the bgwriter from entering hibernation mode unless no buffer allocations have happened recently. In addition, fix the delaying logic to remove the problem of possibly not responding to signals promptly, which was basically caused by trying to use the process latch's is_set flag for multiple purposes. I can't prove it but I'm suspicious that that hack was responsible for the intermittent "postmaster does not shut down" failures we've been seeing in the buildfarm lately. In any case it did nothing to improve the readability or robustness of the code. In passing, express the hibernation sleep time as a multiplier on BgWriterDelay, not a constant. I'm not sure whether there's any value in exposing the longer sleep time as an independently configurable setting, but we can at least make it act like this for little extra code.
2012-05-10 05:36:01 +02:00
/*
* Check if the current process is awaiting a lock.
*/
bool
IsWaitingForLock(void)
{
if (lockAwaited == NULL)
return false;
return true;
}
/*
* Cancel any pending wait for lock, when aborting a transaction, and revert
* any strong lock count acquisition for a lock being acquired.
*
* (Normally, this would only happen if we accept a cancel/die
* interrupt while waiting; but an ereport(ERROR) before or during the lock
* wait is within the realm of possibility, too.)
*/
void
LockErrorCleanup(void)
{
LWLock *partitionLock;
DisableTimeoutParams timeouts[2];
Be more careful to not lose sync in the FE/BE protocol. If any error occurred while we were in the middle of reading a protocol message from the client, we could lose sync, and incorrectly try to interpret a part of another message as a new protocol message. That will usually lead to an "invalid frontend message" error that terminates the connection. However, this is a security issue because an attacker might be able to deliberately cause an error, inject a Query message in what's supposed to be just user data, and have the server execute it. We were quite careful to not have CHECK_FOR_INTERRUPTS() calls or other operations that could ereport(ERROR) in the middle of processing a message, but a query cancel interrupt or statement timeout could nevertheless cause it to happen. Also, the V2 fastpath and COPY handling were not so careful. It's very difficult to recover in the V2 COPY protocol, so we will just terminate the connection on error. In practice, that's what happened previously anyway, as we lost protocol sync. To fix, add a new variable in pqcomm.c, PqCommReadingMsg, that is set whenever we're in the middle of reading a message. When it's set, we cannot safely ERROR out and continue running, because we might've read only part of a message. PqCommReadingMsg acts somewhat similarly to critical sections in that if an error occurs while it's set, the error handler will force the connection to be terminated, as if the error was FATAL. It's not implemented by promoting ERROR to FATAL in elog.c, like ERROR is promoted to PANIC in critical sections, because we want to be able to use PG_TRY/CATCH to recover and regain protocol sync. pq_getmessage() takes advantage of that to prevent an OOM error from terminating the connection. To prevent unnecessary connection terminations, add a holdoff mechanism similar to HOLD/RESUME_INTERRUPTS() that can be used hold off query cancel interrupts, but still allow die interrupts. The rules on which interrupts are processed when are now a bit more complicated, so refactor ProcessInterrupts() and the calls to it in signal handlers so that the signal handlers always call it if ImmediateInterruptOK is set, and ProcessInterrupts() can decide to not do anything if the other conditions are not met. Reported by Emil Lenngren. Patch reviewed by Noah Misch and Andres Freund. Backpatch to all supported versions. Security: CVE-2015-0244
2015-02-02 16:08:45 +01:00
HOLD_INTERRUPTS();
AbortStrongLockAcquire();
/* Nothing to do if we weren't waiting for a lock */
if (lockAwaited == NULL)
Be more careful to not lose sync in the FE/BE protocol. If any error occurred while we were in the middle of reading a protocol message from the client, we could lose sync, and incorrectly try to interpret a part of another message as a new protocol message. That will usually lead to an "invalid frontend message" error that terminates the connection. However, this is a security issue because an attacker might be able to deliberately cause an error, inject a Query message in what's supposed to be just user data, and have the server execute it. We were quite careful to not have CHECK_FOR_INTERRUPTS() calls or other operations that could ereport(ERROR) in the middle of processing a message, but a query cancel interrupt or statement timeout could nevertheless cause it to happen. Also, the V2 fastpath and COPY handling were not so careful. It's very difficult to recover in the V2 COPY protocol, so we will just terminate the connection on error. In practice, that's what happened previously anyway, as we lost protocol sync. To fix, add a new variable in pqcomm.c, PqCommReadingMsg, that is set whenever we're in the middle of reading a message. When it's set, we cannot safely ERROR out and continue running, because we might've read only part of a message. PqCommReadingMsg acts somewhat similarly to critical sections in that if an error occurs while it's set, the error handler will force the connection to be terminated, as if the error was FATAL. It's not implemented by promoting ERROR to FATAL in elog.c, like ERROR is promoted to PANIC in critical sections, because we want to be able to use PG_TRY/CATCH to recover and regain protocol sync. pq_getmessage() takes advantage of that to prevent an OOM error from terminating the connection. To prevent unnecessary connection terminations, add a holdoff mechanism similar to HOLD/RESUME_INTERRUPTS() that can be used hold off query cancel interrupts, but still allow die interrupts. The rules on which interrupts are processed when are now a bit more complicated, so refactor ProcessInterrupts() and the calls to it in signal handlers so that the signal handlers always call it if ImmediateInterruptOK is set, and ProcessInterrupts() can decide to not do anything if the other conditions are not met. Reported by Emil Lenngren. Patch reviewed by Noah Misch and Andres Freund. Backpatch to all supported versions. Security: CVE-2015-0244
2015-02-02 16:08:45 +01:00
{
RESUME_INTERRUPTS();
return;
Be more careful to not lose sync in the FE/BE protocol. If any error occurred while we were in the middle of reading a protocol message from the client, we could lose sync, and incorrectly try to interpret a part of another message as a new protocol message. That will usually lead to an "invalid frontend message" error that terminates the connection. However, this is a security issue because an attacker might be able to deliberately cause an error, inject a Query message in what's supposed to be just user data, and have the server execute it. We were quite careful to not have CHECK_FOR_INTERRUPTS() calls or other operations that could ereport(ERROR) in the middle of processing a message, but a query cancel interrupt or statement timeout could nevertheless cause it to happen. Also, the V2 fastpath and COPY handling were not so careful. It's very difficult to recover in the V2 COPY protocol, so we will just terminate the connection on error. In practice, that's what happened previously anyway, as we lost protocol sync. To fix, add a new variable in pqcomm.c, PqCommReadingMsg, that is set whenever we're in the middle of reading a message. When it's set, we cannot safely ERROR out and continue running, because we might've read only part of a message. PqCommReadingMsg acts somewhat similarly to critical sections in that if an error occurs while it's set, the error handler will force the connection to be terminated, as if the error was FATAL. It's not implemented by promoting ERROR to FATAL in elog.c, like ERROR is promoted to PANIC in critical sections, because we want to be able to use PG_TRY/CATCH to recover and regain protocol sync. pq_getmessage() takes advantage of that to prevent an OOM error from terminating the connection. To prevent unnecessary connection terminations, add a holdoff mechanism similar to HOLD/RESUME_INTERRUPTS() that can be used hold off query cancel interrupts, but still allow die interrupts. The rules on which interrupts are processed when are now a bit more complicated, so refactor ProcessInterrupts() and the calls to it in signal handlers so that the signal handlers always call it if ImmediateInterruptOK is set, and ProcessInterrupts() can decide to not do anything if the other conditions are not met. Reported by Emil Lenngren. Patch reviewed by Noah Misch and Andres Freund. Backpatch to all supported versions. Security: CVE-2015-0244
2015-02-02 16:08:45 +01:00
}
/*
* Turn off the deadlock and lock timeout timers, if they are still
* running (see ProcSleep). Note we must preserve the LOCK_TIMEOUT
* indicator flag, since this function is executed before
* ProcessInterrupts when responding to SIGINT; else we'd lose the
* knowledge that the SIGINT came from a lock timeout and not an external
* source.
*/
timeouts[0].id = DEADLOCK_TIMEOUT;
timeouts[0].keep_indicator = false;
timeouts[1].id = LOCK_TIMEOUT;
timeouts[1].keep_indicator = true;
disable_timeouts(timeouts, 2);
/* Unlink myself from the wait queue, if on it (might not be anymore!) */
partitionLock = LockHashPartitionLock(lockAwaited->hashcode);
LWLockAcquire(partitionLock, LW_EXCLUSIVE);
if (!dlist_node_is_detached(&MyProc->links))
{
/* We could not have been granted the lock yet */
RemoveFromWaitQueue(MyProc, lockAwaited->hashcode);
}
else
{
/*
* Somebody kicked us off the lock queue already. Perhaps they
* granted us the lock, or perhaps they detected a deadlock. If they
* did grant us the lock, we'd better remember it in our local lock
* table.
*/
if (MyProc->waitStatus == PROC_WAIT_STATUS_OK)
GrantAwaitedLock();
}
lockAwaited = NULL;
LWLockRelease(partitionLock);
Be more careful to not lose sync in the FE/BE protocol. If any error occurred while we were in the middle of reading a protocol message from the client, we could lose sync, and incorrectly try to interpret a part of another message as a new protocol message. That will usually lead to an "invalid frontend message" error that terminates the connection. However, this is a security issue because an attacker might be able to deliberately cause an error, inject a Query message in what's supposed to be just user data, and have the server execute it. We were quite careful to not have CHECK_FOR_INTERRUPTS() calls or other operations that could ereport(ERROR) in the middle of processing a message, but a query cancel interrupt or statement timeout could nevertheless cause it to happen. Also, the V2 fastpath and COPY handling were not so careful. It's very difficult to recover in the V2 COPY protocol, so we will just terminate the connection on error. In practice, that's what happened previously anyway, as we lost protocol sync. To fix, add a new variable in pqcomm.c, PqCommReadingMsg, that is set whenever we're in the middle of reading a message. When it's set, we cannot safely ERROR out and continue running, because we might've read only part of a message. PqCommReadingMsg acts somewhat similarly to critical sections in that if an error occurs while it's set, the error handler will force the connection to be terminated, as if the error was FATAL. It's not implemented by promoting ERROR to FATAL in elog.c, like ERROR is promoted to PANIC in critical sections, because we want to be able to use PG_TRY/CATCH to recover and regain protocol sync. pq_getmessage() takes advantage of that to prevent an OOM error from terminating the connection. To prevent unnecessary connection terminations, add a holdoff mechanism similar to HOLD/RESUME_INTERRUPTS() that can be used hold off query cancel interrupts, but still allow die interrupts. The rules on which interrupts are processed when are now a bit more complicated, so refactor ProcessInterrupts() and the calls to it in signal handlers so that the signal handlers always call it if ImmediateInterruptOK is set, and ProcessInterrupts() can decide to not do anything if the other conditions are not met. Reported by Emil Lenngren. Patch reviewed by Noah Misch and Andres Freund. Backpatch to all supported versions. Security: CVE-2015-0244
2015-02-02 16:08:45 +01:00
RESUME_INTERRUPTS();
}
/*
* ProcReleaseLocks() -- release locks associated with current transaction
* at main transaction commit or abort
*
2012-05-04 23:43:27 +02:00
* At main transaction commit, we release standard locks except session locks.
* At main transaction abort, we release all locks including session locks.
*
2012-05-04 23:43:27 +02:00
* Advisory locks are released only if they are transaction-level;
* session-level holds remain, whether this is a commit or not.
*
* At subtransaction commit, we don't release any locks (so this func is not
* needed at all); we will defer the releasing to the parent transaction.
* At subtransaction abort, we release all locks held by the subtransaction;
* this is implemented by retail releasing of the locks under control of
* the ResourceOwner mechanism.
*/
void
ProcReleaseLocks(bool isCommit)
{
if (!MyProc)
return;
/* If waiting, get off wait queue (should only be needed after error) */
LockErrorCleanup();
2012-05-04 23:43:27 +02:00
/* Release standard locks, including session-level if aborting */
LockReleaseAll(DEFAULT_LOCKMETHOD, !isCommit);
2012-05-04 23:43:27 +02:00
/* Release transaction-level advisory locks */
LockReleaseAll(USER_LOCKMETHOD, false);
}
/*
* RemoveProcFromArray() -- Remove this process from the shared ProcArray.
*/
static void
RemoveProcFromArray(int code, Datum arg)
{
Assert(MyProc != NULL);
ProcArrayRemove(MyProc, InvalidTransactionId);
}
/*
* ProcKill() -- Destroy the per-proc data structure for
* this process. Release any of its held LW locks.
*/
static void
ProcKill(int code, Datum arg)
{
PGPROC *proc;
dlist_head *procgloballist;
Assert(MyProc != NULL);
/* not safe if forked by system(), etc. */
if (MyProc->pid != (int) getpid())
elog(PANIC, "ProcKill() called in child process");
/* Make sure we're out of the sync rep lists */
SyncRepCleanupAtProcExit();
#ifdef USE_ASSERT_CHECKING
{
int i;
/* Last process should have released all locks. */
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
Assert(dlist_is_empty(&(MyProc->myProcLocks[i])));
}
#endif
/*
* Release any LW locks I am holding. There really shouldn't be any, but
* it's cheap to check again before we cut the knees off the LWLock
* facility by releasing our PGPROC ...
*/
LWLockReleaseAll();
/*
* Cleanup waiting for LSN if any.
*/
WaitLSNCleanup();
/* Cancel any pending condition variable sleep, too */
ConditionVariableCancelSleep();
/*
* Detach from any lock group of which we are a member. If the leader
* exits before all other group members, its PGPROC will remain allocated
* until the last group process exits; that process must return the
* leader's PGPROC to the appropriate list.
*/
if (MyProc->lockGroupLeader != NULL)
{
PGPROC *leader = MyProc->lockGroupLeader;
LWLock *leader_lwlock = LockHashPartitionLockByProc(leader);
LWLockAcquire(leader_lwlock, LW_EXCLUSIVE);
Assert(!dlist_is_empty(&leader->lockGroupMembers));
dlist_delete(&MyProc->lockGroupLink);
if (dlist_is_empty(&leader->lockGroupMembers))
{
leader->lockGroupLeader = NULL;
if (leader != MyProc)
{
procgloballist = leader->procgloballist;
/* Leader exited first; return its PGPROC. */
SpinLockAcquire(ProcStructLock);
dlist_push_head(procgloballist, &leader->links);
SpinLockRelease(ProcStructLock);
}
}
else if (leader != MyProc)
MyProc->lockGroupLeader = NULL;
LWLockRelease(leader_lwlock);
}
/*
* Reset MyLatch to the process local one. This is so that signal
* handlers et al can continue using the latch after the shared latch
Improve efficiency of wait event reporting, remove proc.h dependency. pgstat_report_wait_start() and pgstat_report_wait_end() required two conditional branches so far. One to check if MyProc is NULL, the other to check if pgstat_track_activities is set. As wait events are used around comparatively lightweight operations, and are inlined (reducing branch predictor effectiveness), that's not great. The dependency on MyProc has a second disadvantage: Low-level subsystems, like storage/file/fd.c, report wait events, but architecturally it is preferable for them to not depend on inter-process subsystems like proc.h (defining PGPROC). After this change including pgstat.h (nor obviously its sub-components like backend_status.h, wait_event.h, ...) does not pull in IPC related headers anymore. These goals, efficiency and abstraction, are achieved by having pgstat_report_wait_start/end() not interact with MyProc, but instead a new my_wait_event_info variable. At backend startup it points to a local variable, removing the need to check for MyProc being NULL. During process initialization my_wait_event_info is redirected to MyProc->wait_event_info. At shutdown this is reversed. Because wait event reporting now does not need to know about where the wait event is stored, it does not need to know about PGPROC anymore. The removal of the branch for checking pgstat_track_activities is simpler: Don't check anymore. The cost due to the branch are often higher than the store - and even if not, pgstat_track_activities is rarely disabled. The main motivator to commit this work now is that removing the (indirect) pgproc.h include from pgstat.h simplifies a patch to move statistics reporting to shared memory (which still has a chance to get into 14). Author: Andres Freund <andres@anarazel.de> Discussion: https://postgr.es/m/20210402194458.2vu324hkk2djq6ce@alap3.anarazel.de
2021-04-03 20:44:47 +02:00
* isn't ours anymore.
*
* Similarly, stop reporting wait events to MyProc->wait_event_info.
*
* After that clear MyProc and disown the shared latch.
*/
SwitchBackToLocalLatch();
Improve efficiency of wait event reporting, remove proc.h dependency. pgstat_report_wait_start() and pgstat_report_wait_end() required two conditional branches so far. One to check if MyProc is NULL, the other to check if pgstat_track_activities is set. As wait events are used around comparatively lightweight operations, and are inlined (reducing branch predictor effectiveness), that's not great. The dependency on MyProc has a second disadvantage: Low-level subsystems, like storage/file/fd.c, report wait events, but architecturally it is preferable for them to not depend on inter-process subsystems like proc.h (defining PGPROC). After this change including pgstat.h (nor obviously its sub-components like backend_status.h, wait_event.h, ...) does not pull in IPC related headers anymore. These goals, efficiency and abstraction, are achieved by having pgstat_report_wait_start/end() not interact with MyProc, but instead a new my_wait_event_info variable. At backend startup it points to a local variable, removing the need to check for MyProc being NULL. During process initialization my_wait_event_info is redirected to MyProc->wait_event_info. At shutdown this is reversed. Because wait event reporting now does not need to know about where the wait event is stored, it does not need to know about PGPROC anymore. The removal of the branch for checking pgstat_track_activities is simpler: Don't check anymore. The cost due to the branch are often higher than the store - and even if not, pgstat_track_activities is rarely disabled. The main motivator to commit this work now is that removing the (indirect) pgproc.h include from pgstat.h simplifies a patch to move statistics reporting to shared memory (which still has a chance to get into 14). Author: Andres Freund <andres@anarazel.de> Discussion: https://postgr.es/m/20210402194458.2vu324hkk2djq6ce@alap3.anarazel.de
2021-04-03 20:44:47 +02:00
pgstat_reset_wait_event_storage();
proc = MyProc;
MyProc = NULL;
MyProcNumber = INVALID_PROC_NUMBER;
DisownLatch(&proc->procLatch);
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
/* Mark the proc no longer in use */
proc->pid = 0;
proc->vxid.procNumber = INVALID_PROC_NUMBER;
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
proc->vxid.lxid = InvalidTransactionId;
procgloballist = proc->procgloballist;
SpinLockAcquire(ProcStructLock);
/*
* If we're still a member of a locking group, that means we're a leader
* which has somehow exited before its children. The last remaining child
* will release our PGPROC. Otherwise, release it now.
*/
if (proc->lockGroupLeader == NULL)
{
/* Since lockGroupLeader is NULL, lockGroupMembers should be empty. */
Assert(dlist_is_empty(&proc->lockGroupMembers));
/* Return PGPROC structure (and semaphore) to appropriate freelist */
dlist_push_tail(procgloballist, &proc->links);
}
/* Update shared estimate of spins_per_delay */
ProcGlobal->spins_per_delay = update_spins_per_delay(ProcGlobal->spins_per_delay);
SpinLockRelease(ProcStructLock);
/*
* This process is no longer present in shared memory in any meaningful
* way, so tell the postmaster we've cleaned up acceptably well. (XXX
* autovac launcher should be included here someday)
*
* Slot sync worker is also not a postmaster child, so skip this shared
* memory related processing here.
*/
if (IsUnderPostmaster && !AmAutoVacuumLauncherProcess() &&
!AmLogicalSlotSyncWorkerProcess())
MarkPostmasterChildInactive();
/* wake autovac launcher if needed -- see comments in FreeWorkerInfo */
if (AutovacuumLauncherPid != 0)
kill(AutovacuumLauncherPid, SIGUSR2);
}
/*
* AuxiliaryProcKill() -- Cut-down version of ProcKill for auxiliary
* processes (bgwriter, etc). The PGPROC and sema are not released, only
* marked as not-in-use.
*/
static void
AuxiliaryProcKill(int code, Datum arg)
{
int proctype = DatumGetInt32(arg);
PGPROC *auxproc PG_USED_FOR_ASSERTS_ONLY;
PGPROC *proc;
Assert(proctype >= 0 && proctype < NUM_AUXILIARY_PROCS);
/* not safe if forked by system(), etc. */
if (MyProc->pid != (int) getpid())
elog(PANIC, "AuxiliaryProcKill() called in child process");
auxproc = &AuxiliaryProcs[proctype];
Assert(MyProc == auxproc);
/* Release any LW locks I am holding (see notes above) */
LWLockReleaseAll();
/* Cancel any pending condition variable sleep, too */
ConditionVariableCancelSleep();
Improve efficiency of wait event reporting, remove proc.h dependency. pgstat_report_wait_start() and pgstat_report_wait_end() required two conditional branches so far. One to check if MyProc is NULL, the other to check if pgstat_track_activities is set. As wait events are used around comparatively lightweight operations, and are inlined (reducing branch predictor effectiveness), that's not great. The dependency on MyProc has a second disadvantage: Low-level subsystems, like storage/file/fd.c, report wait events, but architecturally it is preferable for them to not depend on inter-process subsystems like proc.h (defining PGPROC). After this change including pgstat.h (nor obviously its sub-components like backend_status.h, wait_event.h, ...) does not pull in IPC related headers anymore. These goals, efficiency and abstraction, are achieved by having pgstat_report_wait_start/end() not interact with MyProc, but instead a new my_wait_event_info variable. At backend startup it points to a local variable, removing the need to check for MyProc being NULL. During process initialization my_wait_event_info is redirected to MyProc->wait_event_info. At shutdown this is reversed. Because wait event reporting now does not need to know about where the wait event is stored, it does not need to know about PGPROC anymore. The removal of the branch for checking pgstat_track_activities is simpler: Don't check anymore. The cost due to the branch are often higher than the store - and even if not, pgstat_track_activities is rarely disabled. The main motivator to commit this work now is that removing the (indirect) pgproc.h include from pgstat.h simplifies a patch to move statistics reporting to shared memory (which still has a chance to get into 14). Author: Andres Freund <andres@anarazel.de> Discussion: https://postgr.es/m/20210402194458.2vu324hkk2djq6ce@alap3.anarazel.de
2021-04-03 20:44:47 +02:00
/* look at the equivalent ProcKill() code for comments */
SwitchBackToLocalLatch();
Improve efficiency of wait event reporting, remove proc.h dependency. pgstat_report_wait_start() and pgstat_report_wait_end() required two conditional branches so far. One to check if MyProc is NULL, the other to check if pgstat_track_activities is set. As wait events are used around comparatively lightweight operations, and are inlined (reducing branch predictor effectiveness), that's not great. The dependency on MyProc has a second disadvantage: Low-level subsystems, like storage/file/fd.c, report wait events, but architecturally it is preferable for them to not depend on inter-process subsystems like proc.h (defining PGPROC). After this change including pgstat.h (nor obviously its sub-components like backend_status.h, wait_event.h, ...) does not pull in IPC related headers anymore. These goals, efficiency and abstraction, are achieved by having pgstat_report_wait_start/end() not interact with MyProc, but instead a new my_wait_event_info variable. At backend startup it points to a local variable, removing the need to check for MyProc being NULL. During process initialization my_wait_event_info is redirected to MyProc->wait_event_info. At shutdown this is reversed. Because wait event reporting now does not need to know about where the wait event is stored, it does not need to know about PGPROC anymore. The removal of the branch for checking pgstat_track_activities is simpler: Don't check anymore. The cost due to the branch are often higher than the store - and even if not, pgstat_track_activities is rarely disabled. The main motivator to commit this work now is that removing the (indirect) pgproc.h include from pgstat.h simplifies a patch to move statistics reporting to shared memory (which still has a chance to get into 14). Author: Andres Freund <andres@anarazel.de> Discussion: https://postgr.es/m/20210402194458.2vu324hkk2djq6ce@alap3.anarazel.de
2021-04-03 20:44:47 +02:00
pgstat_reset_wait_event_storage();
proc = MyProc;
MyProc = NULL;
MyProcNumber = INVALID_PROC_NUMBER;
DisownLatch(&proc->procLatch);
SpinLockAcquire(ProcStructLock);
/* Mark auxiliary proc no longer in use */
proc->pid = 0;
proc->vxid.procNumber = INVALID_PROC_NUMBER;
Redefine backend ID to be an index into the proc array Previously, backend ID was an index into the ProcState array, in the shared cache invalidation manager (sinvaladt.c). The entry in the ProcState array was reserved at backend startup by scanning the array for a free entry, and that was also when the backend got its backend ID. Things become slightly simpler if we redefine backend ID to be the index into the PGPROC array, and directly use it also as an index to the ProcState array. This uses a little more memory, as we reserve a few extra slots in the ProcState array for aux processes that don't need them, but the simplicity is worth it. Aux processes now also have a backend ID. This simplifies the reservation of BackendStatusArray and ProcSignal slots. You can now convert a backend ID into an index into the PGPROC array simply by subtracting 1. We still use 0-based "pgprocnos" in various places, for indexes into the PGPROC array, but the only difference now is that backend IDs start at 1 while pgprocnos start at 0. (The next commmit will get rid of the term "backend ID" altogether and make everything 0-based.) There is still a 'backendId' field in PGPROC, now part of 'vxid' which encapsulates the backend ID and local transaction ID together. It's needed for prepared xacts. For regular backends, the backendId is always equal to pgprocno + 1, but for prepared xact PGPROC entries, it's the ID of the original backend that processed the transaction. Reviewed-by: Andres Freund, Reid Thompson Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
2024-03-03 18:37:28 +01:00
proc->vxid.lxid = InvalidTransactionId;
/* Update shared estimate of spins_per_delay */
ProcGlobal->spins_per_delay = update_spins_per_delay(ProcGlobal->spins_per_delay);
SpinLockRelease(ProcStructLock);
}
/*
* AuxiliaryPidGetProc -- get PGPROC for an auxiliary process
* given its PID
*
* Returns NULL if not found.
*/
PGPROC *
AuxiliaryPidGetProc(int pid)
{
PGPROC *result = NULL;
int index;
if (pid == 0) /* never match dummy PGPROCs */
return NULL;
for (index = 0; index < NUM_AUXILIARY_PROCS; index++)
{
PGPROC *proc = &AuxiliaryProcs[index];
if (proc->pid == pid)
{
result = proc;
break;
}
}
return result;
}
/*
* ProcSleep -- put a process to sleep on the specified lock
*
* Caller must have set MyProc->heldLocks to reflect locks already held
* on the lockable object by this process (under all XIDs).
*
* It's not actually guaranteed that we need to wait when this function is
* called, because it could be that when we try to find a position at which
* to insert ourself into the wait queue, we discover that we must be inserted
* ahead of everyone who wants a lock that conflict with ours. In that case,
* we get the lock immediately. Beause of this, it's sensible for this function
* to have a dontWait argument, despite the name.
*
* The lock table's partition lock must be held at entry, and will be held
* at exit.
*
* Result: PROC_WAIT_STATUS_OK if we acquired the lock, PROC_WAIT_STATUS_ERROR
* if not (if dontWait = true, this is a deadlock; if dontWait = false, we
* would have had to wait).
*
* ASSUME: that no one will fiddle with the queue until after
* we release the partition lock.
*
* NOTES: The process queue is now a priority queue for locking.
*/
ProcWaitStatus
ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable, bool dontWait)
{
LOCKMODE lockmode = locallock->tag.mode;
LOCK *lock = locallock->lock;
PROCLOCK *proclock = locallock->proclock;
uint32 hashcode = locallock->hashcode;
LWLock *partitionLock = LockHashPartitionLock(hashcode);
dclist_head *waitQueue = &lock->waitProcs;
PGPROC *insert_before = NULL;
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
LOCKMASK myHeldLocks = MyProc->heldLocks;
TimestampTz standbyWaitStart = 0;
bool early_deadlock = false;
bool allow_autovacuum_cancel = true;
bool logged_recovery_conflict = false;
ProcWaitStatus myWaitStatus;
PGPROC *leader = MyProc->lockGroupLeader;
/*
* If group locking is in use, locks held by members of my locking group
* need to be included in myHeldLocks. This is not required for relation
* extension lock which conflict among group members. However, including
* them in myHeldLocks will give group members the priority to get those
* locks as compared to other backends which are also trying to acquire
* those locks. OTOH, we can avoid giving priority to group members for
* that kind of locks, but there doesn't appear to be a clear advantage of
* the same.
*/
if (leader != NULL)
{
dlist_iter iter;
dlist_foreach(iter, &lock->procLocks)
{
PROCLOCK *otherproclock;
otherproclock = dlist_container(PROCLOCK, lockLink, iter.cur);
if (otherproclock->groupLeader == leader)
myHeldLocks |= otherproclock->holdMask;
}
}
/*
* Determine where to add myself in the wait queue.
*
* Normally I should go at the end of the queue. However, if I already
* hold locks that conflict with the request of any previous waiter, put
* myself in the queue just in front of the first such waiter. This is not
* a necessary step, since deadlock detection would move me to before that
* waiter anyway; but it's relatively cheap to detect such a conflict
* immediately, and avoid delaying till deadlock timeout.
*
* Special case: if I find I should go in front of some waiter, check to
* see if I conflict with already-held locks or the requests before that
* waiter. If not, then just grant myself the requested lock immediately.
* This is the same as the test for immediate grant in LockAcquire, except
* we are only considering the part of the wait queue before my insertion
* point.
*/
if (myHeldLocks != 0 && !dclist_is_empty(waitQueue))
{
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
LOCKMASK aheadRequests = 0;
dlist_iter iter;
dclist_foreach(iter, waitQueue)
{
PGPROC *proc = dlist_container(PGPROC, links, iter.cur);
/*
* If we're part of the same locking group as this waiter, its
* locks neither conflict with ours nor contribute to
* aheadRequests.
*/
if (leader != NULL && leader == proc->lockGroupLeader)
continue;
/* Must he wait for me? */
if (lockMethodTable->conflictTab[proc->waitLockMode] & myHeldLocks)
{
/* Must I wait for him ? */
if (lockMethodTable->conflictTab[lockmode] & proc->heldLocks)
{
/*
* Yes, so we have a deadlock. Easiest way to clean up
* correctly is to call RemoveFromWaitQueue(), but we
* can't do that until we are *on* the wait queue. So, set
* a flag to check below, and break out of loop. Also,
* record deadlock info for later message.
*/
RememberSimpleDeadLock(MyProc, lockmode, lock, proc);
early_deadlock = true;
break;
}
/* I must go before this waiter. Check special case. */
if ((lockMethodTable->conflictTab[lockmode] & aheadRequests) == 0 &&
!LockCheckConflicts(lockMethodTable, lockmode, lock,
proclock))
{
/* Skip the wait and just grant myself the lock. */
GrantLock(lock, proclock, lockmode);
GrantAwaitedLock();
return PROC_WAIT_STATUS_OK;
}
/* Put myself into wait queue before conflicting process */
insert_before = proc;
break;
}
/* Nope, so advance to next waiter */
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
aheadRequests |= LOCKBIT_ON(proc->waitLockMode);
}
}
/*
* At this point we know that we'd really need to sleep. If we've been
* commanded not to do that, bail out.
*/
if (dontWait)
return PROC_WAIT_STATUS_ERROR;
/*
* Insert self into queue, at the position determined above.
*/
if (insert_before)
dclist_insert_before(waitQueue, &insert_before->links, &MyProc->links);
else
dclist_push_tail(waitQueue, &MyProc->links);
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
lock->waitMask |= LOCKBIT_ON(lockmode);
/* Set up wait information in PGPROC object, too */
MyProc->waitLock = lock;
MyProc->waitProcLock = proclock;
MyProc->waitLockMode = lockmode;
MyProc->waitStatus = PROC_WAIT_STATUS_WAITING;
/*
* If we detected deadlock, give up without waiting. This must agree with
* CheckDeadLock's recovery code.
*/
if (early_deadlock)
{
RemoveFromWaitQueue(MyProc, hashcode);
return PROC_WAIT_STATUS_ERROR;
}
/* mark that we are waiting for a lock */
lockAwaited = locallock;
/*
* Release the lock table's partition lock.
*
* NOTE: this may also cause us to exit critical-section state, possibly
* allowing a cancel/die interrupt to be accepted. This is OK because we
* have recorded the fact that we are waiting for a lock, and so
* LockErrorCleanup will clean up if cancel/die happens.
*/
LWLockRelease(partitionLock);
/*
* Also, now that we will successfully clean up after an ereport, it's
* safe to check to see if there's a buffer pin deadlock against the
* Startup process. Of course, that's only necessary if we're doing Hot
* Standby and are not the Startup process ourselves.
*/
if (RecoveryInProgress() && !InRecovery)
CheckRecoveryConflictDeadlock();
/* Reset deadlock_state before enabling the timeout handler */
deadlock_state = DS_NOT_YET_CHECKED;
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
got_deadlock_timeout = false;
/*
* Set timer so we can wake up after awhile and check for a deadlock. If a
* deadlock is detected, the handler sets MyProc->waitStatus =
* PROC_WAIT_STATUS_ERROR, allowing us to know that we must report failure
* rather than success.
*
* By delaying the check until we've waited for a bit, we can avoid
* running the rather expensive deadlock-check code in most cases.
*
* If LockTimeout is set, also enable the timeout for that. We can save a
* few cycles by enabling both timeout sources in one call.
*
* If InHotStandby we set lock waits slightly later for clarity with other
* code.
*/
if (!InHotStandby)
{
if (LockTimeout > 0)
{
EnableTimeoutParams timeouts[2];
timeouts[0].id = DEADLOCK_TIMEOUT;
timeouts[0].type = TMPARAM_AFTER;
timeouts[0].delay_ms = DeadlockTimeout;
timeouts[1].id = LOCK_TIMEOUT;
timeouts[1].type = TMPARAM_AFTER;
timeouts[1].delay_ms = LockTimeout;
enable_timeouts(timeouts, 2);
}
else
enable_timeout_after(DEADLOCK_TIMEOUT, DeadlockTimeout);
Display the time when the process started waiting for the lock, in pg_locks, take 2 This commit adds new column "waitstart" into pg_locks view. This column reports the time when the server process started waiting for the lock if the lock is not held. This information is useful, for example, when examining the amount of time to wait on a lock by subtracting "waitstart" in pg_locks from the current time, and identify the lock that the processes are waiting for very long. This feature uses the current time obtained for the deadlock timeout timer as "waitstart" (i.e., the time when this process started waiting for the lock). Since getting the current time newly can cause overhead, we reuse the already-obtained time to avoid that overhead. Note that "waitstart" is updated without holding the lock table's partition lock, to avoid the overhead by additional lock acquisition. This can cause "waitstart" in pg_locks to become NULL for a very short period of time after the wait started even though "granted" is false. This is OK in practice because we can assume that users are likely to look at "waitstart" when waiting for the lock for a long time. The first attempt of this patch (commit 3b733fcd04) caused the buildfarm member "rorqual" (built with --disable-atomics --disable-spinlocks) to report the failure of the regression test. It was reverted by commit 890d2182a2. The cause of this failure was that the atomic variable for "waitstart" in the dummy process entry created at the end of prepare transaction was not initialized. This second attempt fixes that issue. Bump catalog version. Author: Atsushi Torikoshi Reviewed-by: Ian Lawrence Barwick, Robert Haas, Justin Pryzby, Fujii Masao Discussion: https://postgr.es/m/a96013dc51cdc56b2a2b84fa8a16a993@oss.nttdata.com
2021-02-15 07:13:37 +01:00
/*
* Use the current time obtained for the deadlock timeout timer as
* waitStart (i.e., the time when this process started waiting for the
* lock). Since getting the current time newly can cause overhead, we
* reuse the already-obtained time to avoid that overhead.
*
* Note that waitStart is updated without holding the lock table's
* partition lock, to avoid the overhead by additional lock
* acquisition. This can cause "waitstart" in pg_locks to become NULL
* for a very short period of time after the wait started even though
* "granted" is false. This is OK in practice because we can assume
* that users are likely to look at "waitstart" when waiting for the
* lock for a long time.
*/
pg_atomic_write_u64(&MyProc->waitStart,
get_timeout_start_time(DEADLOCK_TIMEOUT));
}
else if (log_recovery_conflict_waits)
{
/*
* Set the wait start timestamp if logging is enabled and in hot
* standby.
*/
standbyWaitStart = GetCurrentTimestamp();
}
/*
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
* If somebody wakes us between LWLockRelease and WaitLatch, the latch
* will not wait. But a set latch does not necessarily mean that the lock
* is free now, as there are many other sources for latch sets than
* somebody releasing the lock.
*
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
* We process interrupts whenever the latch has been set, so cancel/die
* interrupts are processed quickly. This means we must not mind losing
* control to a cancel/die interrupt here. We don't, because we have no
* shared-state-change work to do after being granted the lock (the
* grantor did it all). We do have to worry about canceling the deadlock
* timeout and updating the locallock table, but if we lose control to an
* error, LockErrorCleanup will fix that up.
*/
do
{
if (InHotStandby)
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
{
bool maybe_log_conflict =
(standbyWaitStart != 0 && !logged_recovery_conflict);
/* Set a timer and wait for that or for the lock to be granted */
ResolveRecoveryConflictWithLock(locallock->tag.lock,
maybe_log_conflict);
/*
* Emit the log message if the startup process is waiting longer
* than deadlock_timeout for recovery conflict on lock.
*/
if (maybe_log_conflict)
{
TimestampTz now = GetCurrentTimestamp();
if (TimestampDifferenceExceeds(standbyWaitStart, now,
DeadlockTimeout))
{
VirtualTransactionId *vxids;
int cnt;
vxids = GetLockConflicts(&locallock->tag.lock,
AccessExclusiveLock, &cnt);
/*
* Log the recovery conflict and the list of PIDs of
* backends holding the conflicting lock. Note that we do
* logging even if there are no such backends right now
* because the startup process here has already waited
* longer than deadlock_timeout.
*/
LogRecoveryConflict(PROCSIG_RECOVERY_CONFLICT_LOCK,
standbyWaitStart, now,
cnt > 0 ? vxids : NULL, true);
logged_recovery_conflict = true;
}
}
}
else
{
2018-11-23 08:16:41 +01:00
(void) WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 0,
PG_WAIT_LOCK | locallock->tag.lock.locktag_type);
ResetLatch(MyLatch);
/* check for deadlocks first, as that's probably log-worthy */
if (got_deadlock_timeout)
{
CheckDeadLock();
got_deadlock_timeout = false;
}
CHECK_FOR_INTERRUPTS();
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
}
/*
* waitStatus could change from PROC_WAIT_STATUS_WAITING to something
* else asynchronously. Read it just once per loop to prevent
* surprising behavior (such as missing log messages).
*/
myWaitStatus = *((volatile ProcWaitStatus *) &MyProc->waitStatus);
/*
* If we are not deadlocked, but are waiting on an autovacuum-induced
* task, send a signal to interrupt it.
*/
if (deadlock_state == DS_BLOCKED_BY_AUTOVACUUM && allow_autovacuum_cancel)
{
PGPROC *autovac = GetBlockingAutoVacuumPgproc();
uint8 statusFlags;
uint8 lockmethod_copy;
LOCKTAG locktag_copy;
/*
* Grab info we need, then release lock immediately. Note this
* coding means that there is a tiny chance that the process
* terminates its current transaction and starts a different one
* before we have a change to send the signal; the worst possible
* consequence is that a for-wraparound vacuum is canceled. But
* that could happen in any case unless we were to do kill() with
* the lock held, which is much more undesirable.
*/
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
statusFlags = ProcGlobal->statusFlags[autovac->pgxactoff];
lockmethod_copy = lock->tag.locktag_lockmethodid;
locktag_copy = lock->tag;
LWLockRelease(ProcArrayLock);
/*
* Only do it if the worker is not working to protect against Xid
* wraparound.
*/
if ((statusFlags & PROC_IS_AUTOVACUUM) &&
!(statusFlags & PROC_VACUUM_FOR_WRAPAROUND))
{
int pid = autovac->pid;
/* report the case, if configured to do so */
if (message_level_is_interesting(DEBUG1))
{
StringInfoData locktagbuf;
StringInfoData logbuf; /* errdetail for server log */
initStringInfo(&locktagbuf);
initStringInfo(&logbuf);
DescribeLockTag(&locktagbuf, &locktag_copy);
appendStringInfo(&logbuf,
"Process %d waits for %s on %s.",
MyProcPid,
GetLockmodeName(lockmethod_copy, lockmode),
locktagbuf.data);
ereport(DEBUG1,
(errmsg_internal("sending cancel to blocking autovacuum PID %d",
pid),
errdetail_log("%s", logbuf.data)));
pfree(locktagbuf.data);
pfree(logbuf.data);
}
/* send the autovacuum worker Back to Old Kent Road */
if (kill(pid, SIGINT) < 0)
{
/*
* There's a race condition here: once we release the
* ProcArrayLock, it's possible for the autovac worker to
* close up shop and exit before we can do the kill().
* Therefore, we do not whinge about no-such-process.
* Other errors such as EPERM could conceivably happen if
* the kernel recycles the PID fast enough, but such cases
* seem improbable enough that it's probably best to issue
* a warning if we see some other errno.
*/
if (errno != ESRCH)
ereport(WARNING,
(errmsg("could not send signal to process %d: %m",
pid)));
}
}
/* prevent signal from being sent again more than once */
allow_autovacuum_cancel = false;
}
/*
* If awoken after the deadlock check interrupt has run, and
* log_lock_waits is on, then report about the wait.
*/
if (log_lock_waits && deadlock_state != DS_NOT_YET_CHECKED)
{
StringInfoData buf,
lock_waiters_sbuf,
lock_holders_sbuf;
const char *modename;
long secs;
int usecs;
long msecs;
dlist_iter proc_iter;
PROCLOCK *curproclock;
bool first_holder = true,
first_waiter = true;
int lockHoldersNum = 0;
initStringInfo(&buf);
initStringInfo(&lock_waiters_sbuf);
initStringInfo(&lock_holders_sbuf);
DescribeLockTag(&buf, &locallock->tag.lock);
modename = GetLockmodeName(locallock->tag.lock.locktag_lockmethodid,
lockmode);
TimestampDifference(get_timeout_start_time(DEADLOCK_TIMEOUT),
GetCurrentTimestamp(),
&secs, &usecs);
msecs = secs * 1000 + usecs / 1000;
usecs = usecs % 1000;
/*
* we loop over the lock's procLocks to gather a list of all
* holders and waiters. Thus we will be able to provide more
* detailed information for lock debugging purposes.
*
* lock->procLocks contains all processes which hold or wait for
* this lock.
*/
LWLockAcquire(partitionLock, LW_SHARED);
dlist_foreach(proc_iter, &lock->procLocks)
{
curproclock =
dlist_container(PROCLOCK, lockLink, proc_iter.cur);
/*
* we are a waiter if myProc->waitProcLock == curproclock; we
* are a holder if it is NULL or something different
*/
if (curproclock->tag.myProc->waitProcLock == curproclock)
{
if (first_waiter)
{
appendStringInfo(&lock_waiters_sbuf, "%d",
curproclock->tag.myProc->pid);
first_waiter = false;
}
else
appendStringInfo(&lock_waiters_sbuf, ", %d",
curproclock->tag.myProc->pid);
}
else
{
if (first_holder)
{
appendStringInfo(&lock_holders_sbuf, "%d",
curproclock->tag.myProc->pid);
first_holder = false;
}
else
appendStringInfo(&lock_holders_sbuf, ", %d",
curproclock->tag.myProc->pid);
lockHoldersNum++;
}
}
LWLockRelease(partitionLock);
if (deadlock_state == DS_SOFT_DEADLOCK)
ereport(LOG,
(errmsg("process %d avoided deadlock for %s on %s by rearranging queue order after %ld.%03d ms",
MyProcPid, modename, buf.data, msecs, usecs),
(errdetail_log_plural("Process holding the lock: %s. Wait queue: %s.",
"Processes holding the lock: %s. Wait queue: %s.",
lockHoldersNum, lock_holders_sbuf.data, lock_waiters_sbuf.data))));
else if (deadlock_state == DS_HARD_DEADLOCK)
{
/*
* This message is a bit redundant with the error that will be
* reported subsequently, but in some cases the error report
* might not make it to the log (eg, if it's caught by an
* exception handler), and we want to ensure all long-wait
* events get logged.
*/
ereport(LOG,
(errmsg("process %d detected deadlock while waiting for %s on %s after %ld.%03d ms",
MyProcPid, modename, buf.data, msecs, usecs),
(errdetail_log_plural("Process holding the lock: %s. Wait queue: %s.",
"Processes holding the lock: %s. Wait queue: %s.",
lockHoldersNum, lock_holders_sbuf.data, lock_waiters_sbuf.data))));
}
if (myWaitStatus == PROC_WAIT_STATUS_WAITING)
ereport(LOG,
(errmsg("process %d still waiting for %s on %s after %ld.%03d ms",
MyProcPid, modename, buf.data, msecs, usecs),
(errdetail_log_plural("Process holding the lock: %s. Wait queue: %s.",
"Processes holding the lock: %s. Wait queue: %s.",
lockHoldersNum, lock_holders_sbuf.data, lock_waiters_sbuf.data))));
else if (myWaitStatus == PROC_WAIT_STATUS_OK)
ereport(LOG,
(errmsg("process %d acquired %s on %s after %ld.%03d ms",
MyProcPid, modename, buf.data, msecs, usecs)));
else
{
Assert(myWaitStatus == PROC_WAIT_STATUS_ERROR);
2007-11-15 22:14:46 +01:00
/*
* Currently, the deadlock checker always kicks its own
* process, which means that we'll only see
* PROC_WAIT_STATUS_ERROR when deadlock_state ==
* DS_HARD_DEADLOCK, and there's no need to print redundant
* messages. But for completeness and future-proofing, print
* a message if it looks like someone else kicked us off the
* lock.
*/
if (deadlock_state != DS_HARD_DEADLOCK)
ereport(LOG,
(errmsg("process %d failed to acquire %s on %s after %ld.%03d ms",
MyProcPid, modename, buf.data, msecs, usecs),
(errdetail_log_plural("Process holding the lock: %s. Wait queue: %s.",
"Processes holding the lock: %s. Wait queue: %s.",
lockHoldersNum, lock_holders_sbuf.data, lock_waiters_sbuf.data))));
}
/*
* At this point we might still need to wait for the lock. Reset
* state so we don't print the above messages again.
*/
deadlock_state = DS_NO_DEADLOCK;
pfree(buf.data);
pfree(lock_holders_sbuf.data);
pfree(lock_waiters_sbuf.data);
}
} while (myWaitStatus == PROC_WAIT_STATUS_WAITING);
1998-12-29 20:32:08 +01:00
/*
Fix assorted race conditions in the new timeout infrastructure. Prevent handle_sig_alarm from losing control partway through due to a query cancel (either an asynchronous SIGINT, or a cancel triggered by one of the timeout handler functions). That would at least result in failure to schedule any required future interrupt, and might result in actual corruption of timeout.c's data structures, if the interrupt happened while we were updating those. We could still lose control if an asynchronous SIGINT arrives just as the function is entered. This wouldn't break any data structures, but it would have the same effect as if the SIGALRM interrupt had been silently lost: we'd not fire any currently-due handlers, nor schedule any new interrupt. To forestall that scenario, forcibly reschedule any pending timer interrupt during AbortTransaction and AbortSubTransaction. We can avoid any extra kernel call in most cases by not doing that until we've allowed LockErrorCleanup to kill the DEADLOCK_TIMEOUT and LOCK_TIMEOUT events. Another hazard is that some platforms (at least Linux and *BSD) block a signal before calling its handler and then unblock it on return. When we longjmp out of the handler, the unblock doesn't happen, and the signal is left blocked indefinitely. Again, we can fix that by forcibly unblocking signals during AbortTransaction and AbortSubTransaction. These latter two problems do not manifest when the longjmp reaches postgres.c, because the error recovery code there kills all pending timeout events anyway, and it uses sigsetjmp(..., 1) so that the appropriate signal mask is restored. So errors thrown outside any transaction should be OK already, and cleaning up in AbortTransaction and AbortSubTransaction should be enough to fix these issues. (We're assuming that any code that catches a query cancel error and doesn't re-throw it will do at least a subtransaction abort to clean up; but that was pretty much required already by other subsystems.) Lastly, ProcSleep should not clear the LOCK_TIMEOUT indicator flag when disabling that event: if a lock timeout interrupt happened after the lock was granted, the ensuing query cancel is still going to happen at the next CHECK_FOR_INTERRUPTS, and we want to report it as a lock timeout not a user cancel. Per reports from Dan Wood. Back-patch to 9.3 where the new timeout handling infrastructure was introduced. We may at some point decide to back-patch the signal unblocking changes further, but I'll desist from that until we hear actual field complaints about it.
2013-11-29 22:41:00 +01:00
* Disable the timers, if they are still running. As in LockErrorCleanup,
* we must preserve the LOCK_TIMEOUT indicator flag: if a lock timeout has
* already caused QueryCancelPending to become set, we want the cancel to
* be reported as a lock timeout, not a user cancel.
1998-12-29 20:32:08 +01:00
*/
if (!InHotStandby)
{
if (LockTimeout > 0)
{
DisableTimeoutParams timeouts[2];
timeouts[0].id = DEADLOCK_TIMEOUT;
timeouts[0].keep_indicator = false;
timeouts[1].id = LOCK_TIMEOUT;
timeouts[1].keep_indicator = true;
disable_timeouts(timeouts, 2);
}
else
disable_timeout(DEADLOCK_TIMEOUT, false);
}
1998-12-29 20:32:08 +01:00
/*
* Emit the log message if recovery conflict on lock was resolved but the
* startup process waited longer than deadlock_timeout for it.
*/
if (InHotStandby && logged_recovery_conflict)
LogRecoveryConflict(PROCSIG_RECOVERY_CONFLICT_LOCK,
standbyWaitStart, GetCurrentTimestamp(),
NULL, false);
/*
* Re-acquire the lock table's partition lock. We have to do this to hold
* off cancel/die interrupts before we can mess with lockAwaited (else we
* might have a missed or duplicated locallock update).
*/
LWLockAcquire(partitionLock, LW_EXCLUSIVE);
/*
* We no longer want LockErrorCleanup to do anything.
*/
lockAwaited = NULL;
/*
* If we got the lock, be sure to remember it in the locallock table.
*/
if (MyProc->waitStatus == PROC_WAIT_STATUS_OK)
GrantAwaitedLock();
/*
* We don't have to do anything else, because the awaker did all the
* necessary update of the lock table and MyProc.
*/
return MyProc->waitStatus;
}
/*
* ProcWakeup -- wake up a process by setting its latch.
*
* Also remove the process from the wait queue and set its links invalid.
*
* The appropriate lock partition lock must be held by caller.
*
* XXX: presently, this code is only used for the "success" case, and only
* works correctly for that case. To clean up in failure case, would need
* to twiddle the lock's request counts too --- see RemoveFromWaitQueue.
* Hence, in practice the waitStatus parameter must be PROC_WAIT_STATUS_OK.
*/
void
ProcWakeup(PGPROC *proc, ProcWaitStatus waitStatus)
{
if (dlist_node_is_detached(&proc->links))
return;
Assert(proc->waitStatus == PROC_WAIT_STATUS_WAITING);
/* Remove process from wait queue */
dclist_delete_from_thoroughly(&proc->waitLock->waitProcs, &proc->links);
/* Clean up process' state and pass it the ok/fail signal */
proc->waitLock = NULL;
proc->waitProcLock = NULL;
proc->waitStatus = waitStatus;
Display the time when the process started waiting for the lock, in pg_locks, take 2 This commit adds new column "waitstart" into pg_locks view. This column reports the time when the server process started waiting for the lock if the lock is not held. This information is useful, for example, when examining the amount of time to wait on a lock by subtracting "waitstart" in pg_locks from the current time, and identify the lock that the processes are waiting for very long. This feature uses the current time obtained for the deadlock timeout timer as "waitstart" (i.e., the time when this process started waiting for the lock). Since getting the current time newly can cause overhead, we reuse the already-obtained time to avoid that overhead. Note that "waitstart" is updated without holding the lock table's partition lock, to avoid the overhead by additional lock acquisition. This can cause "waitstart" in pg_locks to become NULL for a very short period of time after the wait started even though "granted" is false. This is OK in practice because we can assume that users are likely to look at "waitstart" when waiting for the lock for a long time. The first attempt of this patch (commit 3b733fcd04) caused the buildfarm member "rorqual" (built with --disable-atomics --disable-spinlocks) to report the failure of the regression test. It was reverted by commit 890d2182a2. The cause of this failure was that the atomic variable for "waitstart" in the dummy process entry created at the end of prepare transaction was not initialized. This second attempt fixes that issue. Bump catalog version. Author: Atsushi Torikoshi Reviewed-by: Ian Lawrence Barwick, Robert Haas, Justin Pryzby, Fujii Masao Discussion: https://postgr.es/m/a96013dc51cdc56b2a2b84fa8a16a993@oss.nttdata.com
2021-02-15 07:13:37 +01:00
pg_atomic_write_u64(&MyProc->waitStart, 0);
/* And awaken it */
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
SetLatch(&proc->procLatch);
}
/*
* ProcLockWakeup -- routine for waking up processes when a lock is
* released (or a prior waiter is aborted). Scan all waiters
* for lock, waken any that are no longer blocked.
*
* The appropriate lock partition lock must be held by caller.
*/
void
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
ProcLockWakeup(LockMethod lockMethodTable, LOCK *lock)
{
dclist_head *waitQueue = &lock->waitProcs;
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
LOCKMASK aheadRequests = 0;
dlist_mutable_iter miter;
if (dclist_is_empty(waitQueue))
return;
dclist_foreach_modify(miter, waitQueue)
{
PGPROC *proc = dlist_container(PGPROC, links, miter.cur);
LOCKMODE lockmode = proc->waitLockMode;
/*
* Waken if (a) doesn't conflict with requests of earlier waiters, and
* (b) doesn't conflict with already-held locks.
*/
if ((lockMethodTable->conflictTab[lockmode] & aheadRequests) == 0 &&
!LockCheckConflicts(lockMethodTable, lockmode, lock,
proc->waitProcLock))
{
/* OK to waken */
GrantLock(lock, proc->waitProcLock, lockmode);
/* removes proc from the lock's waiting process queue */
ProcWakeup(proc, PROC_WAIT_STATUS_OK);
}
else
{
/*
* Lock conflicts: Don't wake, but remember requested mode for
* later checks.
*/
Try to reduce confusion about what is a lock method identifier, a lock method control structure, or a table of control structures. . Use type LOCKMASK where an int is not a counter. . Get rid of INVALID_TABLEID, use INVALID_LOCKMETHOD instead. . Use INVALID_LOCKMETHOD instead of (LOCKMETHOD) NULL, because LOCKMETHOD is not a pointer. . Define and use macro LockMethodIsValid. . Rename LOCKMETHOD to LOCKMETHODID. . Remove global variable LongTermTableId in lmgr.c, because it is never used. . Make LockTableId static in lmgr.c, because it is used nowhere else. Why not remove it and use DEFAULT_LOCKMETHOD? . Rename the lock method control structure from LOCKMETHODTABLE to LockMethodData. Introduce a pointer type named LockMethod. . Remove elog(FATAL) after InitLockTable() call in CreateSharedMemoryAndSemaphores(), because if something goes wrong, there is elog(FATAL) in LockMethodTableInit(), and if this doesn't help, an elog(ERROR) in InitLockTable() is promoted to FATAL. . Make InitLockTable() void, because its only caller does not use its return value any more. . Rename variables in lock.c to avoid statements like LockMethodTable[NumLockMethods] = lockMethodTable; lockMethodTable = LockMethodTable[lockmethod]; . Change LOCKMETHODID type to uint16 to fit into struct LOCKTAG. . Remove static variables BITS_OFF and BITS_ON from lock.c, because I agree to this doubt: * XXX is a fetch from a static array really faster than a shift? . Define and use macros LOCKBIT_ON/OFF. Manfred Koizar
2003-12-01 22:59:25 +01:00
aheadRequests |= LOCKBIT_ON(lockmode);
}
}
}
/*
* CheckDeadLock
*
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
* We only get to this routine, if DEADLOCK_TIMEOUT fired while waiting for a
* lock to be released by some other process. Check if there's a deadlock; if
* not, just return. (But signal ProcSleep to log a message, if
* log_lock_waits is true.) If we have a real deadlock, remove ourselves from
* the lock's wait queue and signal an error to ProcSleep.
*/
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
static void
CheckDeadLock(void)
{
int i;
/*
* Acquire exclusive lock on the entire shared lock data structures. Must
* grab LWLocks in partition-number order to avoid LWLock deadlock.
*
* Note that the deadlock check interrupt had better not be enabled
* anywhere that this process itself holds lock partition locks, else this
* will wait forever. Also note that LWLockAcquire creates a critical
* section, so that this routine cannot be interrupted by cancel/die
* interrupts.
*/
for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
LWLockAcquire(LockHashPartitionLockByIndex(i), LW_EXCLUSIVE);
/*
* Check to see if we've been awoken by anyone in the interim.
*
* If we have, we can return and resume our transaction -- happy day.
* Before we are awoken the process releasing the lock grants it to us so
* we know that we don't have to wait anymore.
*
* We check by looking to see if we've been unlinked from the wait queue.
* This is safe because we hold the lock partition lock.
*/
if (MyProc->links.prev == NULL ||
MyProc->links.next == NULL)
goto check_done;
#ifdef LOCK_DEBUG
if (Debug_deadlocks)
DumpAllLocks();
#endif
/* Run the deadlock check, and set deadlock_state for use by ProcSleep */
deadlock_state = DeadLockCheck(MyProc);
if (deadlock_state == DS_HARD_DEADLOCK)
1998-01-27 04:00:43 +01:00
{
/*
* Oops. We have a deadlock.
*
* Get this process out of wait state. (Note: we could do this more
* efficiently by relying on lockAwaited, but use this coding to
* preserve the flexibility to kill some other transaction than the
* one detecting the deadlock.)
*
* RemoveFromWaitQueue sets MyProc->waitStatus to
* PROC_WAIT_STATUS_ERROR, so ProcSleep will report an error after we
* return from the signal handler.
*/
Assert(MyProc->waitLock != NULL);
RemoveFromWaitQueue(MyProc, LockTagHashCode(&(MyProc->waitLock->tag)));
/*
* We're done here. Transaction abort caused by the error that
* ProcSleep will raise will cause any other locks we hold to be
* released, thus allowing other processes to wake up; we don't need
* to do that here. NOTE: an exception is that releasing locks we
* hold doesn't consider the possibility of waiters that were blocked
* behind us on the lock we just failed to get, and might now be
* wakable because we're not in front of them anymore. However,
* RemoveFromWaitQueue took care of waking up any such processes.
*/
}
/*
* And release locks. We do this in reverse order for two reasons: (1)
* Anyone else who needs more than one of the locks will be trying to lock
* them in increasing order; we don't want to release the other process
* until it can get all the locks it needs. (2) This avoids O(N^2)
* behavior inside LWLockRelease.
*/
check_done:
for (i = NUM_LOCK_PARTITIONS; --i >= 0;)
LWLockRelease(LockHashPartitionLockByIndex(i));
}
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
/*
* CheckDeadLockAlert - Handle the expiry of deadlock_timeout.
*
* NB: Runs inside a signal handler, be careful.
*/
void
CheckDeadLockAlert(void)
{
int save_errno = errno;
got_deadlock_timeout = true;
2015-05-24 03:35:49 +02:00
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
/*
* Have to set the latch again, even if handle_sig_alarm already did. Back
* then got_deadlock_timeout wasn't yet set... It's unlikely that this
* ever would be a problem, but setting a set latch again is cheap.
Detect the deadlocks between backends and the startup process. The deadlocks that the recovery conflict on lock is involved in can happen between hot-standby backends and the startup process. If a backend takes an access exclusive lock on the table and which finally triggers the deadlock, that deadlock can be detected as expected. On the other hand, previously, if the startup process took an access exclusive lock and which finally triggered the deadlock, that deadlock could not be detected and could remain even after deadlock_timeout passed. This is a bug. The cause of this bug was that the code for handling the recovery conflict on lock didn't take care of deadlock case at all. It assumed that deadlocks involving the startup process and backends were able to be detected by the deadlock detector invoked within backends. But this assumption was incorrect. The startup process also should have invoked the deadlock detector if necessary. To fix this bug, this commit makes the startup process invoke the deadlock detector if deadlock_timeout is reached while handling the recovery conflict on lock. Specifically, in that case, the startup process requests all the backends holding the conflicting locks to check themselves for deadlocks. Back-patch to v9.6. v9.5 has also this bug, but per discussion we decided not to back-patch the fix to v9.5. Because v9.5 doesn't have some infrastructure codes (e.g., 37c54863cf) that this bug fix patch depends on. We can apply those codes for the back-patch, but since the next minor version release is the final one for v9.5, it's risky to do that. If we unexpectedly introduce new bug to v9.5 by the back-patch, there is no chance to fix that. We determined that the back-patch to v9.5 would give more risk than gain. Author: Fujii Masao Reviewed-by: Bertrand Drouvot, Masahiko Sawada, Kyotaro Horiguchi Discussion: https://postgr.es/m/4041d6b6-cf24-a120-36fa-1294220f8243@oss.nttdata.com
2021-01-06 04:39:18 +01:00
*
* Note that, when this function runs inside procsignal_sigusr1_handler(),
* the handler function sets the latch again after the latch is set here.
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
*/
SetLatch(MyLatch);
errno = save_errno;
}
/*
* ProcWaitForSignal - wait for a signal from another backend.
*
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
* As this uses the generic process latch the caller has to be robust against
* unrelated wakeups: Always check that the desired state has occurred, and
* wait again if not.
*/
void
ProcWaitForSignal(uint32 wait_event_info)
{
2018-11-23 08:16:41 +01:00
(void) WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 0,
wait_event_info);
Move deadlock and other interrupt handling in proc.c out of signal handlers. Deadlock checking was performed inside signal handlers up to now. While it's a remarkable feat to have made this work reliably, it's quite complex to understand why that is the case. Partially it worked due to the assumption that semaphores are signal safe - which is not actually documented to be the case for sysv semaphores. The reason we had to rely on performing this work inside signal handlers is that semaphores aren't guaranteed to be interruptable by signals on all platforms. But now that latches provide a somewhat similar API, which actually has the guarantee of being interruptible, we can avoid doing so. Signalling between ProcSleep, ProcWakeup, ProcWaitForSignal and ProcSendSignal is now done using latches. This increases the likelihood of spurious wakeups. As spurious wakeup already were possible and aren't likely to be frequent enough to be an actual problem, this seems acceptable. This change would allow for further simplification of the deadlock checking, now that it doesn't have to run in a signal handler. But even if I were motivated to do so right now, it would still be better to do that separately. Such a cleanup shouldn't have to be reviewed a the same time as the more fundamental changes in this commit. There is one possible usability regression due to this commit. Namely it is more likely than before that log_lock_waits messages are output more than once. Reviewed-By: Heikki Linnakangas
2015-02-03 23:24:38 +01:00
ResetLatch(MyLatch);
CHECK_FOR_INTERRUPTS();
}
/*
* ProcSendSignal - set the latch of a backend identified by ProcNumber
*/
void
ProcSendSignal(ProcNumber procNumber)
{
if (procNumber < 0 || procNumber >= ProcGlobal->allProcCount)
elog(ERROR, "procNumber out of range");
Allow read only connections during recovery, known as Hot Standby. Enabled by recovery_connections = on (default) and forcing archive recovery using a recovery.conf. Recovery processing now emulates the original transactions as they are replayed, providing full locking and MVCC behaviour for read only queries. Recovery must enter consistent state before connections are allowed, so there is a delay, typically short, before connections succeed. Replay of recovering transactions can conflict and in some cases deadlock with queries during recovery; these result in query cancellation after max_standby_delay seconds have expired. Infrastructure changes have minor effects on normal running, though introduce four new types of WAL record. New test mode "make standbycheck" allows regression tests of static command behaviour on a standby server while in recovery. Typical and extreme dynamic behaviours have been checked via code inspection and manual testing. Few port specific behaviours have been utilised, though primary testing has been on Linux only so far. This commit is the basic patch. Additional changes will follow in this release to enhance some aspects of behaviour, notably improved handling of conflicts, deadlock detection and query cancellation. Changes to VACUUM FULL are also required. Simon Riggs, with significant and lengthy review by Heikki Linnakangas, including streamlined redesign of snapshot creation and two-phase commit. Important contributions from Florian Pflug, Mark Kirkwood, Merlin Moncure, Greg Stark, Gianni Ciolli, Gabriele Bartolini, Hannu Krosing, Robert Haas, Tatsuo Ishii, Hiroyuki Yamada plus support and feedback from many other community members.
2009-12-19 02:32:45 +01:00
SetLatch(&ProcGlobal->allProcs[procNumber].procLatch);
}
/*
* BecomeLockGroupLeader - designate process as lock group leader
*
* Once this function has returned, other processes can join the lock group
* by calling BecomeLockGroupMember.
*/
void
BecomeLockGroupLeader(void)
{
LWLock *leader_lwlock;
/* If we already did it, we don't need to do it again. */
if (MyProc->lockGroupLeader == MyProc)
return;
/* We had better not be a follower. */
Assert(MyProc->lockGroupLeader == NULL);
/* Create single-member group, containing only ourselves. */
leader_lwlock = LockHashPartitionLockByProc(MyProc);
LWLockAcquire(leader_lwlock, LW_EXCLUSIVE);
MyProc->lockGroupLeader = MyProc;
dlist_push_head(&MyProc->lockGroupMembers, &MyProc->lockGroupLink);
LWLockRelease(leader_lwlock);
}
/*
* BecomeLockGroupMember - designate process as lock group member
*
* This is pretty straightforward except for the possibility that the leader
* whose group we're trying to join might exit before we manage to do so;
* and the PGPROC might get recycled for an unrelated process. To avoid
* that, we require the caller to pass the PID of the intended PGPROC as
* an interlock. Returns true if we successfully join the intended lock
* group, and false if not.
*/
bool
BecomeLockGroupMember(PGPROC *leader, int pid)
{
LWLock *leader_lwlock;
bool ok = false;
/* Group leader can't become member of group */
Assert(MyProc != leader);
/* Can't already be a member of a group */
Assert(MyProc->lockGroupLeader == NULL);
/* PID must be valid. */
Assert(pid != 0);
/*
* Get lock protecting the group fields. Note LockHashPartitionLockByProc
* calculates the proc number based on the PGPROC slot without looking at
* its contents, so we will acquire the correct lock even if the leader
* PGPROC is in process of being recycled.
*/
leader_lwlock = LockHashPartitionLockByProc(leader);
LWLockAcquire(leader_lwlock, LW_EXCLUSIVE);
/* Is this the leader we're looking for? */
if (leader->pid == pid && leader->lockGroupLeader == leader)
{
/* OK, join the group */
ok = true;
MyProc->lockGroupLeader = leader;
dlist_push_tail(&leader->lockGroupMembers, &MyProc->lockGroupLink);
}
LWLockRelease(leader_lwlock);
return ok;
}