aboutsummaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
Diffstat (limited to 'src')
-rw-r--r--src/backend/access/transam/twophase.c37
-rw-r--r--src/backend/access/transam/xact.c12
-rw-r--r--src/backend/catalog/namespace.c2
-rw-r--r--src/backend/commands/sequence.c2
-rw-r--r--src/backend/executor/functions.c4
-rw-r--r--src/backend/postmaster/auxprocess.c12
-rw-r--r--src/backend/storage/ipc/procarray.c73
-rw-r--r--src/backend/storage/ipc/procsignal.c27
-rw-r--r--src/backend/storage/ipc/sinvaladt.c200
-rw-r--r--src/backend/storage/ipc/standby.c1
-rw-r--r--src/backend/storage/lmgr/lock.c28
-rw-r--r--src/backend/storage/lmgr/proc.c32
-rw-r--r--src/backend/utils/activity/backend_status.c52
-rw-r--r--src/backend/utils/adt/lockfuncs.c2
-rw-r--r--src/backend/utils/adt/mcxtfuncs.c14
-rw-r--r--src/backend/utils/error/csvlog.c4
-rw-r--r--src/backend/utils/error/elog.c6
-rw-r--r--src/backend/utils/error/jsonlog.c6
-rw-r--r--src/backend/utils/init/postinit.c10
-rw-r--r--src/backend/utils/time/snapmgr.c5
-rw-r--r--src/include/miscadmin.h2
-rw-r--r--src/include/storage/backendid.h12
-rw-r--r--src/include/storage/lock.h9
-rw-r--r--src/include/storage/proc.h32
-rw-r--r--src/include/storage/procarray.h4
-rw-r--r--src/include/storage/procsignal.h2
-rw-r--r--src/include/storage/sinvaladt.h4
-rw-r--r--src/pl/plpgsql/src/pl_exec.c10
28 files changed, 282 insertions, 322 deletions
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 234c8d08ebc..5c282002900 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -151,7 +151,6 @@ typedef struct GlobalTransactionData
{
GlobalTransaction next; /* list link for free list */
int pgprocno; /* ID of associated dummy PGPROC */
- BackendId dummyBackendId; /* similar to backend id for backends */
TimestampTz prepared_at; /* time of preparation */
/*
@@ -285,20 +284,6 @@ TwoPhaseShmemInit(void)
/* associate it with a PGPROC assigned by InitProcGlobal */
gxacts[i].pgprocno = GetNumberFromPGProc(&PreparedXactProcs[i]);
-
- /*
- * Assign a unique ID for each dummy proc, so that the range of
- * dummy backend IDs immediately follows the range of normal
- * backend IDs. We don't dare to assign a real backend ID to dummy
- * procs, because prepared transactions don't take part in cache
- * invalidation like a real backend ID would imply, but having a
- * unique ID for them is nevertheless handy. This arrangement
- * allows you to allocate an array of size (MaxBackends +
- * max_prepared_xacts + 1), and have a slot for every backend and
- * prepared transaction. Currently multixact.c uses that
- * technique.
- */
- gxacts[i].dummyBackendId = MaxBackends + 1 + i;
}
}
else
@@ -457,24 +442,24 @@ MarkAsPreparingGuts(GlobalTransaction gxact, TransactionId xid, const char *gid,
Assert(LWLockHeldByMeInMode(TwoPhaseStateLock, LW_EXCLUSIVE));
Assert(gxact != NULL);
- proc = &ProcGlobal->allProcs[gxact->pgprocno];
+ proc = GetPGProcByNumber(gxact->pgprocno);
/* Initialize the PGPROC entry */
MemSet(proc, 0, sizeof(PGPROC));
dlist_node_init(&proc->links);
proc->waitStatus = PROC_WAIT_STATUS_OK;
- if (LocalTransactionIdIsValid(MyProc->lxid))
+ if (LocalTransactionIdIsValid(MyProc->vxid.lxid))
{
/* clone VXID, for TwoPhaseGetXidByVirtualXID() to find */
- proc->lxid = MyProc->lxid;
- proc->backendId = MyBackendId;
+ proc->vxid.lxid = MyProc->vxid.lxid;
+ proc->vxid.backendId = MyBackendId;
}
else
{
Assert(AmStartupProcess() || !IsPostmasterEnvironment);
/* GetLockConflicts() uses this to specify a wait on the XID */
- proc->lxid = xid;
- proc->backendId = InvalidBackendId;
+ proc->vxid.lxid = xid;
+ proc->vxid.backendId = InvalidBackendId;
}
proc->xid = xid;
Assert(proc->xmin == InvalidTransactionId);
@@ -522,7 +507,7 @@ static void
GXactLoadSubxactData(GlobalTransaction gxact, int nsubxacts,
TransactionId *children)
{
- PGPROC *proc = &ProcGlobal->allProcs[gxact->pgprocno];
+ PGPROC *proc = GetPGProcByNumber(gxact->pgprocno);
/* We need no extra lock since the GXACT isn't valid yet */
if (nsubxacts > PGPROC_MAX_CACHED_SUBXIDS)
@@ -559,7 +544,7 @@ MarkAsPrepared(GlobalTransaction gxact, bool lock_held)
* Put it into the global ProcArray so TransactionIdIsInProgress considers
* the XID as still running.
*/
- ProcArrayAdd(&ProcGlobal->allProcs[gxact->pgprocno]);
+ ProcArrayAdd(GetPGProcByNumber(gxact->pgprocno));
}
/*
@@ -583,7 +568,7 @@ LockGXact(const char *gid, Oid user)
for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
{
GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
- PGPROC *proc = &ProcGlobal->allProcs[gxact->pgprocno];
+ PGPROC *proc = GetPGProcByNumber(gxact->pgprocno);
/* Ignore not-yet-valid GIDs */
if (!gxact->valid)
@@ -884,7 +869,7 @@ TwoPhaseGetXidByVirtualXID(VirtualTransactionId vxid,
if (!gxact->valid)
continue;
- proc = &ProcGlobal->allProcs[gxact->pgprocno];
+ proc = GetPGProcByNumber(gxact->pgprocno);
GET_VXID_FROM_PGPROC(proc_vxid, *proc);
if (VirtualTransactionIdEquals(vxid, proc_vxid))
{
@@ -919,7 +904,7 @@ TwoPhaseGetDummyBackendId(TransactionId xid, bool lock_held)
{
GlobalTransaction gxact = TwoPhaseGetGXact(xid, lock_held);
- return gxact->dummyBackendId;
+ return gxact->pgprocno + 1;
}
/*
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 70ab6e27a13..4ac5b9ea834 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -600,9 +600,9 @@ GetStableLatestTransactionId(void)
static LocalTransactionId lxid = InvalidLocalTransactionId;
static TransactionId stablexid = InvalidTransactionId;
- if (lxid != MyProc->lxid)
+ if (lxid != MyProc->vxid.lxid)
{
- lxid = MyProc->lxid;
+ lxid = MyProc->vxid.lxid;
stablexid = GetTopTransactionIdIfAny();
if (!TransactionIdIsValid(stablexid))
stablexid = ReadNextTransactionId();
@@ -2099,8 +2099,8 @@ StartTransaction(void)
* Advertise it in the proc array. We assume assignment of
* localTransactionId is atomic, and the backendId should be set already.
*/
- Assert(MyProc->backendId == vxid.backendId);
- MyProc->lxid = vxid.localTransactionId;
+ Assert(MyProc->vxid.backendId == vxid.backendId);
+ MyProc->vxid.lxid = vxid.localTransactionId;
TRACE_POSTGRESQL_TRANSACTION_START(vxid.localTransactionId);
@@ -2289,7 +2289,7 @@ CommitTransaction(void)
ParallelWorkerReportLastRecEnd(XactLastRecEnd);
}
- TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
+ TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->vxid.lxid);
/*
* Let others know about no transaction in progress by me. Note that this
@@ -2840,7 +2840,7 @@ AbortTransaction(void)
XLogSetAsyncXactLSN(XactLastRecEnd);
}
- TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
+ TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->vxid.lxid);
/*
* Let others know about no transaction in progress by me. Note that this
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index 8df30b24401..620ce7e75d2 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -49,7 +49,7 @@
#include "parser/parse_func.h"
#include "storage/ipc.h"
#include "storage/lmgr.h"
-#include "storage/sinvaladt.h"
+#include "storage/procarray.h"
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/catcache.h"
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 1bed9c74d17..24b9ee42c2a 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -1077,7 +1077,7 @@ setval3_oid(PG_FUNCTION_ARGS)
static Relation
lock_and_open_sequence(SeqTable seq)
{
- LocalTransactionId thislxid = MyProc->lxid;
+ LocalTransactionId thislxid = MyProc->vxid.lxid;
/* Get the lock if not already held in this xact */
if (seq->lxid != thislxid)
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 0f811fd2fc9..a4b6e1effdb 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -799,7 +799,7 @@ init_sql_fcache(FunctionCallInfo fcinfo, Oid collation, bool lazyEvalOK)
lazyEvalOK);
/* Mark fcache with time of creation to show it's valid */
- fcache->lxid = MyProc->lxid;
+ fcache->lxid = MyProc->vxid.lxid;
fcache->subxid = GetCurrentSubTransactionId();
ReleaseSysCache(procedureTuple);
@@ -1081,7 +1081,7 @@ fmgr_sql(PG_FUNCTION_ARGS)
if (fcache != NULL)
{
- if (fcache->lxid != MyProc->lxid ||
+ if (fcache->lxid != MyProc->vxid.lxid ||
!SubTransactionIsActive(fcache->subxid))
{
/* It's stale; unlink and delete */
diff --git a/src/backend/postmaster/auxprocess.c b/src/backend/postmaster/auxprocess.c
index ab86e802f21..39171fea06b 100644
--- a/src/backend/postmaster/auxprocess.c
+++ b/src/backend/postmaster/auxprocess.c
@@ -107,17 +107,7 @@ AuxiliaryProcessMain(AuxProcType auxtype)
BaseInit();
- /*
- * Assign the ProcSignalSlot for an auxiliary process. Since it doesn't
- * have a BackendId, the slot is statically allocated based on the
- * auxiliary process type (MyAuxProcType). Backends use slots indexed in
- * the range from 1 to MaxBackends (inclusive), so we use MaxBackends +
- * AuxProcType + 1 as the index of the slot for an auxiliary process.
- *
- * This will need rethinking if we ever want more than one of a particular
- * auxiliary process type.
- */
- ProcSignalInit(MaxBackends + MyAuxProcType + 1);
+ ProcSignalInit();
/*
* Auxiliary processes don't run transactions, but they may need a
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index dd329a86ef4..d96606ebba5 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -701,7 +701,7 @@ ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid)
Assert(proc->subxidStatus.count == 0);
Assert(!proc->subxidStatus.overflowed);
- proc->lxid = InvalidLocalTransactionId;
+ proc->vxid.lxid = InvalidLocalTransactionId;
proc->xmin = InvalidTransactionId;
/* be sure this is cleared in abort */
@@ -743,7 +743,7 @@ ProcArrayEndTransactionInternal(PGPROC *proc, TransactionId latestXid)
ProcGlobal->xids[pgxactoff] = InvalidTransactionId;
proc->xid = InvalidTransactionId;
- proc->lxid = InvalidLocalTransactionId;
+ proc->vxid.lxid = InvalidLocalTransactionId;
proc->xmin = InvalidTransactionId;
/* be sure this is cleared in abort */
@@ -930,7 +930,7 @@ ProcArrayClearTransaction(PGPROC *proc)
ProcGlobal->xids[pgxactoff] = InvalidTransactionId;
proc->xid = InvalidTransactionId;
- proc->lxid = InvalidLocalTransactionId;
+ proc->vxid.lxid = InvalidLocalTransactionId;
proc->xmin = InvalidTransactionId;
proc->recoveryConflictPending = false;
@@ -2536,6 +2536,11 @@ ProcArrayInstallImportedXmin(TransactionId xmin,
/* Get lock so source xact can't end while we're doing this */
LWLockAcquire(ProcArrayLock, LW_SHARED);
+ /*
+ * Find the PGPROC entry of the source transaction. (This could use
+ * GetPGProcByBackendId(), unless it's a prepared xact. But this isn't
+ * performance critical.)
+ */
for (index = 0; index < arrayP->numProcs; index++)
{
int pgprocno = arrayP->pgprocnos[index];
@@ -2548,9 +2553,9 @@ ProcArrayInstallImportedXmin(TransactionId xmin,
continue;
/* We are only interested in the specific virtual transaction. */
- if (proc->backendId != sourcevxid->backendId)
+ if (proc->vxid.backendId != sourcevxid->backendId)
continue;
- if (proc->lxid != sourcevxid->localTransactionId)
+ if (proc->vxid.lxid != sourcevxid->localTransactionId)
continue;
/*
@@ -3100,6 +3105,64 @@ HaveVirtualXIDsDelayingChkpt(VirtualTransactionId *vxids, int nvxids, int type)
}
/*
+ * BackendIdGetProc -- get a backend's PGPROC given its backend ID
+ *
+ * The result may be out of date arbitrarily quickly, so the caller
+ * must be careful about how this information is used. NULL is
+ * returned if the backend is not active.
+ */
+PGPROC *
+BackendIdGetProc(int backendID)
+{
+ PGPROC *result;
+
+ if (backendID < 1 || backendID > ProcGlobal->allProcCount)
+ return NULL;
+ result = GetPGProcByBackendId(backendID);
+
+ if (result->pid == 0)
+ return NULL;
+
+ return result;
+}
+
+/*
+ * BackendIdGetTransactionIds -- get a backend's transaction status
+ *
+ * Get the xid, xmin, nsubxid and overflow status of the backend. The
+ * result may be out of date arbitrarily quickly, so the caller must be
+ * careful about how this information is used.
+ */
+void
+BackendIdGetTransactionIds(int backendID, TransactionId *xid,
+ TransactionId *xmin, int *nsubxid, bool *overflowed)
+{
+ PGPROC *proc;
+
+ *xid = InvalidTransactionId;
+ *xmin = InvalidTransactionId;
+ *nsubxid = 0;
+ *overflowed = false;
+
+ if (backendID < 1 || backendID > ProcGlobal->allProcCount)
+ return;
+ proc = GetPGProcByBackendId(backendID);
+
+ /* Need to lock out additions/removals of backends */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ if (proc->pid != 0)
+ {
+ *xid = proc->xid;
+ *xmin = proc->xmin;
+ *nsubxid = proc->subxidStatus.count;
+ *overflowed = proc->subxidStatus.overflowed;
+ }
+
+ LWLockRelease(ProcArrayLock);
+}
+
+/*
* BackendPidGetProc -- get a backend's PGPROC given its PID
*
* Returns NULL if not found. Note that it is up to the caller to be
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 0f9f90d2c7b..199dd182253 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -87,7 +87,7 @@ typedef struct
* possible auxiliary process type. (This scheme assumes there is not
* more than one of any auxiliary process type at a time.)
*/
-#define NumProcSignalSlots (MaxBackends + NUM_AUXPROCTYPES)
+#define NumProcSignalSlots (MaxBackends + NUM_AUXILIARY_PROCS)
/* Check whether the relevant type bit is set in the flags. */
#define BARRIER_SHOULD_CHECK(flags, type) \
@@ -154,24 +154,23 @@ ProcSignalShmemInit(void)
/*
* ProcSignalInit
* Register the current process in the ProcSignal array
- *
- * The passed index should be my BackendId if the process has one,
- * or MaxBackends + aux process type if not.
*/
void
-ProcSignalInit(int pss_idx)
+ProcSignalInit(void)
{
ProcSignalSlot *slot;
uint64 barrier_generation;
- Assert(pss_idx >= 1 && pss_idx <= NumProcSignalSlots);
-
- slot = &ProcSignal->psh_slot[pss_idx - 1];
+ if (MyBackendId <= 0)
+ elog(ERROR, "MyBackendId not set");
+ if (MyBackendId > NumProcSignalSlots)
+ elog(ERROR, "unexpected MyBackendId %d in ProcSignalInit (max %d)", MyBackendId, NumProcSignalSlots);
+ slot = &ProcSignal->psh_slot[MyBackendId - 1];
/* sanity check */
if (slot->pss_pid != 0)
elog(LOG, "process %d taking over ProcSignal slot %d, but it's not empty",
- MyProcPid, pss_idx);
+ MyProcPid, MyBackendId - 1);
/* Clear out any leftover signal reasons */
MemSet(slot->pss_signalFlags, 0, NUM_PROCSIGNALS * sizeof(sig_atomic_t));
@@ -200,7 +199,7 @@ ProcSignalInit(int pss_idx)
MyProcSignalSlot = slot;
/* Set up to release the slot on process exit */
- on_shmem_exit(CleanupProcSignalState, Int32GetDatum(pss_idx));
+ on_shmem_exit(CleanupProcSignalState, (Datum) 0);
}
/*
@@ -212,11 +211,7 @@ ProcSignalInit(int pss_idx)
static void
CleanupProcSignalState(int status, Datum arg)
{
- int pss_idx = DatumGetInt32(arg);
- ProcSignalSlot *slot;
-
- slot = &ProcSignal->psh_slot[pss_idx - 1];
- Assert(slot == MyProcSignalSlot);
+ ProcSignalSlot *slot = MyProcSignalSlot;
/*
* Clear MyProcSignalSlot, so that a SIGUSR1 received after this point
@@ -233,7 +228,7 @@ CleanupProcSignalState(int status, Datum arg)
* infinite loop trying to exit
*/
elog(LOG, "process %d releasing ProcSignal slot %d, but it contains %d",
- MyProcPid, pss_idx, (int) slot->pss_pid);
+ MyProcPid, (int) (slot - ProcSignal->psh_slot), (int) slot->pss_pid);
return; /* XXX better to zero the slot anyway? */
}
diff --git a/src/backend/storage/ipc/sinvaladt.c b/src/backend/storage/ipc/sinvaladt.c
index 748a792a854..f624bfc7d78 100644
--- a/src/backend/storage/ipc/sinvaladt.c
+++ b/src/backend/storage/ipc/sinvaladt.c
@@ -139,7 +139,6 @@ typedef struct ProcState
{
/* procPid is zero in an inactive ProcState array entry. */
pid_t procPid; /* PID of backend, for signaling */
- PGPROC *proc; /* PGPROC of backend */
/* nextMsgNum is meaningless if procPid == 0 or resetState is true. */
int nextMsgNum; /* next message number to read */
bool resetState; /* backend needs to reset its state */
@@ -172,8 +171,6 @@ typedef struct SISeg
int minMsgNum; /* oldest message still needed */
int maxMsgNum; /* next message number to be assigned */
int nextThreshold; /* # of messages to call SICleanupQueue */
- int lastBackend; /* index of last active procState entry, +1 */
- int maxBackends; /* size of procState array */
slock_t msgnumLock; /* spinlock protecting maxMsgNum */
@@ -183,11 +180,29 @@ typedef struct SISeg
SharedInvalidationMessage buffer[MAXNUMMESSAGES];
/*
- * Per-backend invalidation state info (has MaxBackends entries).
+ * Per-backend invalidation state info.
+ *
+ * 'procState' has NumProcStateSlots entries, and is indexed by pgprocno.
+ * 'numProcs' is the number of slots currently in use, and 'pgprocnos' is
+ * a dense array of their indexes, to speed up scanning all in-use slots.
+ *
+ * 'pgprocnos' is largely redundant with ProcArrayStruct->pgprocnos, but
+ * having our separate copy avoids contention on ProcArrayLock, and allows
+ * us to track only the processes that participate in shared cache
+ * invalidations.
*/
+ int numProcs;
+ int *pgprocnos;
ProcState procState[FLEXIBLE_ARRAY_MEMBER];
} SISeg;
+/*
+ * We reserve a slot for each possible BackendId, plus one for each
+ * possible auxiliary process type. (This scheme assumes there is not
+ * more than one of any auxiliary process type at a time.)
+ */
+#define NumProcStateSlots (MaxBackends + NUM_AUXILIARY_PROCS)
+
static SISeg *shmInvalBuffer; /* pointer to the shared inval buffer */
@@ -205,16 +220,8 @@ SInvalShmemSize(void)
Size size;
size = offsetof(SISeg, procState);
-
- /*
- * In Hot Standby mode, the startup process requests a procState array
- * slot using InitRecoveryTransactionEnvironment(). Even though
- * MaxBackends doesn't account for the startup process, it is guaranteed
- * to get a free slot. This is because the autovacuum launcher and worker
- * processes, which are included in MaxBackends, are not started in Hot
- * Standby mode.
- */
- size = add_size(size, mul_size(sizeof(ProcState), MaxBackends));
+ size = add_size(size, mul_size(sizeof(ProcState), NumProcStateSlots)); /* procState */
+ size = add_size(size, mul_size(sizeof(int), NumProcStateSlots)); /* pgprocnos */
return size;
}
@@ -239,23 +246,22 @@ CreateSharedInvalidationState(void)
shmInvalBuffer->minMsgNum = 0;
shmInvalBuffer->maxMsgNum = 0;
shmInvalBuffer->nextThreshold = CLEANUP_MIN;
- shmInvalBuffer->lastBackend = 0;
- shmInvalBuffer->maxBackends = MaxBackends;
SpinLockInit(&shmInvalBuffer->msgnumLock);
/* The buffer[] array is initially all unused, so we need not fill it */
/* Mark all backends inactive, and initialize nextLXID */
- for (i = 0; i < shmInvalBuffer->maxBackends; i++)
+ for (i = 0; i < NumProcStateSlots; i++)
{
shmInvalBuffer->procState[i].procPid = 0; /* inactive */
- shmInvalBuffer->procState[i].proc = NULL;
shmInvalBuffer->procState[i].nextMsgNum = 0; /* meaningless */
shmInvalBuffer->procState[i].resetState = false;
shmInvalBuffer->procState[i].signaled = false;
shmInvalBuffer->procState[i].hasMessages = false;
shmInvalBuffer->procState[i].nextLXID = InvalidLocalTransactionId;
}
+ shmInvalBuffer->numProcs = 0;
+ shmInvalBuffer->pgprocnos = (int *) &shmInvalBuffer->procState[i];
}
/*
@@ -265,59 +271,41 @@ CreateSharedInvalidationState(void)
void
SharedInvalBackendInit(bool sendOnly)
{
- int index;
- ProcState *stateP = NULL;
+ ProcState *stateP;
+ pid_t oldPid;
SISeg *segP = shmInvalBuffer;
+ int pgprocno;
+
+ if (MyBackendId <= 0)
+ elog(ERROR, "MyBackendId not set");
+ if (MyBackendId > NumProcStateSlots)
+ elog(PANIC, "unexpected MyBackendId %d in SharedInvalBackendInit (max %d)",
+ MyBackendId, NumProcStateSlots);
+ pgprocno = MyBackendId - 1;
+ stateP = &segP->procState[pgprocno];
/*
* This can run in parallel with read operations, but not with write
- * operations, since SIInsertDataEntries relies on lastBackend to set
- * hasMessages appropriately.
+ * operations, since SIInsertDataEntries relies on the pgprocnos array to
+ * set hasMessages appropriately.
*/
LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
- /* Look for a free entry in the procState array */
- for (index = 0; index < segP->lastBackend; index++)
- {
- if (segP->procState[index].procPid == 0) /* inactive slot? */
- {
- stateP = &segP->procState[index];
- break;
- }
- }
-
- if (stateP == NULL)
+ oldPid = stateP->procPid;
+ if (oldPid != 0)
{
- if (segP->lastBackend < segP->maxBackends)
- {
- stateP = &segP->procState[segP->lastBackend];
- Assert(stateP->procPid == 0);
- segP->lastBackend++;
- }
- else
- {
- /*
- * out of procState slots: MaxBackends exceeded -- report normally
- */
- MyBackendId = InvalidBackendId;
- LWLockRelease(SInvalWriteLock);
- ereport(FATAL,
- (errcode(ERRCODE_TOO_MANY_CONNECTIONS),
- errmsg("sorry, too many clients already")));
- }
+ LWLockRelease(SInvalWriteLock);
+ elog(ERROR, "sinval slot for backend %d is already in use by process %d",
+ MyBackendId, (int) oldPid);
}
- MyBackendId = (stateP - &segP->procState[0]) + 1;
-
- /* Advertise assigned backend ID in MyProc */
- MyProc->backendId = MyBackendId;
+ shmInvalBuffer->pgprocnos[shmInvalBuffer->numProcs++] = pgprocno;
/* Fetch next local transaction ID into local memory */
nextLocalTransactionId = stateP->nextLXID;
/* mark myself active, with all extant messages already read */
stateP->procPid = MyProcPid;
- stateP->proc = MyProc;
stateP->nextMsgNum = segP->maxMsgNum;
stateP->resetState = false;
stateP->signaled = false;
@@ -328,8 +316,6 @@ SharedInvalBackendInit(bool sendOnly)
/* register exit routine to mark my entry inactive at exit */
on_shmem_exit(CleanupInvalidationState, PointerGetDatum(segP));
-
- elog(DEBUG4, "my backend ID is %d", MyBackendId);
}
/*
@@ -345,96 +331,36 @@ CleanupInvalidationState(int status, Datum arg)
{
SISeg *segP = (SISeg *) DatumGetPointer(arg);
ProcState *stateP;
+ int pgprocno = MyBackendId - 1;
int i;
Assert(PointerIsValid(segP));
LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
- stateP = &segP->procState[MyBackendId - 1];
+ stateP = &segP->procState[pgprocno];
/* Update next local transaction ID for next holder of this backendID */
stateP->nextLXID = nextLocalTransactionId;
/* Mark myself inactive */
stateP->procPid = 0;
- stateP->proc = NULL;
stateP->nextMsgNum = 0;
stateP->resetState = false;
stateP->signaled = false;
- /* Recompute index of last active backend */
- for (i = segP->lastBackend; i > 0; i--)
+ for (i = segP->numProcs - 1; i >= 0; i--)
{
- if (segP->procState[i - 1].procPid != 0)
- break;
- }
- segP->lastBackend = i;
-
- LWLockRelease(SInvalWriteLock);
-}
-
-/*
- * BackendIdGetProc
- * Get the PGPROC structure for a backend, given the backend ID.
- * The result may be out of date arbitrarily quickly, so the caller
- * must be careful about how this information is used. NULL is
- * returned if the backend is not active.
- */
-PGPROC *
-BackendIdGetProc(int backendID)
-{
- PGPROC *result = NULL;
- SISeg *segP = shmInvalBuffer;
-
- /* Need to lock out additions/removals of backends */
- LWLockAcquire(SInvalWriteLock, LW_SHARED);
-
- if (backendID > 0 && backendID <= segP->lastBackend)
- {
- ProcState *stateP = &segP->procState[backendID - 1];
-
- result = stateP->proc;
- }
-
- LWLockRelease(SInvalWriteLock);
-
- return result;
-}
-
-/*
- * BackendIdGetTransactionIds
- * Get the xid, xmin, nsubxid and overflow status of the backend. The
- * result may be out of date arbitrarily quickly, so the caller must be
- * careful about how this information is used.
- */
-void
-BackendIdGetTransactionIds(int backendID, TransactionId *xid,
- TransactionId *xmin, int *nsubxid, bool *overflowed)
-{
- SISeg *segP = shmInvalBuffer;
-
- *xid = InvalidTransactionId;
- *xmin = InvalidTransactionId;
- *nsubxid = 0;
- *overflowed = false;
-
- /* Need to lock out additions/removals of backends */
- LWLockAcquire(SInvalWriteLock, LW_SHARED);
-
- if (backendID > 0 && backendID <= segP->lastBackend)
- {
- ProcState *stateP = &segP->procState[backendID - 1];
- PGPROC *proc = stateP->proc;
-
- if (proc != NULL)
+ if (segP->pgprocnos[i] == pgprocno)
{
- *xid = proc->xid;
- *xmin = proc->xmin;
- *nsubxid = proc->subxidStatus.count;
- *overflowed = proc->subxidStatus.overflowed;
+ if (i != segP->numProcs - 1)
+ segP->pgprocnos[i] = segP->pgprocnos[segP->numProcs - 1];
+ break;
}
}
+ if (i < 0)
+ elog(PANIC, "could not find entry in sinval array");
+ segP->numProcs--;
LWLockRelease(SInvalWriteLock);
}
@@ -507,9 +433,9 @@ SIInsertDataEntries(const SharedInvalidationMessage *data, int n)
* these (unlocked) changes will be committed to memory before we exit
* the function.
*/
- for (i = 0; i < segP->lastBackend; i++)
+ for (i = 0; i < segP->numProcs; i++)
{
- ProcState *stateP = &segP->procState[i];
+ ProcState *stateP = &segP->procState[segP->pgprocnos[i]];
stateP->hasMessages = true;
}
@@ -677,13 +603,14 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
minsig = min - SIG_THRESHOLD;
lowbound = min - MAXNUMMESSAGES + minFree;
- for (i = 0; i < segP->lastBackend; i++)
+ for (i = 0; i < segP->numProcs; i++)
{
- ProcState *stateP = &segP->procState[i];
+ ProcState *stateP = &segP->procState[segP->pgprocnos[i]];
int n = stateP->nextMsgNum;
- /* Ignore if inactive or already in reset state */
- if (stateP->procPid == 0 || stateP->resetState || stateP->sendOnly)
+ /* Ignore if already in reset state */
+ Assert(stateP->procPid != 0);
+ if (stateP->resetState || stateP->sendOnly)
continue;
/*
@@ -719,11 +646,8 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
{
segP->minMsgNum -= MSGNUMWRAPAROUND;
segP->maxMsgNum -= MSGNUMWRAPAROUND;
- for (i = 0; i < segP->lastBackend; i++)
- {
- /* we don't bother skipping inactive entries here */
- segP->procState[i].nextMsgNum -= MSGNUMWRAPAROUND;
- }
+ for (i = 0; i < segP->numProcs; i++)
+ segP->procState[segP->pgprocnos[i]].nextMsgNum -= MSGNUMWRAPAROUND;
}
/*
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index d8755a106d5..97d1ab65740 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -137,6 +137,7 @@ InitRecoveryTransactionEnvironment(void)
* are held by vxids and row level locks are held by xids. All queries
* hold AccessShareLocks so never block while we write or lock new rows.
*/
+ MyProc->vxid.backendId = MyBackendId;
vxid.backendId = MyBackendId;
vxid.localTransactionId = GetNextLocalTransactionId();
VirtualXactLockTableInsert(vxid);
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index c70a1adb9ad..e62968b4a86 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -3625,8 +3625,8 @@ GetLockStatusData(void)
proc->fpRelId[f]);
instance->holdMask = lockbits << FAST_PATH_LOCKNUMBER_OFFSET;
instance->waitLockMode = NoLock;
- instance->backend = proc->backendId;
- instance->lxid = proc->lxid;
+ instance->vxid.backendId = proc->vxid.backendId;
+ instance->vxid.localTransactionId = proc->vxid.lxid;
instance->pid = proc->pid;
instance->leaderPid = proc->pid;
instance->fastpath = true;
@@ -3652,15 +3652,15 @@ GetLockStatusData(void)
repalloc(data->locks, sizeof(LockInstanceData) * els);
}
- vxid.backendId = proc->backendId;
+ vxid.backendId = proc->vxid.backendId;
vxid.localTransactionId = proc->fpLocalTransactionId;
instance = &data->locks[el];
SET_LOCKTAG_VIRTUALTRANSACTION(instance->locktag, vxid);
instance->holdMask = LOCKBIT_ON(ExclusiveLock);
instance->waitLockMode = NoLock;
- instance->backend = proc->backendId;
- instance->lxid = proc->lxid;
+ instance->vxid.backendId = proc->vxid.backendId;
+ instance->vxid.localTransactionId = proc->vxid.lxid;
instance->pid = proc->pid;
instance->leaderPid = proc->pid;
instance->fastpath = true;
@@ -3712,8 +3712,8 @@ GetLockStatusData(void)
instance->waitLockMode = proc->waitLockMode;
else
instance->waitLockMode = NoLock;
- instance->backend = proc->backendId;
- instance->lxid = proc->lxid;
+ instance->vxid.backendId = proc->vxid.backendId;
+ instance->vxid.localTransactionId = proc->vxid.lxid;
instance->pid = proc->pid;
instance->leaderPid = proclock->groupLeader->pid;
instance->fastpath = false;
@@ -3888,8 +3888,8 @@ GetSingleProcBlockerStatusData(PGPROC *blocked_proc, BlockedProcsData *data)
instance->waitLockMode = proc->waitLockMode;
else
instance->waitLockMode = NoLock;
- instance->backend = proc->backendId;
- instance->lxid = proc->lxid;
+ instance->vxid.backendId = proc->vxid.backendId;
+ instance->vxid.localTransactionId = proc->vxid.lxid;
instance->pid = proc->pid;
instance->leaderPid = proclock->groupLeader->pid;
instance->fastpath = false;
@@ -4374,8 +4374,8 @@ lock_twophase_postabort(TransactionId xid, uint16 info,
* lockers, as we haven't advertised this vxid via the ProcArray yet.
*
* Since MyProc->fpLocalTransactionId will normally contain the same data
- * as MyProc->lxid, you might wonder if we really need both. The
- * difference is that MyProc->lxid is set and cleared unlocked, and
+ * as MyProc->vxid.lxid, you might wonder if we really need both. The
+ * difference is that MyProc->vxid.lxid is set and cleared unlocked, and
* examined by procarray.c, while fpLocalTransactionId is protected by
* fpInfoLock and is used only by the locking subsystem. Doing it this
* way makes it easier to verify that there are no funny race conditions.
@@ -4391,7 +4391,7 @@ VirtualXactLockTableInsert(VirtualTransactionId vxid)
LWLockAcquire(&MyProc->fpInfoLock, LW_EXCLUSIVE);
- Assert(MyProc->backendId == vxid.backendId);
+ Assert(MyProc->vxid.backendId == vxid.backendId);
Assert(MyProc->fpLocalTransactionId == InvalidLocalTransactionId);
Assert(MyProc->fpVXIDLock == false);
@@ -4413,7 +4413,7 @@ VirtualXactLockTableCleanup(void)
bool fastpath;
LocalTransactionId lxid;
- Assert(MyProc->backendId != InvalidBackendId);
+ Assert(MyProc->vxid.backendId != InvalidBackendId);
/*
* Clean up shared memory state.
@@ -4541,7 +4541,7 @@ VirtualXactLock(VirtualTransactionId vxid, bool wait)
*/
LWLockAcquire(&proc->fpInfoLock, LW_EXCLUSIVE);
- if (proc->backendId != vxid.backendId
+ if (proc->vxid.backendId != vxid.backendId
|| proc->fpLocalTransactionId != vxid.localTransactionId)
{
/* VXID ended */
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 6e334971dc9..f98575fcaf1 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -242,25 +242,25 @@ InitProcGlobal(void)
if (i < MaxConnections)
{
/* PGPROC for normal backend, add to freeProcs list */
- dlist_push_head(&ProcGlobal->freeProcs, &proc->links);
+ dlist_push_tail(&ProcGlobal->freeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->freeProcs;
}
else if (i < MaxConnections + autovacuum_max_workers + 1)
{
/* PGPROC for AV launcher/worker, add to autovacFreeProcs list */
- dlist_push_head(&ProcGlobal->autovacFreeProcs, &proc->links);
+ dlist_push_tail(&ProcGlobal->autovacFreeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->autovacFreeProcs;
}
else if (i < MaxConnections + autovacuum_max_workers + 1 + max_worker_processes)
{
/* PGPROC for bgworker, add to bgworkerFreeProcs list */
- dlist_push_head(&ProcGlobal->bgworkerFreeProcs, &proc->links);
+ dlist_push_tail(&ProcGlobal->bgworkerFreeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->bgworkerFreeProcs;
}
else if (i < MaxBackends)
{
/* PGPROC for walsender, add to walsenderFreeProcs list */
- dlist_push_head(&ProcGlobal->walsenderFreeProcs, &proc->links);
+ dlist_push_tail(&ProcGlobal->walsenderFreeProcs, &proc->links);
proc->procgloballist = &ProcGlobal->walsenderFreeProcs;
}
@@ -355,6 +355,7 @@ InitProcess(void)
errmsg("sorry, too many clients already")));
}
MyProcNumber = GetNumberFromPGProc(MyProc);
+ MyBackendId = GetBackendIdFromPGProc(MyProc);
/*
* Cross-check that the PGPROC is of the type we expect; if this were not
@@ -381,14 +382,14 @@ InitProcess(void)
*/
dlist_node_init(&MyProc->links);
MyProc->waitStatus = PROC_WAIT_STATUS_OK;
- MyProc->lxid = InvalidLocalTransactionId;
MyProc->fpVXIDLock = false;
MyProc->fpLocalTransactionId = InvalidLocalTransactionId;
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
MyProc->pid = MyProcPid;
- /* backendId, databaseId and roleId will be filled in later */
- MyProc->backendId = InvalidBackendId;
+ MyProc->vxid.backendId = MyBackendId;
+ MyProc->vxid.lxid = InvalidLocalTransactionId;
+ /* databaseId and roleId will be filled in later */
MyProc->databaseId = InvalidOid;
MyProc->roleId = InvalidOid;
MyProc->tempNamespaceId = InvalidOid;
@@ -568,11 +569,11 @@ InitAuxiliaryProcess(void)
/* use volatile pointer to prevent code rearrangement */
((volatile PGPROC *) auxproc)->pid = MyProcPid;
- MyProc = auxproc;
-
SpinLockRelease(ProcStructLock);
+ MyProc = auxproc;
MyProcNumber = GetNumberFromPGProc(MyProc);
+ MyBackendId = GetBackendIdFromPGProc(MyProc);
/*
* Initialize all fields of MyProc, except for those previously
@@ -580,12 +581,12 @@ InitAuxiliaryProcess(void)
*/
dlist_node_init(&MyProc->links);
MyProc->waitStatus = PROC_WAIT_STATUS_OK;
- MyProc->lxid = InvalidLocalTransactionId;
MyProc->fpVXIDLock = false;
MyProc->fpLocalTransactionId = InvalidLocalTransactionId;
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
- MyProc->backendId = InvalidBackendId;
+ MyProc->vxid.backendId = InvalidBackendId;
+ MyProc->vxid.lxid = InvalidLocalTransactionId;
MyProc->databaseId = InvalidOid;
MyProc->roleId = InvalidOid;
MyProc->tempNamespaceId = InvalidOid;
@@ -916,8 +917,14 @@ ProcKill(int code, Datum arg)
proc = MyProc;
MyProc = NULL;
MyProcNumber = INVALID_PGPROCNO;
+ MyBackendId = InvalidBackendId;
DisownLatch(&proc->procLatch);
+ /* Mark the proc no longer in use */
+ proc->pid = 0;
+ proc->vxid.backendId = InvalidBackendId;
+ proc->vxid.lxid = InvalidTransactionId;
+
procgloballist = proc->procgloballist;
SpinLockAcquire(ProcStructLock);
@@ -992,12 +999,15 @@ AuxiliaryProcKill(int code, Datum arg)
proc = MyProc;
MyProc = NULL;
MyProcNumber = INVALID_PGPROCNO;
+ MyBackendId = InvalidBackendId;
DisownLatch(&proc->procLatch);
SpinLockAcquire(ProcStructLock);
/* Mark auxiliary proc no longer in use */
proc->pid = 0;
+ proc->vxid.backendId = InvalidBackendId;
+ proc->vxid.lxid = InvalidTransactionId;
/* Update shared estimate of spins_per_delay */
ProcGlobal->spins_per_delay = update_spins_per_delay(ProcGlobal->spins_per_delay);
diff --git a/src/backend/utils/activity/backend_status.c b/src/backend/utils/activity/backend_status.c
index 1a1050c8da1..3d3f7b06723 100644
--- a/src/backend/utils/activity/backend_status.c
+++ b/src/backend/utils/activity/backend_status.c
@@ -19,6 +19,7 @@
#include "port/atomics.h" /* for memory barriers */
#include "storage/ipc.h"
#include "storage/proc.h" /* for MyProc */
+#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "utils/ascii.h"
#include "utils/backend_status.h"
@@ -29,13 +30,12 @@
/* ----------
* Total number of backends including auxiliary
*
- * We reserve a slot for each possible BackendId, plus one for each
- * possible auxiliary process type. (This scheme assumes there is not
- * more than one of any auxiliary process type at a time.) MaxBackends
- * includes autovacuum workers and background workers as well.
+ * We reserve a slot for each possible PGPROC entry, including aux processes.
+ * (But not including PGPROC entries reserved for prepared xacts; they are not
+ * real processes.)
* ----------
*/
-#define NumBackendStatSlots (MaxBackends + NUM_AUXPROCTYPES)
+#define NumBackendStatSlots (MaxBackends + NUM_AUXILIARY_PROCS)
/* ----------
@@ -238,10 +238,9 @@ CreateSharedBackendStatus(void)
/*
* Initialize pgstats backend activity state, and set up our on-proc-exit
- * hook. Called from InitPostgres and AuxiliaryProcessMain. For auxiliary
- * process, MyBackendId is invalid. Otherwise, MyBackendId must be set, but we
- * must not have started any transaction yet (since the exit hook must run
- * after the last transaction exit).
+ * hook. Called from InitPostgres and AuxiliaryProcessMain. MyBackendId must
+ * be set, but we must not have started any transaction yet (since the exit
+ * hook must run after the last transaction exit).
*
* NOTE: MyDatabaseId isn't set yet; so the shutdown hook has to be careful.
*/
@@ -249,26 +248,9 @@ void
pgstat_beinit(void)
{
/* Initialize MyBEEntry */
- if (MyBackendId != InvalidBackendId)
- {
- Assert(MyBackendId >= 1 && MyBackendId <= MaxBackends);
- MyBEEntry = &BackendStatusArray[MyBackendId - 1];
- }
- else
- {
- /* Must be an auxiliary process */
- Assert(MyAuxProcType != NotAnAuxProcess);
-
- /*
- * Assign the MyBEEntry for an auxiliary process. Since it doesn't
- * have a BackendId, the slot is statically allocated based on the
- * auxiliary process type (MyAuxProcType). Backends use slots indexed
- * in the range from 0 to MaxBackends (exclusive), so we use
- * MaxBackends + AuxProcType as the index of the slot for an auxiliary
- * process.
- */
- MyBEEntry = &BackendStatusArray[MaxBackends + MyAuxProcType];
- }
+ Assert(MyBackendId != InvalidBackendId);
+ Assert(MyBackendId >= 1 && MyBackendId <= NumBackendStatSlots);
+ MyBEEntry = &BackendStatusArray[MyBackendId - 1];
/* Set up a process-exit hook to clean up */
on_shmem_exit(pgstat_beshutdown_hook, 0);
@@ -281,12 +263,12 @@ pgstat_beinit(void)
* Initialize this backend's entry in the PgBackendStatus array.
* Called from InitPostgres.
*
- * Apart from auxiliary processes, MyBackendId, MyDatabaseId,
- * session userid, and application_name must be set for a
- * backend (hence, this cannot be combined with pgstat_beinit).
- * Note also that we must be inside a transaction if this isn't an aux
- * process, as we may need to do encoding conversion on some strings.
- * ----------
+ * Apart from auxiliary processes, MyDatabaseId, session userid, and
+ * application_name must already be set (hence, this cannot be combined
+ * with pgstat_beinit). Note also that we must be inside a transaction
+ * if this isn't an aux process, as we may need to do encoding conversion
+ * on some strings.
+ *----------
*/
void
pgstat_bestart(void)
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index 4b49f7fe3d8..bbe5cc0806e 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -353,7 +353,7 @@ pg_lock_status(PG_FUNCTION_ARGS)
break;
}
- values[10] = VXIDGetDatum(instance->backend, instance->lxid);
+ values[10] = VXIDGetDatum(instance->vxid.backendId, instance->vxid.localTransactionId);
if (instance->pid != 0)
values[11] = Int32GetDatum(instance->pid);
else
diff --git a/src/backend/utils/adt/mcxtfuncs.c b/src/backend/utils/adt/mcxtfuncs.c
index 4708d73f5fa..a7267dc15d1 100644
--- a/src/backend/utils/adt/mcxtfuncs.c
+++ b/src/backend/utils/adt/mcxtfuncs.c
@@ -148,19 +148,11 @@ pg_log_backend_memory_contexts(PG_FUNCTION_ARGS)
PGPROC *proc;
BackendId backendId = InvalidBackendId;
- proc = BackendPidGetProc(pid);
-
/*
* See if the process with given pid is a backend or an auxiliary process.
- *
- * If the given process is a backend, use its backend id in
- * SendProcSignal() later to speed up the operation. Otherwise, don't do
- * that because auxiliary processes (except the startup process) don't
- * have a valid backend id.
*/
- if (proc != NULL)
- backendId = proc->backendId;
- else
+ proc = BackendPidGetProc(pid);
+ if (proc == NULL)
proc = AuxiliaryPidGetProc(pid);
/*
@@ -183,6 +175,8 @@ pg_log_backend_memory_contexts(PG_FUNCTION_ARGS)
PG_RETURN_BOOL(false);
}
+ if (proc != NULL)
+ backendId = GetBackendIdFromPGProc(proc);
if (SendProcSignal(pid, PROCSIG_LOG_MEMORY_CONTEXT, backendId) < 0)
{
/* Again, just a warning to allow loops */
diff --git a/src/backend/utils/error/csvlog.c b/src/backend/utils/error/csvlog.c
index 1b62b07f231..1d44d8a6a31 100644
--- a/src/backend/utils/error/csvlog.c
+++ b/src/backend/utils/error/csvlog.c
@@ -152,8 +152,8 @@ write_csvlog(ErrorData *edata)
/* Virtual transaction id */
/* keep VXID format in sync with lockfuncs.c */
- if (MyProc != NULL && MyProc->backendId != InvalidBackendId)
- appendStringInfo(&buf, "%d/%u", MyProc->backendId, MyProc->lxid);
+ if (MyProc != NULL && MyProc->vxid.backendId != InvalidBackendId)
+ appendStringInfo(&buf, "%d/%u", MyProc->vxid.backendId, MyProc->vxid.lxid);
appendStringInfoChar(&buf, ',');
/* Transaction id */
diff --git a/src/backend/utils/error/elog.c b/src/backend/utils/error/elog.c
index c9719f358b6..149b4b8df13 100644
--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -3076,18 +3076,18 @@ log_status_format(StringInfo buf, const char *format, ErrorData *edata)
break;
case 'v':
/* keep VXID format in sync with lockfuncs.c */
- if (MyProc != NULL && MyProc->backendId != InvalidBackendId)
+ if (MyProc != NULL && MyProc->vxid.backendId != InvalidBackendId)
{
if (padding != 0)
{
char strfbuf[128];
snprintf(strfbuf, sizeof(strfbuf) - 1, "%d/%u",
- MyProc->backendId, MyProc->lxid);
+ MyProc->vxid.backendId, MyProc->vxid.lxid);
appendStringInfo(buf, "%*s", padding, strfbuf);
}
else
- appendStringInfo(buf, "%d/%u", MyProc->backendId, MyProc->lxid);
+ appendStringInfo(buf, "%d/%u", MyProc->vxid.backendId, MyProc->vxid.lxid);
}
else if (padding != 0)
appendStringInfoSpaces(buf,
diff --git a/src/backend/utils/error/jsonlog.c b/src/backend/utils/error/jsonlog.c
index 2903561f1c4..067d9e30b16 100644
--- a/src/backend/utils/error/jsonlog.c
+++ b/src/backend/utils/error/jsonlog.c
@@ -197,9 +197,9 @@ write_jsonlog(ErrorData *edata)
/* Virtual transaction id */
/* keep VXID format in sync with lockfuncs.c */
- if (MyProc != NULL && MyProc->backendId != InvalidBackendId)
- appendJSONKeyValueFmt(&buf, "vxid", true, "%d/%u", MyProc->backendId,
- MyProc->lxid);
+ if (MyProc != NULL && MyProc->vxid.backendId != InvalidBackendId)
+ appendJSONKeyValueFmt(&buf, "vxid", true, "%d/%u",
+ MyProc->vxid.backendId, MyProc->vxid.lxid);
/* Transaction id */
appendJSONKeyValueFmt(&buf, "txid", false, "%u",
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index 5ffe9bdd987..c49c048441b 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -742,18 +742,10 @@ InitPostgres(const char *in_dbname, Oid dboid,
/*
* Initialize my entry in the shared-invalidation manager's array of
* per-backend data.
- *
- * Sets up MyBackendId, a unique backend identifier.
*/
- MyBackendId = InvalidBackendId;
-
SharedInvalBackendInit(false);
- if (MyBackendId > MaxBackends || MyBackendId <= 0)
- elog(FATAL, "bad backend ID: %d", MyBackendId);
-
- /* Now that we have a BackendId, we can participate in ProcSignal */
- ProcSignalInit(MyBackendId);
+ ProcSignalInit();
/*
* Also set up timeout handlers needed for backend operation. We need
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 675e81d82d7..a0916959b17 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -1154,7 +1154,8 @@ ExportSnapshot(Snapshot snapshot)
* inside the transaction from 1.
*/
snprintf(path, sizeof(path), SNAPSHOT_EXPORT_DIR "/%08X-%08X-%d",
- MyProc->backendId, MyProc->lxid, list_length(exportedSnapshots) + 1);
+ MyProc->vxid.backendId, MyProc->vxid.lxid,
+ list_length(exportedSnapshots) + 1);
/*
* Copy the snapshot into TopTransactionContext, add it to the
@@ -1181,7 +1182,7 @@ ExportSnapshot(Snapshot snapshot)
*/
initStringInfo(&buf);
- appendStringInfo(&buf, "vxid:%d/%u\n", MyProc->backendId, MyProc->lxid);
+ appendStringInfo(&buf, "vxid:%d/%u\n", MyProc->vxid.backendId, MyProc->vxid.lxid);
appendStringInfo(&buf, "pid:%d\n", MyProcPid);
appendStringInfo(&buf, "dbid:%u\n", MyDatabaseId);
appendStringInfo(&buf, "iso:%d\n", XactIsoLevel);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 756d144c323..519ef8ad684 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -454,8 +454,6 @@ typedef enum
WalWriterProcess,
WalReceiverProcess,
WalSummarizerProcess,
-
- NUM_AUXPROCTYPES /* Must be last! */
} AuxProcType;
extern PGDLLIMPORT AuxProcType MyAuxProcType;
diff --git a/src/include/storage/backendid.h b/src/include/storage/backendid.h
index 50ac982da19..01387723f79 100644
--- a/src/include/storage/backendid.h
+++ b/src/include/storage/backendid.h
@@ -14,11 +14,15 @@
#ifndef BACKENDID_H
#define BACKENDID_H
-/* ----------------
- * -cim 8/17/90
- * ----------------
+/*
+ * BackendId uniquely identifies an active backend or auxiliary process. It's
+ * assigned at backend startup after authentication. Note that a backend ID
+ * can be reused for a different backend immediately after a backend exits.
+ *
+ * Backend IDs are assigned starting from 1. For historical reasons, BackendId
+ * 0 is unused, but InvalidBackendId is defined as -1.
*/
-typedef int BackendId; /* unique currently active backend identifier */
+typedef int BackendId;
#define InvalidBackendId (-1)
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index ed6071f3286..13b81228e5b 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -74,9 +74,9 @@ typedef struct
#define SetInvalidVirtualTransactionId(vxid) \
((vxid).backendId = InvalidBackendId, \
(vxid).localTransactionId = InvalidLocalTransactionId)
-#define GET_VXID_FROM_PGPROC(vxid, proc) \
- ((vxid).backendId = (proc).backendId, \
- (vxid).localTransactionId = (proc).lxid)
+#define GET_VXID_FROM_PGPROC(vxid_dst, proc) \
+ ((vxid_dst).backendId = (proc).vxid.backendId, \
+ (vxid_dst).localTransactionId = (proc).vxid.lxid)
/* MAX_LOCKMODES cannot be larger than the # of bits in LOCKMASK */
#define MAX_LOCKMODES 10
@@ -454,8 +454,7 @@ typedef struct LockInstanceData
LOCKTAG locktag; /* tag for locked object */
LOCKMASK holdMask; /* locks held by this PGPROC */
LOCKMODE waitLockMode; /* lock awaited by this PGPROC, if any */
- BackendId backend; /* backend ID of this PGPROC */
- LocalTransactionId lxid; /* local transaction ID of this PGPROC */
+ VirtualTransactionId vxid; /* virtual transaction ID of this PGPROC */
TimestampTz waitStart; /* time at which this PGPROC started waiting
* for lock */
int pid; /* pid of this PGPROC */
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 4453c6df877..1e804463370 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -186,16 +186,31 @@ struct PGPROC
* vacuum must not remove tuples deleted by
* xid >= xmin ! */
- LocalTransactionId lxid; /* local id of top-level transaction currently
- * being executed by this proc, if running;
- * else InvalidLocalTransactionId */
int pid; /* Backend's process ID; 0 if prepared xact */
int pgxactoff; /* offset into various ProcGlobal->arrays with
* data mirrored from this PGPROC */
+ /*
+ * Currently running top-level transaction's virtual xid. Together these
+ * form a VirtualTransactionId, but we don't use that struct because this
+ * is not atomically assignable as whole, and we want to enforce code to
+ * consider both parts separately. See comments at VirtualTransactionId.
+ */
+ struct
+ {
+ BackendId backendId; /* For regular backends, equal to
+ * GetBackendIdFromPGProc(proc). For prepared
+ * xacts, ID of the original backend that
+ * processed the transaction. For unused
+ * PGPROC entries, InvalidBackendID. */
+ LocalTransactionId lxid; /* local id of top-level transaction
+ * currently * being executed by this
+ * proc, if running; else
+ * InvalidLocaltransactionId */
+ } vxid;
+
/* These fields are zero while a backend is still starting up: */
- BackendId backendId; /* This backend's backend ID (if assigned) */
Oid databaseId; /* OID of database this backend is using */
Oid roleId; /* OID of role using this backend */
@@ -406,9 +421,16 @@ extern PGDLLIMPORT PROC_HDR *ProcGlobal;
extern PGDLLIMPORT PGPROC *PreparedXactProcs;
-/* Accessor for PGPROC given a pgprocno, and vice versa. */
+/*
+ * Accessors for getting PGPROC given a pgprocno or BackendId, and vice versa.
+ *
+ * For historical reasons, some code uses 0-based "proc numbers", while other
+ * code uses 1-based backend IDs.
+ */
#define GetPGProcByNumber(n) (&ProcGlobal->allProcs[(n)])
#define GetNumberFromPGProc(proc) ((proc) - &ProcGlobal->allProcs[0])
+#define GetPGProcByBackendId(n) (&ProcGlobal->allProcs[(n) - 1])
+#define GetBackendIdFromPGProc(proc) (GetNumberFromPGProc(proc) + 1)
/*
* We set aside some extra PGPROC structures for auxiliary processes,
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index f3eba9b7640..3af7577e8c6 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -64,6 +64,10 @@ extern VirtualTransactionId *GetVirtualXIDsDelayingChkpt(int *nvxids, int type);
extern bool HaveVirtualXIDsDelayingChkpt(VirtualTransactionId *vxids,
int nvxids, int type);
+extern PGPROC *BackendIdGetProc(int backendID);
+extern void BackendIdGetTransactionIds(int backendID, TransactionId *xid,
+ TransactionId *xmin, int *nsubxid,
+ bool *overflowed);
extern PGPROC *BackendPidGetProc(int pid);
extern PGPROC *BackendPidGetProcWithLock(int pid);
extern int BackendXidGetPid(TransactionId xid);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index 52dcb4c2adf..febdda3611c 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -62,7 +62,7 @@ typedef enum
extern Size ProcSignalShmemSize(void);
extern void ProcSignalShmemInit(void);
-extern void ProcSignalInit(int pss_idx);
+extern void ProcSignalInit(void);
extern int SendProcSignal(pid_t pid, ProcSignalReason reason,
BackendId backendId);
diff --git a/src/include/storage/sinvaladt.h b/src/include/storage/sinvaladt.h
index aa3d203efca..c3c97b3f8b7 100644
--- a/src/include/storage/sinvaladt.h
+++ b/src/include/storage/sinvaladt.h
@@ -31,10 +31,6 @@
extern Size SInvalShmemSize(void);
extern void CreateSharedInvalidationState(void);
extern void SharedInvalBackendInit(bool sendOnly);
-extern PGPROC *BackendIdGetProc(int backendID);
-extern void BackendIdGetTransactionIds(int backendID, TransactionId *xid,
- TransactionId *xmin, int *nsubxid,
- bool *overflowed);
extern void SIInsertDataEntries(const SharedInvalidationMessage *data, int n);
extern int SIGetDataEntries(SharedInvalidationMessage *data, int datasize);
diff --git a/src/pl/plpgsql/src/pl_exec.c b/src/pl/plpgsql/src/pl_exec.c
index 6d1691340c5..ed51694428a 100644
--- a/src/pl/plpgsql/src/pl_exec.c
+++ b/src/pl/plpgsql/src/pl_exec.c
@@ -2211,7 +2211,7 @@ exec_stmt_call(PLpgSQL_execstate *estate, PLpgSQL_stmt_call *stmt)
paramLI = setup_param_list(estate, expr);
- before_lxid = MyProc->lxid;
+ before_lxid = MyProc->vxid.lxid;
/*
* If we have a procedure-lifespan resowner, use that to hold the refcount
@@ -2232,7 +2232,7 @@ exec_stmt_call(PLpgSQL_execstate *estate, PLpgSQL_stmt_call *stmt)
elog(ERROR, "SPI_execute_plan_extended failed executing query \"%s\": %s",
expr->query, SPI_result_code_string(rc));
- after_lxid = MyProc->lxid;
+ after_lxid = MyProc->vxid.lxid;
if (before_lxid != after_lxid)
{
@@ -6037,7 +6037,7 @@ exec_eval_simple_expr(PLpgSQL_execstate *estate,
int32 *rettypmod)
{
ExprContext *econtext = estate->eval_econtext;
- LocalTransactionId curlxid = MyProc->lxid;
+ LocalTransactionId curlxid = MyProc->vxid.lxid;
ParamListInfo paramLI;
void *save_setup_arg;
bool need_snapshot;
@@ -7943,7 +7943,7 @@ get_cast_hashentry(PLpgSQL_execstate *estate,
* functions do; DO blocks have private simple_eval_estates, and private
* cast hash tables to go with them.)
*/
- curlxid = MyProc->lxid;
+ curlxid = MyProc->vxid.lxid;
if (cast_entry->cast_lxid != curlxid || cast_entry->cast_in_use)
{
oldcontext = MemoryContextSwitchTo(estate->simple_eval_estate->es_query_cxt);
@@ -8070,7 +8070,7 @@ exec_simple_check_plan(PLpgSQL_execstate *estate, PLpgSQL_expr *expr)
/* Remember that we have the refcount */
expr->expr_simple_plansource = plansource;
expr->expr_simple_plan = cplan;
- expr->expr_simple_plan_lxid = MyProc->lxid;
+ expr->expr_simple_plan_lxid = MyProc->vxid.lxid;
/* Share the remaining work with the replan code path */
exec_save_simple_expr(expr, cplan);