On 07/02/2024 20:25, Andres Freund wrote:
On 2024-01-30 02:08:36 +0200, Heikki Linnakangas wrote:
 From 54f22231bb2540fc5957c14005956161e6fc9dac Mon Sep 17 00:00:00 2001
From: Heikki Linnakangas <heikki.linnakan...@iki.fi>
Date: Wed, 24 Jan 2024 23:15:55 +0200
Subject: [PATCH v8 1/5] Remove superfluous 'pgprocno' field from PGPROC

It was always just the index of the PGPROC entry from the beginning of
the proc array. Introduce a macro to compute it from the pointer
instead.

Hm. The pointer math here is bit more expensive than in some other cases, as
the struct is fairly large and sizeof(PGPROC) isn't a power of two. Adding
more math into loops like in TransactionGroupUpdateXidStatus() might end up
showing up.

I added a MyProcNumber global variable that is set to GetNumberFromPGProc(MyProc). I'm not really concerned about the extra math, but with MyProcNumber it should definitely not be an issue. The few GetNumberFromPGProc() invocations that remain are in less performance-critical paths.

(In later patch, I switch backend ids to 0-based indexing, which replaces MyProcNumber references with MyBackendId)

Is this really related to the rest of the series?

It's not strictly necessary, but it felt prudent to remove it now, since I'm removing the backendID field too.

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.

Why aren't we using 0-based indexing for both? InvalidBackendId is -1, so
there'd not be a conflict, right?

Correct. I was being conservative and didn't dare to change the old convention. The backend ids are visible in a few places like "pg_temp_0" schema names, and pg_stat_get_*() functions.

One alternative would be to reserve and waste allProcs[0]. Then pgprocno and backend ID could both be direct indexes to the array, but 0 would not be used.

If we switch to 0-based indexing, it begs the question: why don't we merge the concepts of "pgprocno" and "BackendId" completely and call it the same thing everywhere? It probably would be best in the long run. It feels like a lot of churn though.

Anyway, I switched to 0-based indexing in the attached new version, to see what it looks like.

@@ -457,7 +442,7 @@ 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));

This set of changes is independent of this commit, isn't it?

Yes. It's just for symmetry, now that we use GetNumberFromPGProc() to get the pgprocno.

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();

Now that we don't need the offset here, we could move ProcSignalInit() into
BsaeInit() I think?

Hmm, doesn't feel right to me. BaseInit() is mostly concerned with setting up backend-private structures, and it's also called for a standalone backend.

I feel the process initialization codepaths could use some cleanup in general. Not sure what exactly.

+/*
+ * 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;

Hm, doesn't calling BackendIdGetProc() with these values a bug? That's not
about being out of date or such.

Perhaps. I just followed the example of the old implementation, which also returns NULL on bogus inputs.

+/*
+ * 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);
+}

Hm, I'm not sure about the locking here. For one, previously we weren't
holding ProcArrayLock. For another, holding ProcArrayLock guarantees that the
backend doesn't end its transaction, but it can still assign xids etc.  And,
for that matter, the backendid could have been recycled between the caller
acquiring the backendId and calling BackendIdGetTransactionIds().

Yeah, the returned values could be out-of-date and even inconsistent with each other. I just faithfully copied the old implementation.

Perhaps this should just skip the ProcArrayLock altogether.

--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -3074,18 +3074,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)

Doesn't this mean we'll include a vxid in more cases now, particularly
including aux processes? That might be ok, but I also suspect that it'll never
have meaningful values...

Fixed. (I thought I changed that back already in the last patch version, but apparently I only did it in jsonlog.c)

 From 94fd46c9ef30ba5e8ac1a8873fce577a4be425f4 Mon Sep 17 00:00:00 2001
From: Heikki Linnakangas <heikki.linnakan...@iki.fi>
Date: Mon, 29 Jan 2024 22:57:49 +0200
Subject: [PATCH v8 3/5] Replace 'lastBackend' with an array of in-use slots

Now that the procState array is indexed by pgprocno, the 'lastBackend'
optimization is useless, because aux processes are assigned PGPROC
slots and hence have numbers higher than max_connection. So
'lastBackend' was always set to almost the end of the array.

To replace that optimization, mantain a dense array of in-use
indexes. This's redundant with ProgGlobal->procarray, but I was afraid
of adding any more contention to ProcArrayLock, and this keeps the
code isolated to sinvaladt.c too.

I think it'd be good to include that explanation and justification in the code
as well.

Added a comment.


Attached is a new version of these BackendId changes. I kept it as three separate patches to highlight the changes from switching to 0-based indexing, but I think they should be squashed together before pushing.

I think the last remaining question here is about the 0- vs 1-based indexing of BackendIds. Is it a good idea to switch to 0-based indexing? And if we do it, should we reserve PGPROC 0. I'm on the fence on this one.

And if we switch to 0-based indexing, should we do a more comprehensive search & replace of "pgprocno" to "backendId", or something like that. My vote is no, the code churn doesn't feel worth it. And it can also be done separately later if we want to.

--
Heikki Linnakangas
Neon (https://neon.tech)
From 2f0649151e28e9f2fdd572f7c4e329991bdc2570 Mon Sep 17 00:00:00 2001
From: Heikki Linnakangas <heikki.linnakan...@iki.fi>
Date: Thu, 8 Feb 2024 12:57:39 +0200
Subject: [PATCH v10 1/3] Remove superfluous 'pgprocno' field from PGPROC

It was always just the index of the PGPROC entry from the beginning of
the proc array. Introduce a macro to compute it from the pointer
instead.

Reviewed-by: Andres Freund
Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407%40iki.fi
---
 src/backend/access/transam/clog.c             |  5 +++--
 src/backend/access/transam/twophase.c         | 11 +++++------
 src/backend/access/transam/xlog.c             |  2 +-
 src/backend/postmaster/bgwriter.c             |  2 +-
 src/backend/postmaster/pgarch.c               |  2 +-
 src/backend/postmaster/walsummarizer.c        |  2 +-
 src/backend/storage/buffer/bufmgr.c           |  6 +++---
 src/backend/storage/ipc/procarray.c           | 14 ++++++++------
 src/backend/storage/lmgr/condition_variable.c | 12 ++++++------
 src/backend/storage/lmgr/lwlock.c             |  6 +++---
 src/backend/storage/lmgr/predicate.c          |  2 +-
 src/backend/storage/lmgr/proc.c               |  7 ++++++-
 src/include/storage/lock.h                    |  2 +-
 src/include/storage/proc.h                    |  7 ++-----
 14 files changed, 42 insertions(+), 38 deletions(-)

diff --git a/src/backend/access/transam/clog.c b/src/backend/access/transam/clog.c
index 06fc2989bab..97f7434da34 100644
--- a/src/backend/access/transam/clog.c
+++ b/src/backend/access/transam/clog.c
@@ -425,6 +425,7 @@ TransactionGroupUpdateXidStatus(TransactionId xid, XidStatus status,
 {
 	volatile PROC_HDR *procglobal = ProcGlobal;
 	PGPROC	   *proc = MyProc;
+	int			pgprocno = MyProcNumber;
 	uint32		nextidx;
 	uint32		wakeidx;
 
@@ -458,7 +459,7 @@ TransactionGroupUpdateXidStatus(TransactionId xid, XidStatus status,
 		 * less efficiently.
 		 */
 		if (nextidx != INVALID_PGPROCNO &&
-			ProcGlobal->allProcs[nextidx].clogGroupMemberPage != proc->clogGroupMemberPage)
+			GetPGProcByNumber(nextidx)->clogGroupMemberPage != proc->clogGroupMemberPage)
 		{
 			/*
 			 * Ensure that this proc is not a member of any clog group that
@@ -473,7 +474,7 @@ TransactionGroupUpdateXidStatus(TransactionId xid, XidStatus status,
 
 		if (pg_atomic_compare_exchange_u32(&procglobal->clogGroupFirst,
 										   &nextidx,
-										   (uint32) proc->pgprocno))
+										   (uint32) pgprocno))
 			break;
 	}
 
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 8426458f7f5..234c8d08ebc 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -284,7 +284,7 @@ TwoPhaseShmemInit(void)
 			TwoPhaseState->freeGXacts = &gxacts[i];
 
 			/* associate it with a PGPROC assigned by InitProcGlobal */
-			gxacts[i].pgprocno = PreparedXactProcs[i].pgprocno;
+			gxacts[i].pgprocno = GetNumberFromPGProc(&PreparedXactProcs[i]);
 
 			/*
 			 * Assign a unique ID for each dummy proc, so that the range of
@@ -461,7 +461,6 @@ MarkAsPreparingGuts(GlobalTransaction gxact, TransactionId xid, const char *gid,
 
 	/* Initialize the PGPROC entry */
 	MemSet(proc, 0, sizeof(PGPROC));
-	proc->pgprocno = gxact->pgprocno;
 	dlist_node_init(&proc->links);
 	proc->waitStatus = PROC_WAIT_STATUS_OK;
 	if (LocalTransactionIdIsValid(MyProc->lxid))
@@ -780,7 +779,7 @@ pg_prepared_xact(PG_FUNCTION_ARGS)
 	while (status->array != NULL && status->currIdx < status->ngxacts)
 	{
 		GlobalTransaction gxact = &status->array[status->currIdx++];
-		PGPROC	   *proc = &ProcGlobal->allProcs[gxact->pgprocno];
+		PGPROC	   *proc = GetPGProcByNumber(gxact->pgprocno);
 		Datum		values[5] = {0};
 		bool		nulls[5] = {0};
 		HeapTuple	tuple;
@@ -935,7 +934,7 @@ TwoPhaseGetDummyProc(TransactionId xid, bool lock_held)
 {
 	GlobalTransaction gxact = TwoPhaseGetGXact(xid, lock_held);
 
-	return &ProcGlobal->allProcs[gxact->pgprocno];
+	return GetPGProcByNumber(gxact->pgprocno);
 }
 
 /************************************************************************/
@@ -1080,7 +1079,7 @@ save_state_data(const void *data, uint32 len)
 void
 StartPrepare(GlobalTransaction gxact)
 {
-	PGPROC	   *proc = &ProcGlobal->allProcs[gxact->pgprocno];
+	PGPROC	   *proc = GetPGProcByNumber(gxact->pgprocno);
 	TransactionId xid = gxact->xid;
 	TwoPhaseFileHeader hdr;
 	TransactionId *children;
@@ -1539,7 +1538,7 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	 * try to commit the same GID at once.
 	 */
 	gxact = LockGXact(gid, GetUserId());
-	proc = &ProcGlobal->allProcs[gxact->pgprocno];
+	proc = GetPGProcByNumber(gxact->pgprocno);
 	xid = gxact->xid;
 
 	/*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 478377c4a23..1e4abbca8be 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -1379,7 +1379,7 @@ WALInsertLockAcquire(void)
 	static int	lockToTry = -1;
 
 	if (lockToTry == -1)
-		lockToTry = MyProc->pgprocno % NUM_XLOGINSERT_LOCKS;
+		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
 	MyLockNo = lockToTry;
 
 	/*
diff --git a/src/backend/postmaster/bgwriter.c b/src/backend/postmaster/bgwriter.c
index f11ce27084e..6364b16261f 100644
--- a/src/backend/postmaster/bgwriter.c
+++ b/src/backend/postmaster/bgwriter.c
@@ -326,7 +326,7 @@ BackgroundWriterMain(void)
 		if (rc == WL_TIMEOUT && can_hibernate && prev_hibernate)
 		{
 			/* Ask for notification at next buffer allocation */
-			StrategyNotifyBgWriter(MyProc->pgprocno);
+			StrategyNotifyBgWriter(MyProcNumber);
 			/* Sleep ... */
 			(void) WaitLatch(MyLatch,
 							 WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
diff --git a/src/backend/postmaster/pgarch.c b/src/backend/postmaster/pgarch.c
index 67693b05806..2aa3e9890a2 100644
--- a/src/backend/postmaster/pgarch.c
+++ b/src/backend/postmaster/pgarch.c
@@ -242,7 +242,7 @@ PgArchiverMain(void)
 	 * Advertise our pgprocno so that backends can use our latch to wake us up
 	 * while we're sleeping.
 	 */
-	PgArch->pgprocno = MyProc->pgprocno;
+	PgArch->pgprocno = MyProcNumber;
 
 	/* Create workspace for pgarch_readyXlog() */
 	arch_files = palloc(sizeof(struct arch_files_state));
diff --git a/src/backend/postmaster/walsummarizer.c b/src/backend/postmaster/walsummarizer.c
index 9b883c21ca4..5d61d1517e8 100644
--- a/src/backend/postmaster/walsummarizer.c
+++ b/src/backend/postmaster/walsummarizer.c
@@ -248,7 +248,7 @@ WalSummarizerMain(void)
 	/* Advertise ourselves. */
 	on_shmem_exit(WalSummarizerShutdown, (Datum) 0);
 	LWLockAcquire(WALSummarizerLock, LW_EXCLUSIVE);
-	WalSummarizerCtl->summarizer_pgprocno = MyProc->pgprocno;
+	WalSummarizerCtl->summarizer_pgprocno = MyProcNumber;
 	LWLockRelease(WALSummarizerLock);
 
 	/* Create and switch to a memory context that we can reset on error. */
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index eb1ec3b86df..87a31745864 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -4780,7 +4780,7 @@ UnlockBuffers(void)
 		 * got a cancel/die interrupt before getting the signal.
 		 */
 		if ((buf_state & BM_PIN_COUNT_WAITER) != 0 &&
-			buf->wait_backend_pgprocno == MyProc->pgprocno)
+			buf->wait_backend_pgprocno == MyProcNumber)
 			buf_state &= ~BM_PIN_COUNT_WAITER;
 
 		UnlockBufHdr(buf, buf_state);
@@ -4930,7 +4930,7 @@ LockBufferForCleanup(Buffer buffer)
 			LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 			elog(ERROR, "multiple backends attempting to wait for pincount 1");
 		}
-		bufHdr->wait_backend_pgprocno = MyProc->pgprocno;
+		bufHdr->wait_backend_pgprocno = MyProcNumber;
 		PinCountWaitBuf = bufHdr;
 		buf_state |= BM_PIN_COUNT_WAITER;
 		UnlockBufHdr(bufHdr, buf_state);
@@ -4994,7 +4994,7 @@ LockBufferForCleanup(Buffer buffer)
 		 */
 		buf_state = LockBufHdr(bufHdr);
 		if ((buf_state & BM_PIN_COUNT_WAITER) != 0 &&
-			bufHdr->wait_backend_pgprocno == MyProc->pgprocno)
+			bufHdr->wait_backend_pgprocno == MyProcNumber)
 			buf_state &= ~BM_PIN_COUNT_WAITER;
 		UnlockBufHdr(bufHdr, buf_state);
 
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index ee2d7f8585a..dd329a86ef4 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -468,6 +468,7 @@ CreateSharedProcArray(void)
 void
 ProcArrayAdd(PGPROC *proc)
 {
+	int			pgprocno = GetNumberFromPGProc(proc);
 	ProcArrayStruct *arrayP = procArray;
 	int			index;
 	int			movecount;
@@ -499,13 +500,13 @@ ProcArrayAdd(PGPROC *proc)
 	 */
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		int			procno PG_USED_FOR_ASSERTS_ONLY = arrayP->pgprocnos[index];
+		int			this_procno = arrayP->pgprocnos[index];
 
-		Assert(procno >= 0 && procno < (arrayP->maxProcs + NUM_AUXILIARY_PROCS));
-		Assert(allProcs[procno].pgxactoff == index);
+		Assert(this_procno >= 0 && this_procno < (arrayP->maxProcs + NUM_AUXILIARY_PROCS));
+		Assert(allProcs[this_procno].pgxactoff == index);
 
 		/* If we have found our right position in the array, break */
-		if (arrayP->pgprocnos[index] > proc->pgprocno)
+		if (this_procno > pgprocno)
 			break;
 	}
 
@@ -523,7 +524,7 @@ ProcArrayAdd(PGPROC *proc)
 			&ProcGlobal->statusFlags[index],
 			movecount * sizeof(*ProcGlobal->statusFlags));
 
-	arrayP->pgprocnos[index] = proc->pgprocno;
+	arrayP->pgprocnos[index] = GetNumberFromPGProc(proc);
 	proc->pgxactoff = index;
 	ProcGlobal->xids[index] = proc->xid;
 	ProcGlobal->subxidStates[index] = proc->subxidStatus;
@@ -791,6 +792,7 @@ ProcArrayEndTransactionInternal(PGPROC *proc, TransactionId latestXid)
 static void
 ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
 {
+	int			pgprocno = GetNumberFromPGProc(proc);
 	PROC_HDR   *procglobal = ProcGlobal;
 	uint32		nextidx;
 	uint32		wakeidx;
@@ -808,7 +810,7 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
 
 		if (pg_atomic_compare_exchange_u32(&procglobal->procArrayGroupFirst,
 										   &nextidx,
-										   (uint32) proc->pgprocno))
+										   (uint32) pgprocno))
 			break;
 	}
 
diff --git a/src/backend/storage/lmgr/condition_variable.c b/src/backend/storage/lmgr/condition_variable.c
index e2d6d685220..10fdae19dcc 100644
--- a/src/backend/storage/lmgr/condition_variable.c
+++ b/src/backend/storage/lmgr/condition_variable.c
@@ -57,7 +57,7 @@ ConditionVariableInit(ConditionVariable *cv)
 void
 ConditionVariablePrepareToSleep(ConditionVariable *cv)
 {
-	int			pgprocno = MyProc->pgprocno;
+	int			pgprocno = MyProcNumber;
 
 	/*
 	 * If some other sleep is already prepared, cancel it; this is necessary
@@ -181,10 +181,10 @@ ConditionVariableTimedSleep(ConditionVariable *cv, long timeout,
 		 * guarantee not to return spuriously, we'll avoid this obvious case.
 		 */
 		SpinLockAcquire(&cv->mutex);
-		if (!proclist_contains(&cv->wakeup, MyProc->pgprocno, cvWaitLink))
+		if (!proclist_contains(&cv->wakeup, MyProcNumber, cvWaitLink))
 		{
 			done = true;
-			proclist_push_tail(&cv->wakeup, MyProc->pgprocno, cvWaitLink);
+			proclist_push_tail(&cv->wakeup, MyProcNumber, cvWaitLink);
 		}
 		SpinLockRelease(&cv->mutex);
 
@@ -236,8 +236,8 @@ ConditionVariableCancelSleep(void)
 		return false;
 
 	SpinLockAcquire(&cv->mutex);
-	if (proclist_contains(&cv->wakeup, MyProc->pgprocno, cvWaitLink))
-		proclist_delete(&cv->wakeup, MyProc->pgprocno, cvWaitLink);
+	if (proclist_contains(&cv->wakeup, MyProcNumber, cvWaitLink))
+		proclist_delete(&cv->wakeup, MyProcNumber, cvWaitLink);
 	else
 		signaled = true;
 	SpinLockRelease(&cv->mutex);
@@ -281,7 +281,7 @@ ConditionVariableSignal(ConditionVariable *cv)
 void
 ConditionVariableBroadcast(ConditionVariable *cv)
 {
-	int			pgprocno = MyProc->pgprocno;
+	int			pgprocno = MyProcNumber;
 	PGPROC	   *proc = NULL;
 	bool		have_sentinel = false;
 
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 71677cf0317..997857679ed 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -1056,9 +1056,9 @@ LWLockQueueSelf(LWLock *lock, LWLockMode mode)
 
 	/* LW_WAIT_UNTIL_FREE waiters are always at the front of the queue */
 	if (mode == LW_WAIT_UNTIL_FREE)
-		proclist_push_head(&lock->waiters, MyProc->pgprocno, lwWaitLink);
+		proclist_push_head(&lock->waiters, MyProcNumber, lwWaitLink);
 	else
-		proclist_push_tail(&lock->waiters, MyProc->pgprocno, lwWaitLink);
+		proclist_push_tail(&lock->waiters, MyProcNumber, lwWaitLink);
 
 	/* Can release the mutex now */
 	LWLockWaitListUnlock(lock);
@@ -1097,7 +1097,7 @@ LWLockDequeueSelf(LWLock *lock)
 	 */
 	on_waitlist = MyProc->lwWaiting == LW_WS_WAITING;
 	if (on_waitlist)
-		proclist_delete(&lock->waiters, MyProc->pgprocno, lwWaitLink);
+		proclist_delete(&lock->waiters, MyProcNumber, lwWaitLink);
 
 	if (proclist_is_empty(&lock->waiters) &&
 		(pg_atomic_read_u32(&lock->state) & LW_FLAG_HAS_WAITERS) != 0)
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index eed63a05ed8..d62060d58c8 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1830,7 +1830,7 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
 	sxact->finishedBefore = InvalidTransactionId;
 	sxact->xmin = snapshot->xmin;
 	sxact->pid = MyProcPid;
-	sxact->pgprocno = MyProc->pgprocno;
+	sxact->pgprocno = MyProcNumber;
 	dlist_init(&sxact->predicateLocks);
 	dlist_node_init(&sxact->finishedLink);
 	sxact->flags = 0;
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index e5977548fe2..fddf44015fd 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -64,6 +64,7 @@ bool		log_lock_waits = false;
 
 /* Pointer to this process's PGPROC struct, if any */
 PGPROC	   *MyProc = NULL;
+int			MyProcNumber = INVALID_PGPROCNO;
 
 /*
  * This spinlock protects the freelist of recycled PGPROC structures.
@@ -227,7 +228,6 @@ InitProcGlobal(void)
 			InitSharedLatch(&(proc->procLatch));
 			LWLockInitialize(&(proc->fpInfoLock), LWTRANCHE_LOCK_FASTPATH);
 		}
-		proc->pgprocno = i;
 
 		/*
 		 * Newly created PGPROCs for normal backends, autovacuum and bgworkers
@@ -352,6 +352,7 @@ InitProcess(void)
 				(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
@@ -565,6 +566,8 @@ InitAuxiliaryProcess(void)
 
 	SpinLockRelease(ProcStructLock);
 
+	MyProcNumber = GetNumberFromPGProc(MyProc);
+
 	/*
 	 * Initialize all fields of MyProc, except for those previously
 	 * initialized by InitProcGlobal.
@@ -906,6 +909,7 @@ ProcKill(int code, Datum arg)
 
 	proc = MyProc;
 	MyProc = NULL;
+	MyProcNumber = INVALID_PGPROCNO;
 	DisownLatch(&proc->procLatch);
 
 	procgloballist = proc->procgloballist;
@@ -977,6 +981,7 @@ AuxiliaryProcKill(int code, Datum arg)
 
 	proc = MyProc;
 	MyProc = NULL;
+	MyProcNumber = INVALID_PGPROCNO;
 	DisownLatch(&proc->procLatch);
 
 	SpinLockAcquire(ProcStructLock);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 00679624f7d..ed6071f3286 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -540,7 +540,7 @@ typedef enum
  * used for a given lock group is determined by the group leader's pgprocno.
  */
 #define LockHashPartitionLockByProc(leader_pgproc) \
-	LockHashPartitionLock((leader_pgproc)->pgprocno)
+	LockHashPartitionLock(GetNumberFromPGProc(leader_pgproc))
 
 /*
  * function prototypes
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 4bc226e36cd..ca21518f522 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -194,11 +194,6 @@ struct PGPROC
 	int			pgxactoff;		/* offset into various ProcGlobal->arrays with
 								 * data mirrored from this PGPROC */
 
-	int			pgprocno;		/* Number of this PGPROC in
-								 * ProcGlobal->allProcs array. This is set
-								 * once by InitProcGlobal().
-								 * ProcGlobal->allProcs[n].pgprocno == n */
-
 	/* 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 */
@@ -307,6 +302,7 @@ struct PGPROC
 
 
 extern PGDLLIMPORT PGPROC *MyProc;
+extern PGDLLIMPORT int MyProcNumber;	/* same as GetNumberFromPGProc(MyProc) */
 
 /*
  * There is one ProcGlobal struct for the whole database cluster.
@@ -412,6 +408,7 @@ extern PGDLLIMPORT PGPROC *PreparedXactProcs;
 
 /* Accessor for PGPROC given a pgprocno. */
 #define GetPGProcByNumber(n) (&ProcGlobal->allProcs[(n)])
+#define GetNumberFromPGProc(proc) ((proc) - &ProcGlobal->allProcs[0])
 
 /*
  * We set aside some extra PGPROC structures for auxiliary processes,
-- 
2.39.2

From d4cf14da8400cfb43d8aee1592f8af3486fa8de9 Mon Sep 17 00:00:00 2001
From: Heikki Linnakangas <heikki.linnakan...@iki.fi>
Date: Thu, 8 Feb 2024 12:57:54 +0200
Subject: [PATCH v10 2/3] 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 becomes 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.

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
Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b...@iki.fi
---
 src/backend/access/transam/twophase.c       |  37 ++--
 src/backend/access/transam/xact.c           |  12 +-
 src/backend/catalog/namespace.c             |   2 +-
 src/backend/commands/sequence.c             |   2 +-
 src/backend/executor/functions.c            |   4 +-
 src/backend/postmaster/auxprocess.c         |  12 +-
 src/backend/storage/ipc/procarray.c         |  73 ++++++-
 src/backend/storage/ipc/procsignal.c        |  27 ++-
 src/backend/storage/ipc/sinvaladt.c         | 200 ++++++--------------
 src/backend/storage/ipc/standby.c           |   1 +
 src/backend/storage/lmgr/lock.c             |  28 +--
 src/backend/storage/lmgr/proc.c             |  30 +--
 src/backend/utils/activity/backend_status.c |  52 ++---
 src/backend/utils/adt/lockfuncs.c           |   2 +-
 src/backend/utils/adt/mcxtfuncs.c           |  14 +-
 src/backend/utils/error/csvlog.c            |   4 +-
 src/backend/utils/error/elog.c              |   6 +-
 src/backend/utils/error/jsonlog.c           |   6 +-
 src/backend/utils/init/postinit.c           |  10 +-
 src/backend/utils/time/snapmgr.c            |   5 +-
 src/include/miscadmin.h                     |   2 -
 src/include/storage/backendid.h             |  12 +-
 src/include/storage/lock.h                  |   9 +-
 src/include/storage/proc.h                  |  32 +++-
 src/include/storage/procarray.h             |   4 +
 src/include/storage/procsignal.h            |   2 +-
 src/include/storage/sinvaladt.h             |   4 -
 src/pl/plpgsql/src/pl_exec.c                |  10 +-
 28 files changed, 280 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 464858117e0..f791340d3df 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);
 
@@ -2281,7 +2281,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
@@ -2824,7 +2824,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 f7803744a5b..f99dc48e8cf 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -1076,7 +1076,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;
 
 		/*
@@ -3099,6 +3104,64 @@ HaveVirtualXIDsDelayingChkpt(VirtualTransactionId *vxids, int nvxids, int type)
 	return result;
 }
 
+/*
+ * 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
  *
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index e84619e5a58..d1d5bf0c152 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, (int) (slot - ProcSignal->psh_slot));
 
 	/* 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 fddf44015fd..4226e41d80c 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -240,25 +240,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;
 		}
 
@@ -353,6 +353,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
@@ -375,14 +376,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;
@@ -562,11 +563,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
@@ -574,12 +575,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;
@@ -910,8 +911,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);
 
@@ -982,6 +989,7 @@ AuxiliaryProcKill(int code, Datum arg)
 	proc = MyProc;
 	MyProc = NULL;
 	MyProcNumber = INVALID_PGPROCNO;
+	MyBackendId = InvalidBackendId;
 	DisownLatch(&proc->procLatch);
 
 	SpinLockAcquire(ProcStructLock);
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 700fbde6db4..208e1bc0ee4 100644
--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -3078,18 +3078,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 1ad33671598..e822cd61d5f 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -740,18 +740,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 0b01c1f0935..cbdc61b8576 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -444,8 +444,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..c5c84201378 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;	/* local 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 ca21518f522..d1e50380b7c 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. */
+/*
+ * 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);
-- 
2.39.2

From b00703d869aaf5c0e76f6e8a15adeb1e8b99fbc1 Mon Sep 17 00:00:00 2001
From: Heikki Linnakangas <heikki.linnakan...@iki.fi>
Date: Thu, 8 Feb 2024 12:58:08 +0200
Subject: [PATCH v10 3/3] Use 0-based indexing for BackendIds

---
 doc/src/sgml/monitoring.sgml                  |  2 +-
 src/backend/access/transam/clog.c             |  2 +-
 src/backend/access/transam/multixact.c        | 19 ++++++++-----------
 src/backend/access/transam/twophase.c         |  4 ++--
 src/backend/access/transam/xlog.c             |  2 +-
 src/backend/commands/async.c                  | 12 ++++++------
 src/backend/postmaster/bgwriter.c             |  2 +-
 src/backend/postmaster/pgarch.c               |  2 +-
 src/backend/postmaster/walsummarizer.c        |  2 +-
 src/backend/storage/buffer/bufmgr.c           |  6 +++---
 src/backend/storage/ipc/procarray.c           | 10 +++++-----
 src/backend/storage/ipc/procsignal.c          |  8 ++++----
 src/backend/storage/ipc/sinvaladt.c           | 19 ++++++++-----------
 src/backend/storage/lmgr/condition_variable.c | 12 ++++++------
 src/backend/storage/lmgr/lwlock.c             |  6 +++---
 src/backend/storage/lmgr/predicate.c          |  2 +-
 src/backend/storage/lmgr/proc.c               |  9 ++-------
 src/backend/utils/activity/backend_status.c   | 12 ++++++------
 src/backend/utils/adt/mcxtfuncs.c             |  2 +-
 src/include/storage/backendid.h               |  6 ++++--
 src/include/storage/proc.h                    | 11 +++--------
 21 files changed, 68 insertions(+), 82 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5cf9363ac82..adec1b92a31 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -4933,7 +4933,7 @@ description | Waiting for a newly initialized WAL file to reach durable storage
    access functions can be used; these are shown in <xref
    linkend="monitoring-stats-backend-funcs-table"/>.
    These access functions use the session's backend ID number, which is a
-   small positive integer that is distinct from the backend ID of any
+   small non-negative integer that is distinct from the backend ID of any
    concurrent session, although a session's ID can be recycled as soon as
    it exits.  The backend ID is used, among other things, to identify the
    session's temporary schema if it has one.
diff --git a/src/backend/access/transam/clog.c b/src/backend/access/transam/clog.c
index 97f7434da34..7b79067efb1 100644
--- a/src/backend/access/transam/clog.c
+++ b/src/backend/access/transam/clog.c
@@ -425,7 +425,7 @@ TransactionGroupUpdateXidStatus(TransactionId xid, XidStatus status,
 {
 	volatile PROC_HDR *procglobal = ProcGlobal;
 	PGPROC	   *proc = MyProc;
-	int			pgprocno = MyProcNumber;
+	int			pgprocno = MyBackendId;
 	uint32		nextidx;
 	uint32		wakeidx;
 
diff --git a/src/backend/access/transam/multixact.c b/src/backend/access/transam/multixact.c
index febc429f724..f577008f4bd 100644
--- a/src/backend/access/transam/multixact.c
+++ b/src/backend/access/transam/multixact.c
@@ -238,9 +238,8 @@ typedef struct MultiXactStateData
 	 * immediately following the MultiXactStateData struct. Each is indexed by
 	 * BackendId.
 	 *
-	 * In both arrays, there's a slot for all normal backends (1..MaxBackends)
-	 * followed by a slot for max_prepared_xacts prepared transactions. Valid
-	 * BackendIds start from 1; element zero of each array is never used.
+	 * In both arrays, there's a slot for all normal backends (0..MaxBackends-1)
+	 * followed by a slot for max_prepared_xacts prepared transactions.
 	 *
 	 * OldestMemberMXactId[k] is the oldest MultiXactId each backend's current
 	 * transaction(s) could possibly be a member of, or InvalidMultiXactId
@@ -284,8 +283,7 @@ typedef struct MultiXactStateData
 } MultiXactStateData;
 
 /*
- * Last element of OldestMemberMXactId and OldestVisibleMXactId arrays.
- * Valid elements are (1..MaxOldestSlot); element 0 is never used.
+ * Size of OldestMemberMXactId and OldestVisibleMXactId arrays.
  */
 #define MaxOldestSlot	(MaxBackends + max_prepared_xacts)
 
@@ -698,7 +696,7 @@ MultiXactIdSetOldestVisible(void)
 		if (oldestMXact < FirstMultiXactId)
 			oldestMXact = FirstMultiXactId;
 
-		for (i = 1; i <= MaxOldestSlot; i++)
+		for (i = 0; i < MaxOldestSlot; i++)
 		{
 			MultiXactId thisoldest = OldestMemberMXactId[i];
 
@@ -1828,9 +1826,9 @@ MultiXactShmemSize(void)
 {
 	Size		size;
 
-	/* We need 2*MaxOldestSlot + 1 perBackendXactIds[] entries */
+	/* We need 2*MaxOldestSlot perBackendXactIds[] entries */
 #define SHARED_MULTIXACT_STATE_SIZE \
-	add_size(offsetof(MultiXactStateData, perBackendXactIds) + sizeof(MultiXactId), \
+	add_size(offsetof(MultiXactStateData, perBackendXactIds), \
 			 mul_size(sizeof(MultiXactId) * 2, MaxOldestSlot))
 
 	size = SHARED_MULTIXACT_STATE_SIZE;
@@ -1880,8 +1878,7 @@ MultiXactShmemInit(void)
 		Assert(found);
 
 	/*
-	 * Set up array pointers.  Note that perBackendXactIds[0] is wasted space
-	 * since we only use indexes 1..MaxOldestSlot in each array.
+	 * Set up array pointers.
 	 */
 	OldestMemberMXactId = MultiXactState->perBackendXactIds;
 	OldestVisibleMXactId = OldestMemberMXactId + MaxOldestSlot;
@@ -2533,7 +2530,7 @@ GetOldestMultiXactId(void)
 		nextMXact = FirstMultiXactId;
 
 	oldestMXact = nextMXact;
-	for (i = 1; i <= MaxOldestSlot; i++)
+	for (i = 0; i < MaxOldestSlot; i++)
 	{
 		MultiXactId thisoldest;
 
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 5c282002900..709353d4056 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -895,7 +895,7 @@ TwoPhaseGetXidByVirtualXID(VirtualTransactionId vxid,
  *		Get the dummy backend ID for prepared transaction specified by XID
  *
  * Dummy backend IDs are similar to real backend IDs of real backends.
- * They start at MaxBackends + 1, and are unique across all currently active
+ * They start at MaxBackends, and are unique across all currently active
  * real backends and prepared transactions.  If lock_held is set to true,
  * TwoPhaseStateLock will not be taken, so the caller had better hold it.
  */
@@ -904,7 +904,7 @@ TwoPhaseGetDummyBackendId(TransactionId xid, bool lock_held)
 {
 	GlobalTransaction gxact = TwoPhaseGetGXact(xid, lock_held);
 
-	return gxact->pgprocno + 1;
+	return gxact->pgprocno;
 }
 
 /*
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1e4abbca8be..e401684f82f 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -1379,7 +1379,7 @@ WALInsertLockAcquire(void)
 	static int	lockToTry = -1;
 
 	if (lockToTry == -1)
-		lockToTry = MyProcNumber % NUM_XLOGINSERT_LOCKS;
+		lockToTry = MyBackendId % NUM_XLOGINSERT_LOCKS;
 	MyLockNo = lockToTry;
 
 	/*
diff --git a/src/backend/commands/async.c b/src/backend/commands/async.c
index 8b24b222931..5aa893c9141 100644
--- a/src/backend/commands/async.c
+++ b/src/backend/commands/async.c
@@ -1094,7 +1094,7 @@ Exec_ListenPreCommit(void)
 	head = QUEUE_HEAD;
 	max = QUEUE_TAIL;
 	prevListener = InvalidBackendId;
-	for (BackendId i = QUEUE_FIRST_LISTENER; i > 0; i = QUEUE_NEXT_LISTENER(i))
+	for (BackendId i = QUEUE_FIRST_LISTENER; i != InvalidBackendId; i = QUEUE_NEXT_LISTENER(i))
 	{
 		if (QUEUE_BACKEND_DBOID(i) == MyDatabaseId)
 			max = QUEUE_POS_MAX(max, QUEUE_BACKEND_POS(i));
@@ -1106,7 +1106,7 @@ Exec_ListenPreCommit(void)
 	QUEUE_BACKEND_PID(MyBackendId) = MyProcPid;
 	QUEUE_BACKEND_DBOID(MyBackendId) = MyDatabaseId;
 	/* Insert backend into list of listeners at correct position */
-	if (prevListener > 0)
+	if (prevListener != InvalidBackendId)
 	{
 		QUEUE_NEXT_LISTENER(MyBackendId) = QUEUE_NEXT_LISTENER(prevListener);
 		QUEUE_NEXT_LISTENER(prevListener) = MyBackendId;
@@ -1253,7 +1253,7 @@ asyncQueueUnregister(void)
 		QUEUE_FIRST_LISTENER = QUEUE_NEXT_LISTENER(MyBackendId);
 	else
 	{
-		for (BackendId i = QUEUE_FIRST_LISTENER; i > 0; i = QUEUE_NEXT_LISTENER(i))
+		for (BackendId i = QUEUE_FIRST_LISTENER; i != InvalidBackendId; i = QUEUE_NEXT_LISTENER(i))
 		{
 			if (QUEUE_NEXT_LISTENER(i) == MyBackendId)
 			{
@@ -1533,7 +1533,7 @@ asyncQueueFillWarning(void)
 		QueuePosition min = QUEUE_HEAD;
 		int32		minPid = InvalidPid;
 
-		for (BackendId i = QUEUE_FIRST_LISTENER; i > 0; i = QUEUE_NEXT_LISTENER(i))
+		for (BackendId i = QUEUE_FIRST_LISTENER; i != InvalidBackendId; i = QUEUE_NEXT_LISTENER(i))
 		{
 			Assert(QUEUE_BACKEND_PID(i) != InvalidPid);
 			min = QUEUE_POS_MIN(min, QUEUE_BACKEND_POS(i));
@@ -1589,7 +1589,7 @@ SignalBackends(void)
 	count = 0;
 
 	LWLockAcquire(NotifyQueueLock, LW_EXCLUSIVE);
-	for (BackendId i = QUEUE_FIRST_LISTENER; i > 0; i = QUEUE_NEXT_LISTENER(i))
+	for (BackendId i = QUEUE_FIRST_LISTENER; i != InvalidBackendId; i = QUEUE_NEXT_LISTENER(i))
 	{
 		int32		pid = QUEUE_BACKEND_PID(i);
 		QueuePosition pos;
@@ -2126,7 +2126,7 @@ asyncQueueAdvanceTail(void)
 	 */
 	LWLockAcquire(NotifyQueueLock, LW_EXCLUSIVE);
 	min = QUEUE_HEAD;
-	for (BackendId i = QUEUE_FIRST_LISTENER; i > 0; i = QUEUE_NEXT_LISTENER(i))
+	for (BackendId i = QUEUE_FIRST_LISTENER; i != InvalidBackendId; i = QUEUE_NEXT_LISTENER(i))
 	{
 		Assert(QUEUE_BACKEND_PID(i) != InvalidPid);
 		min = QUEUE_POS_MIN(min, QUEUE_BACKEND_POS(i));
diff --git a/src/backend/postmaster/bgwriter.c b/src/backend/postmaster/bgwriter.c
index 6364b16261f..6cef901045d 100644
--- a/src/backend/postmaster/bgwriter.c
+++ b/src/backend/postmaster/bgwriter.c
@@ -326,7 +326,7 @@ BackgroundWriterMain(void)
 		if (rc == WL_TIMEOUT && can_hibernate && prev_hibernate)
 		{
 			/* Ask for notification at next buffer allocation */
-			StrategyNotifyBgWriter(MyProcNumber);
+			StrategyNotifyBgWriter(MyBackendId);
 			/* Sleep ... */
 			(void) WaitLatch(MyLatch,
 							 WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
diff --git a/src/backend/postmaster/pgarch.c b/src/backend/postmaster/pgarch.c
index 2aa3e9890a2..213c9669fda 100644
--- a/src/backend/postmaster/pgarch.c
+++ b/src/backend/postmaster/pgarch.c
@@ -242,7 +242,7 @@ PgArchiverMain(void)
 	 * Advertise our pgprocno so that backends can use our latch to wake us up
 	 * while we're sleeping.
 	 */
-	PgArch->pgprocno = MyProcNumber;
+	PgArch->pgprocno = MyBackendId;
 
 	/* Create workspace for pgarch_readyXlog() */
 	arch_files = palloc(sizeof(struct arch_files_state));
diff --git a/src/backend/postmaster/walsummarizer.c b/src/backend/postmaster/walsummarizer.c
index 5d61d1517e8..71560e8b808 100644
--- a/src/backend/postmaster/walsummarizer.c
+++ b/src/backend/postmaster/walsummarizer.c
@@ -248,7 +248,7 @@ WalSummarizerMain(void)
 	/* Advertise ourselves. */
 	on_shmem_exit(WalSummarizerShutdown, (Datum) 0);
 	LWLockAcquire(WALSummarizerLock, LW_EXCLUSIVE);
-	WalSummarizerCtl->summarizer_pgprocno = MyProcNumber;
+	WalSummarizerCtl->summarizer_pgprocno = MyBackendId;
 	LWLockRelease(WALSummarizerLock);
 
 	/* Create and switch to a memory context that we can reset on error. */
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 87a31745864..bb9da98c9f5 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -4780,7 +4780,7 @@ UnlockBuffers(void)
 		 * got a cancel/die interrupt before getting the signal.
 		 */
 		if ((buf_state & BM_PIN_COUNT_WAITER) != 0 &&
-			buf->wait_backend_pgprocno == MyProcNumber)
+			buf->wait_backend_pgprocno == MyBackendId)
 			buf_state &= ~BM_PIN_COUNT_WAITER;
 
 		UnlockBufHdr(buf, buf_state);
@@ -4930,7 +4930,7 @@ LockBufferForCleanup(Buffer buffer)
 			LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
 			elog(ERROR, "multiple backends attempting to wait for pincount 1");
 		}
-		bufHdr->wait_backend_pgprocno = MyProcNumber;
+		bufHdr->wait_backend_pgprocno = MyBackendId;
 		PinCountWaitBuf = bufHdr;
 		buf_state |= BM_PIN_COUNT_WAITER;
 		UnlockBufHdr(bufHdr, buf_state);
@@ -4994,7 +4994,7 @@ LockBufferForCleanup(Buffer buffer)
 		 */
 		buf_state = LockBufHdr(bufHdr);
 		if ((buf_state & BM_PIN_COUNT_WAITER) != 0 &&
-			bufHdr->wait_backend_pgprocno == MyProcNumber)
+			bufHdr->wait_backend_pgprocno == MyBackendId)
 			buf_state &= ~BM_PIN_COUNT_WAITER;
 		UnlockBufHdr(bufHdr, buf_state);
 
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index d96606ebba5..7ed08d95355 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2538,7 +2538,7 @@ ProcArrayInstallImportedXmin(TransactionId xmin,
 
 	/*
 	 * Find the PGPROC entry of the source transaction. (This could use
-	 * GetPGProcByBackendId(), unless it's a prepared xact.  But this isn't
+	 * GetPGProcByNumber(), unless it's a prepared xact.  But this isn't
 	 * performance critical.)
 	 */
 	for (index = 0; index < arrayP->numProcs; index++)
@@ -3116,9 +3116,9 @@ BackendIdGetProc(int backendID)
 {
 	PGPROC	   *result;
 
-	if (backendID < 1 || backendID > ProcGlobal->allProcCount)
+	if (backendID < 0 || backendID >= ProcGlobal->allProcCount)
 		return NULL;
-	result = GetPGProcByBackendId(backendID);
+	result = GetPGProcByNumber(backendID);
 
 	if (result->pid == 0)
 		return NULL;
@@ -3144,9 +3144,9 @@ BackendIdGetTransactionIds(int backendID, TransactionId *xid,
 	*nsubxid = 0;
 	*overflowed = false;
 
-	if (backendID < 1 || backendID > ProcGlobal->allProcCount)
+	if (backendID < 0 || backendID >= ProcGlobal->allProcCount)
 		return;
-	proc = GetPGProcByBackendId(backendID);
+	proc = GetPGProcByNumber(backendID);
 
 	/* Need to lock out additions/removals of backends */
 	LWLockAcquire(ProcArrayLock, LW_SHARED);
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index d1d5bf0c152..0bf577b8bc0 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -161,11 +161,11 @@ ProcSignalInit(void)
 	ProcSignalSlot *slot;
 	uint64		barrier_generation;
 
-	if (MyBackendId <= 0)
+	if (MyBackendId < 0)
 		elog(ERROR, "MyBackendId not set");
-	if (MyBackendId > NumProcSignalSlots)
+	if (MyBackendId >= NumProcSignalSlots)
 		elog(ERROR, "unexpected MyBackendId %d in ProcSignalInit (max %d)", MyBackendId, NumProcSignalSlots);
-	slot = &ProcSignal->psh_slot[MyBackendId - 1];
+	slot = &ProcSignal->psh_slot[MyBackendId];
 
 	/* sanity check */
 	if (slot->pss_pid != 0)
@@ -260,7 +260,7 @@ SendProcSignal(pid_t pid, ProcSignalReason reason, BackendId backendId)
 
 	if (backendId != InvalidBackendId)
 	{
-		slot = &ProcSignal->psh_slot[backendId - 1];
+		slot = &ProcSignal->psh_slot[backendId];
 
 		/*
 		 * Note: Since there's no locking, it's possible that the target
diff --git a/src/backend/storage/ipc/sinvaladt.c b/src/backend/storage/ipc/sinvaladt.c
index f624bfc7d78..f3ef3eb1ab0 100644
--- a/src/backend/storage/ipc/sinvaladt.c
+++ b/src/backend/storage/ipc/sinvaladt.c
@@ -274,15 +274,13 @@ SharedInvalBackendInit(bool sendOnly)
 	ProcState  *stateP;
 	pid_t		oldPid;
 	SISeg	   *segP = shmInvalBuffer;
-	int			pgprocno;
 
-	if (MyBackendId <= 0)
+	if (MyBackendId < 0)
 		elog(ERROR, "MyBackendId not set");
-	if (MyBackendId > NumProcStateSlots)
+	if (MyBackendId >= NumProcStateSlots)
 		elog(PANIC, "unexpected MyBackendId %d in SharedInvalBackendInit (max %d)",
 			 MyBackendId, NumProcStateSlots);
-	pgprocno = MyBackendId - 1;
-	stateP = &segP->procState[pgprocno];
+	stateP = &segP->procState[MyBackendId];
 
 	/*
 	 * This can run in parallel with read operations, but not with write
@@ -299,7 +297,7 @@ SharedInvalBackendInit(bool sendOnly)
 			 MyBackendId, (int) oldPid);
 	}
 
-	shmInvalBuffer->pgprocnos[shmInvalBuffer->numProcs++] = pgprocno;
+	shmInvalBuffer->pgprocnos[shmInvalBuffer->numProcs++] = MyBackendId;
 
 	/* Fetch next local transaction ID into local memory */
 	nextLocalTransactionId = stateP->nextLXID;
@@ -331,14 +329,13 @@ 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[pgprocno];
+	stateP = &segP->procState[MyBackendId];
 
 	/* Update next local transaction ID for next holder of this backendID */
 	stateP->nextLXID = nextLocalTransactionId;
@@ -351,7 +348,7 @@ CleanupInvalidationState(int status, Datum arg)
 
 	for (i = segP->numProcs - 1; i >= 0; i--)
 	{
-		if (segP->pgprocnos[i] == pgprocno)
+		if (segP->pgprocnos[i] == MyBackendId)
 		{
 			if (i != segP->numProcs - 1)
 				segP->pgprocnos[i] = segP->pgprocnos[segP->numProcs - 1];
@@ -481,7 +478,7 @@ SIGetDataEntries(SharedInvalidationMessage *data, int datasize)
 	int			n;
 
 	segP = shmInvalBuffer;
-	stateP = &segP->procState[MyBackendId - 1];
+	stateP = &segP->procState[MyBackendId];
 
 	/*
 	 * Before starting to take locks, do a quick, unlocked test to see whether
@@ -668,7 +665,7 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
 	if (needSig)
 	{
 		pid_t		his_pid = needSig->procPid;
-		BackendId	his_backendId = (needSig - &segP->procState[0]) + 1;
+		BackendId	his_backendId = (needSig - &segP->procState[0]);
 
 		needSig->signaled = true;
 		LWLockRelease(SInvalReadLock);
diff --git a/src/backend/storage/lmgr/condition_variable.c b/src/backend/storage/lmgr/condition_variable.c
index 10fdae19dcc..daff1368c1f 100644
--- a/src/backend/storage/lmgr/condition_variable.c
+++ b/src/backend/storage/lmgr/condition_variable.c
@@ -57,7 +57,7 @@ ConditionVariableInit(ConditionVariable *cv)
 void
 ConditionVariablePrepareToSleep(ConditionVariable *cv)
 {
-	int			pgprocno = MyProcNumber;
+	int			pgprocno = MyBackendId;
 
 	/*
 	 * If some other sleep is already prepared, cancel it; this is necessary
@@ -181,10 +181,10 @@ ConditionVariableTimedSleep(ConditionVariable *cv, long timeout,
 		 * guarantee not to return spuriously, we'll avoid this obvious case.
 		 */
 		SpinLockAcquire(&cv->mutex);
-		if (!proclist_contains(&cv->wakeup, MyProcNumber, cvWaitLink))
+		if (!proclist_contains(&cv->wakeup, MyBackendId, cvWaitLink))
 		{
 			done = true;
-			proclist_push_tail(&cv->wakeup, MyProcNumber, cvWaitLink);
+			proclist_push_tail(&cv->wakeup, MyBackendId, cvWaitLink);
 		}
 		SpinLockRelease(&cv->mutex);
 
@@ -236,8 +236,8 @@ ConditionVariableCancelSleep(void)
 		return false;
 
 	SpinLockAcquire(&cv->mutex);
-	if (proclist_contains(&cv->wakeup, MyProcNumber, cvWaitLink))
-		proclist_delete(&cv->wakeup, MyProcNumber, cvWaitLink);
+	if (proclist_contains(&cv->wakeup, MyBackendId, cvWaitLink))
+		proclist_delete(&cv->wakeup, MyBackendId, cvWaitLink);
 	else
 		signaled = true;
 	SpinLockRelease(&cv->mutex);
@@ -281,7 +281,7 @@ ConditionVariableSignal(ConditionVariable *cv)
 void
 ConditionVariableBroadcast(ConditionVariable *cv)
 {
-	int			pgprocno = MyProcNumber;
+	int			pgprocno = MyBackendId;
 	PGPROC	   *proc = NULL;
 	bool		have_sentinel = false;
 
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 997857679ed..291ac8d58f8 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -1056,9 +1056,9 @@ LWLockQueueSelf(LWLock *lock, LWLockMode mode)
 
 	/* LW_WAIT_UNTIL_FREE waiters are always at the front of the queue */
 	if (mode == LW_WAIT_UNTIL_FREE)
-		proclist_push_head(&lock->waiters, MyProcNumber, lwWaitLink);
+		proclist_push_head(&lock->waiters, MyBackendId, lwWaitLink);
 	else
-		proclist_push_tail(&lock->waiters, MyProcNumber, lwWaitLink);
+		proclist_push_tail(&lock->waiters, MyBackendId, lwWaitLink);
 
 	/* Can release the mutex now */
 	LWLockWaitListUnlock(lock);
@@ -1097,7 +1097,7 @@ LWLockDequeueSelf(LWLock *lock)
 	 */
 	on_waitlist = MyProc->lwWaiting == LW_WS_WAITING;
 	if (on_waitlist)
-		proclist_delete(&lock->waiters, MyProcNumber, lwWaitLink);
+		proclist_delete(&lock->waiters, MyBackendId, lwWaitLink);
 
 	if (proclist_is_empty(&lock->waiters) &&
 		(pg_atomic_read_u32(&lock->state) & LW_FLAG_HAS_WAITERS) != 0)
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index d62060d58c8..d3dec91a68d 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1830,7 +1830,7 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
 	sxact->finishedBefore = InvalidTransactionId;
 	sxact->xmin = snapshot->xmin;
 	sxact->pid = MyProcPid;
-	sxact->pgprocno = MyProcNumber;
+	sxact->pgprocno = MyBackendId;
 	dlist_init(&sxact->predicateLocks);
 	dlist_node_init(&sxact->finishedLink);
 	sxact->flags = 0;
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 4226e41d80c..402ab53a061 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -64,7 +64,6 @@ bool		log_lock_waits = false;
 
 /* Pointer to this process's PGPROC struct, if any */
 PGPROC	   *MyProc = NULL;
-int			MyProcNumber = INVALID_PGPROCNO;
 
 /*
  * This spinlock protects the freelist of recycled PGPROC structures.
@@ -352,8 +351,7 @@ InitProcess(void)
 				(errcode(ERRCODE_TOO_MANY_CONNECTIONS),
 				 errmsg("sorry, too many clients already")));
 	}
-	MyProcNumber = GetNumberFromPGProc(MyProc);
-	MyBackendId = GetBackendIdFromPGProc(MyProc);
+	MyBackendId = GetNumberFromPGProc(MyProc);
 
 	/*
 	 * Cross-check that the PGPROC is of the type we expect; if this were not
@@ -566,8 +564,7 @@ InitAuxiliaryProcess(void)
 	SpinLockRelease(ProcStructLock);
 
 	MyProc = auxproc;
-	MyProcNumber = GetNumberFromPGProc(MyProc);
-	MyBackendId = GetBackendIdFromPGProc(MyProc);
+	MyBackendId = GetNumberFromPGProc(MyProc);
 
 	/*
 	 * Initialize all fields of MyProc, except for those previously
@@ -910,7 +907,6 @@ ProcKill(int code, Datum arg)
 
 	proc = MyProc;
 	MyProc = NULL;
-	MyProcNumber = INVALID_PGPROCNO;
 	MyBackendId = InvalidBackendId;
 	DisownLatch(&proc->procLatch);
 
@@ -988,7 +984,6 @@ AuxiliaryProcKill(int code, Datum arg)
 
 	proc = MyProc;
 	MyProc = NULL;
-	MyProcNumber = INVALID_PGPROCNO;
 	MyBackendId = InvalidBackendId;
 	DisownLatch(&proc->procLatch);
 
diff --git a/src/backend/utils/activity/backend_status.c b/src/backend/utils/activity/backend_status.c
index 3d3f7b06723..8a4a9cd219b 100644
--- a/src/backend/utils/activity/backend_status.c
+++ b/src/backend/utils/activity/backend_status.c
@@ -249,8 +249,8 @@ pgstat_beinit(void)
 {
 	/* Initialize MyBEEntry */
 	Assert(MyBackendId != InvalidBackendId);
-	Assert(MyBackendId >= 1 && MyBackendId <= NumBackendStatSlots);
-	MyBEEntry = &BackendStatusArray[MyBackendId - 1];
+	Assert(MyBackendId >= 0 && MyBackendId < NumBackendStatSlots);
+	MyBEEntry = &BackendStatusArray[MyBackendId];
 
 	/* Set up a process-exit hook to clean up */
 	on_shmem_exit(pgstat_beshutdown_hook, 0);
@@ -721,7 +721,7 @@ pgstat_read_current_status(void)
 #ifdef ENABLE_GSS
 	PgBackendGSSStatus *localgssstatus;
 #endif
-	int			i;
+	int			backendId;
 
 	if (localBackendStatusTable)
 		return;					/* already done */
@@ -764,7 +764,7 @@ pgstat_read_current_status(void)
 
 	beentry = BackendStatusArray;
 	localentry = localtable;
-	for (i = 1; i <= NumBackendStatSlots; i++)
+	for (backendId = 0; backendId < NumBackendStatSlots; backendId++)
 	{
 		/*
 		 * Follow the protocol of retrying if st_changecount changes while we
@@ -835,8 +835,8 @@ pgstat_read_current_status(void)
 			 * is in order by backend_id.  pgstat_get_beentry_by_backend_id()
 			 * depends on that.
 			 */
-			localentry->backend_id = i;
-			BackendIdGetTransactionIds(i,
+			localentry->backend_id = backendId;
+			BackendIdGetTransactionIds(backendId,
 									   &localentry->backend_xid,
 									   &localentry->backend_xmin,
 									   &localentry->backend_subxact_count,
diff --git a/src/backend/utils/adt/mcxtfuncs.c b/src/backend/utils/adt/mcxtfuncs.c
index a7267dc15d1..a3239ab126b 100644
--- a/src/backend/utils/adt/mcxtfuncs.c
+++ b/src/backend/utils/adt/mcxtfuncs.c
@@ -176,7 +176,7 @@ pg_log_backend_memory_contexts(PG_FUNCTION_ARGS)
 	}
 
 	if (proc != NULL)
-		backendId = GetBackendIdFromPGProc(proc);
+		backendId = GetNumberFromPGProc(proc);
 	if (SendProcSignal(pid, PROCSIG_LOG_MEMORY_CONTEXT, backendId) < 0)
 	{
 		/* Again, just a warning to allow loops */
diff --git a/src/include/storage/backendid.h b/src/include/storage/backendid.h
index 01387723f79..de242e2c039 100644
--- a/src/include/storage/backendid.h
+++ b/src/include/storage/backendid.h
@@ -19,8 +19,10 @@
  * 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.
+ * Backend IDs are assigned starting from 0.  A backend ID is also an index
+ * into the proc array, and is synonymous with "proc numbers" used in other
+ * parts of the code.  (Before version 17, backend IDs were assigned
+ * separately from proc numbers, and started from 1.)
  */
 typedef int BackendId;
 
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index d1e50380b7c..eca0a5809e1 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -199,7 +199,7 @@ struct PGPROC
 	 */
 	struct {
 		BackendId	backendId;		/* For regular backends, equal to
-									 * GetBackendIdFromPGProc(proc).  For
+									 * GetNumberFromPGProc(proc).  For
 									 * prepared xacts, ID of the original
 									 * backend that processed the
 									 * transaction. For unused PGPROC entries,
@@ -317,7 +317,6 @@ struct PGPROC
 
 
 extern PGDLLIMPORT PGPROC *MyProc;
-extern PGDLLIMPORT int MyProcNumber;	/* same as GetNumberFromPGProc(MyProc) */
 
 /*
  * There is one ProcGlobal struct for the whole database cluster.
@@ -422,15 +421,11 @@ extern PGDLLIMPORT PROC_HDR *ProcGlobal;
 extern PGDLLIMPORT PGPROC *PreparedXactProcs;
 
 /*
- * 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.
+ * Accessors for getting PGPROC given a pgprocno (or BackendId which is the
+ * same thing since version 17) and vice versa.
  */
 #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,
-- 
2.39.2

Reply via email to