From ed0b409d22346b1b027a4c2099ca66984d94b6dd Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 25 Nov 2011 08:02:10 -0500
Subject: [PATCH] Move "hot" members of PGPROC into a separate PGXACT array.

This speeds up snapshot-taking and reduces ProcArrayLock contention.
Also, the PGPROC (and PGXACT) structures used by two-phase commit are
now allocated as part of the main array, rather than in a separate
array, and we keep ProcArray sorted in pointer order.  These changes
are intended to minimize the number of cache lines that must be pulled
in to take a snapshot, and testing shows a substantial increase in
performance on both read and write workloads at high concurrencies.

Pavan Deolasee, Heikki Linnakangas, Robert Haas
---
 src/backend/access/transam/twophase.c | 115 ++++++-----
 src/backend/access/transam/varsup.c   |  11 +-
 src/backend/access/transam/xact.c     |   4 +-
 src/backend/commands/analyze.c        |   4 +-
 src/backend/commands/vacuum.c         |   4 +-
 src/backend/postmaster/postmaster.c   |   3 +
 src/backend/replication/walsender.c   |   2 +-
 src/backend/storage/ipc/ipci.c        |   2 +-
 src/backend/storage/ipc/procarray.c   | 276 ++++++++++++++++----------
 src/backend/storage/lmgr/deadlock.c   |   4 +-
 src/backend/storage/lmgr/lock.c       |   3 +-
 src/backend/storage/lmgr/proc.c       |  87 +++++---
 src/backend/utils/time/snapmgr.c      |   2 +-
 src/include/storage/proc.h            |  47 +++--
 14 files changed, 356 insertions(+), 208 deletions(-)

diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 477982d5fa5..d2fecb1ecb9 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -113,7 +113,8 @@ int			max_prepared_xacts = 0;
 
 typedef struct GlobalTransactionData
 {
-	PGPROC		proc;			/* dummy proc */
+	GlobalTransaction next;
+	int			pgprocno;		/* dummy proc */
 	BackendId	dummyBackendId; /* similar to backend id for backends */
 	TimestampTz prepared_at;	/* time of preparation */
 	XLogRecPtr	prepare_lsn;	/* XLOG offset of prepare record */
@@ -207,7 +208,8 @@ TwoPhaseShmemInit(void)
 					  sizeof(GlobalTransaction) * max_prepared_xacts));
 		for (i = 0; i < max_prepared_xacts; i++)
 		{
-			gxacts[i].proc.links.next = (SHM_QUEUE *) TwoPhaseState->freeGXacts;
+			gxacts[i].pgprocno = PreparedXactProcs[i].pgprocno;
+			gxacts[i].next = TwoPhaseState->freeGXacts;
 			TwoPhaseState->freeGXacts = &gxacts[i];
 
 			/*
@@ -243,6 +245,8 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 				TimestampTz prepared_at, Oid owner, Oid databaseid)
 {
 	GlobalTransaction gxact;
+	PGPROC	   *proc;
+	PGXACT	   *pgxact;
 	int			i;
 
 	if (strlen(gid) >= GIDSIZE)
@@ -274,7 +278,7 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 			TwoPhaseState->numPrepXacts--;
 			TwoPhaseState->prepXacts[i] = TwoPhaseState->prepXacts[TwoPhaseState->numPrepXacts];
 			/* and put it back in the freelist */
-			gxact->proc.links.next = (SHM_QUEUE *) TwoPhaseState->freeGXacts;
+			gxact->next = TwoPhaseState->freeGXacts;
 			TwoPhaseState->freeGXacts = gxact;
 			/* Back up index count too, so we don't miss scanning one */
 			i--;
@@ -302,32 +306,36 @@ MarkAsPreparing(TransactionId xid, const char *gid,
 				 errhint("Increase max_prepared_transactions (currently %d).",
 						 max_prepared_xacts)));
 	gxact = TwoPhaseState->freeGXacts;
-	TwoPhaseState->freeGXacts = (GlobalTransaction) gxact->proc.links.next;
+	TwoPhaseState->freeGXacts = (GlobalTransaction) gxact->next;
 
-	/* Initialize it */
-	MemSet(&gxact->proc, 0, sizeof(PGPROC));
-	SHMQueueElemInit(&(gxact->proc.links));
-	gxact->proc.waitStatus = STATUS_OK;
+	proc = &ProcGlobal->allProcs[gxact->pgprocno];
+	pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
+
+	/* Initialize the PGPROC entry */
+	MemSet(proc, 0, sizeof(PGPROC));
+	proc->pgprocno = gxact->pgprocno;
+	SHMQueueElemInit(&(proc->links));
+	proc->waitStatus = STATUS_OK;
 	/* We set up the gxact's VXID as InvalidBackendId/XID */
-	gxact->proc.lxid = (LocalTransactionId) xid;
-	gxact->proc.xid = xid;
-	gxact->proc.xmin = InvalidTransactionId;
-	gxact->proc.pid = 0;
-	gxact->proc.backendId = InvalidBackendId;
-	gxact->proc.databaseId = databaseid;
-	gxact->proc.roleId = owner;
-	gxact->proc.inCommit = false;
-	gxact->proc.vacuumFlags = 0;
-	gxact->proc.lwWaiting = false;
-	gxact->proc.lwExclusive = false;
-	gxact->proc.lwWaitLink = NULL;
-	gxact->proc.waitLock = NULL;
-	gxact->proc.waitProcLock = NULL;
+	proc->lxid = (LocalTransactionId) xid;
+	pgxact->xid = xid;
+	pgxact->xmin = InvalidTransactionId;
+	pgxact->inCommit = false;
+	pgxact->vacuumFlags = 0;
+	proc->pid = 0;
+	proc->backendId = InvalidBackendId;
+	proc->databaseId = databaseid;
+	proc->roleId = owner;
+	proc->lwWaiting = false;
+	proc->lwExclusive = false;
+	proc->lwWaitLink = NULL;
+	proc->waitLock = NULL;
+	proc->waitProcLock = NULL;
 	for (i = 0; i < NUM_LOCK_PARTITIONS; i++)
-		SHMQueueInit(&(gxact->proc.myProcLocks[i]));
+		SHMQueueInit(&(proc->myProcLocks[i]));
 	/* subxid data must be filled later by GXactLoadSubxactData */
-	gxact->proc.subxids.overflowed = false;
-	gxact->proc.subxids.nxids = 0;
+	pgxact->overflowed = false;
+	pgxact->nxids = 0;
 
 	gxact->prepared_at = prepared_at;
 	/* initialize LSN to 0 (start of WAL) */
@@ -358,17 +366,19 @@ static void
 GXactLoadSubxactData(GlobalTransaction gxact, int nsubxacts,
 					 TransactionId *children)
 {
+	PGPROC *proc = &ProcGlobal->allProcs[gxact->pgprocno];
+	PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 	/* We need no extra lock since the GXACT isn't valid yet */
 	if (nsubxacts > PGPROC_MAX_CACHED_SUBXIDS)
 	{
-		gxact->proc.subxids.overflowed = true;
+		pgxact->overflowed = true;
 		nsubxacts = PGPROC_MAX_CACHED_SUBXIDS;
 	}
 	if (nsubxacts > 0)
 	{
-		memcpy(gxact->proc.subxids.xids, children,
+		memcpy(proc->subxids.xids, children,
 			   nsubxacts * sizeof(TransactionId));
-		gxact->proc.subxids.nxids = nsubxacts;
+		pgxact->nxids = nsubxacts;
 	}
 }
 
@@ -389,7 +399,7 @@ MarkAsPrepared(GlobalTransaction gxact)
 	 * Put it into the global ProcArray so TransactionIdIsInProgress considers
 	 * the XID as still running.
 	 */
-	ProcArrayAdd(&gxact->proc);
+	ProcArrayAdd(&ProcGlobal->allProcs[gxact->pgprocno]);
 }
 
 /*
@@ -406,6 +416,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];
 
 		/* Ignore not-yet-valid GIDs */
 		if (!gxact->valid)
@@ -436,7 +447,7 @@ LockGXact(const char *gid, Oid user)
 		 * there may be some other issues as well.	Hence disallow until
 		 * someone gets motivated to make it work.
 		 */
-		if (MyDatabaseId != gxact->proc.databaseId)
+		if (MyDatabaseId != proc->databaseId)
 			ereport(ERROR,
 					(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				  errmsg("prepared transaction belongs to another database"),
@@ -483,7 +494,7 @@ RemoveGXact(GlobalTransaction gxact)
 			TwoPhaseState->prepXacts[i] = TwoPhaseState->prepXacts[TwoPhaseState->numPrepXacts];
 
 			/* and put it back in the freelist */
-			gxact->proc.links.next = (SHM_QUEUE *) TwoPhaseState->freeGXacts;
+			gxact->next = TwoPhaseState->freeGXacts;
 			TwoPhaseState->freeGXacts = gxact;
 
 			LWLockRelease(TwoPhaseStateLock);
@@ -518,8 +529,9 @@ TransactionIdIsPrepared(TransactionId xid)
 	for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
 	{
 		GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+		PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 
-		if (gxact->valid && gxact->proc.xid == xid)
+		if (gxact->valid && pgxact->xid == xid)
 		{
 			result = true;
 			break;
@@ -642,6 +654,8 @@ 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];
+		PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 		Datum		values[5];
 		bool		nulls[5];
 		HeapTuple	tuple;
@@ -656,11 +670,11 @@ pg_prepared_xact(PG_FUNCTION_ARGS)
 		MemSet(values, 0, sizeof(values));
 		MemSet(nulls, 0, sizeof(nulls));
 
-		values[0] = TransactionIdGetDatum(gxact->proc.xid);
+		values[0] = TransactionIdGetDatum(pgxact->xid);
 		values[1] = CStringGetTextDatum(gxact->gid);
 		values[2] = TimestampTzGetDatum(gxact->prepared_at);
 		values[3] = ObjectIdGetDatum(gxact->owner);
-		values[4] = ObjectIdGetDatum(gxact->proc.databaseId);
+		values[4] = ObjectIdGetDatum(proc->databaseId);
 
 		tuple = heap_form_tuple(funcctx->tuple_desc, values, nulls);
 		result = HeapTupleGetDatum(tuple);
@@ -711,10 +725,11 @@ TwoPhaseGetDummyProc(TransactionId xid)
 	for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
 	{
 		GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+		PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 
-		if (gxact->proc.xid == xid)
+		if (pgxact->xid == xid)
 		{
-			result = &gxact->proc;
+			result = &ProcGlobal->allProcs[gxact->pgprocno];
 			break;
 		}
 	}
@@ -841,7 +856,9 @@ save_state_data(const void *data, uint32 len)
 void
 StartPrepare(GlobalTransaction gxact)
 {
-	TransactionId xid = gxact->proc.xid;
+	PGPROC *proc = &ProcGlobal->allProcs[gxact->pgprocno];
+	PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
+	TransactionId xid = pgxact->xid;
 	TwoPhaseFileHeader hdr;
 	TransactionId *children;
 	RelFileNode *commitrels;
@@ -865,7 +882,7 @@ StartPrepare(GlobalTransaction gxact)
 	hdr.magic = TWOPHASE_MAGIC;
 	hdr.total_len = 0;			/* EndPrepare will fill this in */
 	hdr.xid = xid;
-	hdr.database = gxact->proc.databaseId;
+	hdr.database = proc->databaseId;
 	hdr.prepared_at = gxact->prepared_at;
 	hdr.owner = gxact->owner;
 	hdr.nsubxacts = xactGetCommittedChildren(&children);
@@ -913,7 +930,8 @@ StartPrepare(GlobalTransaction gxact)
 void
 EndPrepare(GlobalTransaction gxact)
 {
-	TransactionId xid = gxact->proc.xid;
+	PGXACT	   *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
+	TransactionId xid = pgxact->xid;
 	TwoPhaseFileHeader *hdr;
 	char		path[MAXPGPATH];
 	XLogRecData *record;
@@ -1021,7 +1039,7 @@ EndPrepare(GlobalTransaction gxact)
 	 */
 	START_CRIT_SECTION();
 
-	MyProc->inCommit = true;
+	MyPgXact->inCommit = true;
 
 	gxact->prepare_lsn = XLogInsert(RM_XACT_ID, XLOG_XACT_PREPARE,
 									records.head);
@@ -1069,7 +1087,7 @@ EndPrepare(GlobalTransaction gxact)
 	 * checkpoint starting after this will certainly see the gxact as a
 	 * candidate for fsyncing.
 	 */
-	MyProc->inCommit = false;
+	MyPgXact->inCommit = false;
 
 	END_CRIT_SECTION();
 
@@ -1242,6 +1260,8 @@ void
 FinishPreparedTransaction(const char *gid, bool isCommit)
 {
 	GlobalTransaction gxact;
+	PGPROC	   *proc;
+	PGXACT	   *pgxact;
 	TransactionId xid;
 	char	   *buf;
 	char	   *bufptr;
@@ -1260,7 +1280,9 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	 * try to commit the same GID at once.
 	 */
 	gxact = LockGXact(gid, GetUserId());
-	xid = gxact->proc.xid;
+	proc = &ProcGlobal->allProcs[gxact->pgprocno];
+	pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
+	xid = pgxact->xid;
 
 	/*
 	 * Read and validate the state file
@@ -1309,7 +1331,7 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 									   hdr->nsubxacts, children,
 									   hdr->nabortrels, abortrels);
 
-	ProcArrayRemove(&gxact->proc, latestXid);
+	ProcArrayRemove(proc, latestXid);
 
 	/*
 	 * In case we fail while running the callbacks, mark the gxact invalid so
@@ -1540,10 +1562,11 @@ CheckPointTwoPhase(XLogRecPtr redo_horizon)
 	for (i = 0; i < TwoPhaseState->numPrepXacts; i++)
 	{
 		GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+		PGXACT *pgxact = &ProcGlobal->allPgXact[gxact->pgprocno];
 
 		if (gxact->valid &&
 			XLByteLE(gxact->prepare_lsn, redo_horizon))
-			xids[nxids++] = gxact->proc.xid;
+			xids[nxids++] = pgxact->xid;
 	}
 
 	LWLockRelease(TwoPhaseStateLock);
@@ -1972,7 +1995,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	START_CRIT_SECTION();
 
 	/* See notes in RecordTransactionCommit */
-	MyProc->inCommit = true;
+	MyPgXact->inCommit = true;
 
 	/* Emit the XLOG commit record */
 	xlrec.xid = xid;
@@ -2037,7 +2060,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	TransactionIdCommitTree(xid, nchildren, children);
 
 	/* Checkpoint can proceed now */
-	MyProc->inCommit = false;
+	MyPgXact->inCommit = false;
 
 	END_CRIT_SECTION();
 
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 61dcfedad43..443e5e4ea66 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -54,7 +54,7 @@ GetNewTransactionId(bool isSubXact)
 	if (IsBootstrapProcessingMode())
 	{
 		Assert(!isSubXact);
-		MyProc->xid = BootstrapTransactionId;
+		MyPgXact->xid = BootstrapTransactionId;
 		return BootstrapTransactionId;
 	}
 
@@ -208,20 +208,21 @@ GetNewTransactionId(bool isSubXact)
 		 * TransactionId and int fetch/store are atomic.
 		 */
 		volatile PGPROC *myproc = MyProc;
+		volatile PGXACT *mypgxact = MyPgXact;
 
 		if (!isSubXact)
-			myproc->xid = xid;
+			mypgxact->xid = xid;
 		else
 		{
-			int			nxids = myproc->subxids.nxids;
+			int			nxids = mypgxact->nxids;
 
 			if (nxids < PGPROC_MAX_CACHED_SUBXIDS)
 			{
 				myproc->subxids.xids[nxids] = xid;
-				myproc->subxids.nxids = nxids + 1;
+				mypgxact->nxids = nxids + 1;
 			}
 			else
-				myproc->subxids.overflowed = true;
+				mypgxact->overflowed = true;
 		}
 	}
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index c151d3be191..c383011b5f6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -981,7 +981,7 @@ RecordTransactionCommit(void)
 		 * bit fuzzy, but it doesn't matter.
 		 */
 		START_CRIT_SECTION();
-		MyProc->inCommit = true;
+		MyPgXact->inCommit = true;
 
 		SetCurrentTransactionStopTimestamp();
 
@@ -1155,7 +1155,7 @@ RecordTransactionCommit(void)
 	 */
 	if (markXidCommitted)
 	{
-		MyProc->inCommit = false;
+		MyPgXact->inCommit = false;
 		END_CRIT_SECTION();
 	}
 
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index 32985a4a0a0..314324618a8 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -223,7 +223,7 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
 	 * OK, let's do it.  First let other backends know I'm in ANALYZE.
 	 */
 	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
-	MyProc->vacuumFlags |= PROC_IN_ANALYZE;
+	MyPgXact->vacuumFlags |= PROC_IN_ANALYZE;
 	LWLockRelease(ProcArrayLock);
 
 	/*
@@ -250,7 +250,7 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
 	 * because the vacuum flag is cleared by the end-of-xact code.
 	 */
 	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
-	MyProc->vacuumFlags &= ~PROC_IN_ANALYZE;
+	MyPgXact->vacuumFlags &= ~PROC_IN_ANALYZE;
 	LWLockRelease(ProcArrayLock);
 }
 
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index f42504cf9fd..89e190df956 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -893,9 +893,9 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound)
 		 * which is probably Not Good.
 		 */
 		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
-		MyProc->vacuumFlags |= PROC_IN_VACUUM;
+		MyPgXact->vacuumFlags |= PROC_IN_VACUUM;
 		if (for_wraparound)
-			MyProc->vacuumFlags |= PROC_VACUUM_FOR_WRAPAROUND;
+			MyPgXact->vacuumFlags |= PROC_VACUUM_FOR_WRAPAROUND;
 		LWLockRelease(ProcArrayLock);
 	}
 
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 6758083bdd5..963189d4150 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -430,6 +430,7 @@ typedef struct
 	slock_t    *ProcStructLock;
 	PROC_HDR   *ProcGlobal;
 	PGPROC	   *AuxiliaryProcs;
+	PGPROC	   *PreparedXactProcs;
 	PMSignalData *PMSignalState;
 	InheritableSocket pgStatSock;
 	pid_t		PostmasterPid;
@@ -4724,6 +4725,7 @@ save_backend_variables(BackendParameters *param, Port *port,
 	param->ProcStructLock = ProcStructLock;
 	param->ProcGlobal = ProcGlobal;
 	param->AuxiliaryProcs = AuxiliaryProcs;
+	param->PreparedXactProcs = PreparedXactProcs;
 	param->PMSignalState = PMSignalState;
 	if (!write_inheritable_socket(&param->pgStatSock, pgStatSock, childPid))
 		return false;
@@ -4947,6 +4949,7 @@ restore_backend_variables(BackendParameters *param, Port *port)
 	ProcStructLock = param->ProcStructLock;
 	ProcGlobal = param->ProcGlobal;
 	AuxiliaryProcs = param->AuxiliaryProcs;
+	PreparedXactProcs = param->PreparedXactProcs;
 	PMSignalState = param->PMSignalState;
 	read_inheritable_socket(&pgStatSock, &param->pgStatSock);
 
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index dd2d6ee2184..ea865204172 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -702,7 +702,7 @@ ProcessStandbyHSFeedbackMessage(void)
 	 * safe, and if we're moving it backwards, well, the data is at risk
 	 * already since a VACUUM could have just finished calling GetOldestXmin.)
 	 */
-	MyProc->xmin = reply.xmin;
+	MyPgXact->xmin = reply.xmin;
 }
 
 /* Main loop of walsender process */
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 56c0bd8d498..bb8b832065b 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -192,7 +192,6 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 	XLOGShmemInit();
 	CLOGShmemInit();
 	SUBTRANSShmemInit();
-	TwoPhaseShmemInit();
 	MultiXactShmemInit();
 	InitBufferPool();
 
@@ -213,6 +212,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 		InitProcGlobal();
 	CreateSharedProcArray();
 	CreateSharedBackendStatus();
+	TwoPhaseShmemInit();
 
 	/*
 	 * Set up shared-inval messaging
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 1a48485f970..19ff524a604 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -82,14 +82,17 @@ typedef struct ProcArrayStruct
 	TransactionId lastOverflowedXid;
 
 	/*
-	 * We declare procs[] as 1 entry because C wants a fixed-size array, but
+	 * We declare pgprocnos[] as 1 entry because C wants a fixed-size array, but
 	 * actually it is maxProcs entries long.
 	 */
-	PGPROC	   *procs[1];		/* VARIABLE LENGTH ARRAY */
+	int			pgprocnos[1];		/* VARIABLE LENGTH ARRAY */
 } ProcArrayStruct;
 
 static ProcArrayStruct *procArray;
 
+static PGPROC *allProcs;
+static PGXACT *allPgXact;
+
 /*
  * Bookkeeping for tracking emulated transactions in recovery
  */
@@ -169,8 +172,8 @@ ProcArrayShmemSize(void)
 	/* Size of the ProcArray structure itself */
 #define PROCARRAY_MAXPROCS	(MaxBackends + max_prepared_xacts)
 
-	size = offsetof(ProcArrayStruct, procs);
-	size = add_size(size, mul_size(sizeof(PGPROC *), PROCARRAY_MAXPROCS));
+	size = offsetof(ProcArrayStruct, pgprocnos);
+	size = add_size(size, mul_size(sizeof(int), PROCARRAY_MAXPROCS));
 
 	/*
 	 * During Hot Standby processing we have a data structure called
@@ -211,8 +214,8 @@ CreateSharedProcArray(void)
 	/* Create or attach to the ProcArray shared structure */
 	procArray = (ProcArrayStruct *)
 		ShmemInitStruct("Proc Array",
-						add_size(offsetof(ProcArrayStruct, procs),
-								 mul_size(sizeof(PGPROC *),
+						add_size(offsetof(ProcArrayStruct, pgprocnos),
+								 mul_size(sizeof(int),
 										  PROCARRAY_MAXPROCS)),
 						&found);
 
@@ -231,6 +234,9 @@ CreateSharedProcArray(void)
 		procArray->lastOverflowedXid = InvalidTransactionId;
 	}
 
+	allProcs = ProcGlobal->allProcs;
+	allPgXact = ProcGlobal->allPgXact;
+
 	/* Create or attach to the KnownAssignedXids arrays too, if needed */
 	if (EnableHotStandby)
 	{
@@ -253,6 +259,7 @@ void
 ProcArrayAdd(PGPROC *proc)
 {
 	ProcArrayStruct *arrayP = procArray;
+	int			index;
 
 	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
 
@@ -269,7 +276,28 @@ ProcArrayAdd(PGPROC *proc)
 				 errmsg("sorry, too many clients already")));
 	}
 
-	arrayP->procs[arrayP->numProcs] = proc;
+	/*
+	 * Keep the procs array sorted by (PGPROC *) so that we can utilize
+	 * locality of references much better. This is useful while traversing the
+	 * ProcArray because there is a increased likelyhood of finding the next
+	 * PGPROC structure in the cache.
+	 * 
+	 * Since the occurance of adding/removing a proc is much lower than the
+	 * access to the ProcArray itself, the overhead should be marginal
+	 */
+	for (index = 0; index < arrayP->numProcs; index++)
+	{
+		/*
+		 * If we are the first PGPROC or if we have found our right position in
+		 * the array, break
+		 */
+		if ((arrayP->pgprocnos[index] == -1) || (arrayP->pgprocnos[index] > proc->pgprocno))
+			break;
+	}
+
+	memmove(&arrayP->pgprocnos[index + 1], &arrayP->pgprocnos[index],
+			(arrayP->numProcs - index) * sizeof (int));
+	arrayP->pgprocnos[index] = proc->pgprocno;
 	arrayP->numProcs++;
 
 	LWLockRelease(ProcArrayLock);
@@ -301,7 +329,7 @@ ProcArrayRemove(PGPROC *proc, TransactionId latestXid)
 
 	if (TransactionIdIsValid(latestXid))
 	{
-		Assert(TransactionIdIsValid(proc->xid));
+		Assert(TransactionIdIsValid(allPgXact[proc->pgprocno].xid));
 
 		/* Advance global latestCompletedXid while holding the lock */
 		if (TransactionIdPrecedes(ShmemVariableCache->latestCompletedXid,
@@ -311,15 +339,17 @@ ProcArrayRemove(PGPROC *proc, TransactionId latestXid)
 	else
 	{
 		/* Shouldn't be trying to remove a live transaction here */
-		Assert(!TransactionIdIsValid(proc->xid));
+		Assert(!TransactionIdIsValid(allPgXact[proc->pgprocno].xid));
 	}
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		if (arrayP->procs[index] == proc)
+		if (arrayP->pgprocnos[index] == proc->pgprocno)
 		{
-			arrayP->procs[index] = arrayP->procs[arrayP->numProcs - 1];
-			arrayP->procs[arrayP->numProcs - 1] = NULL; /* for debugging */
+			/* Keep the PGPROC array sorted. See notes above */
+			memmove(&arrayP->pgprocnos[index], &arrayP->pgprocnos[index + 1],
+					(arrayP->numProcs - index - 1) * sizeof (int));
+			arrayP->pgprocnos[arrayP->numProcs - 1] = -1; /* for debugging */
 			arrayP->numProcs--;
 			LWLockRelease(ProcArrayLock);
 			return;
@@ -349,29 +379,31 @@ ProcArrayRemove(PGPROC *proc, TransactionId latestXid)
 void
 ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid)
 {
+	PGXACT *pgxact = &allPgXact[proc->pgprocno];
+
 	if (TransactionIdIsValid(latestXid))
 	{
 		/*
-		 * We must lock ProcArrayLock while clearing proc->xid, so that we do
-		 * not exit the set of "running" transactions while someone else is
-		 * taking a snapshot.  See discussion in
+		 * We must lock ProcArrayLock while clearing our advertised XID, so
+		 * that we do not exit the set of "running" transactions while someone
+		 * else is taking a snapshot.  See discussion in
 		 * src/backend/access/transam/README.
 		 */
-		Assert(TransactionIdIsValid(proc->xid));
+		Assert(TransactionIdIsValid(allPgXact[proc->pgprocno].xid));
 
 		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
 
-		proc->xid = InvalidTransactionId;
+		pgxact->xid = InvalidTransactionId;
 		proc->lxid = InvalidLocalTransactionId;
-		proc->xmin = InvalidTransactionId;
+		pgxact->xmin = InvalidTransactionId;
 		/* must be cleared with xid/xmin: */
-		proc->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
-		proc->inCommit = false; /* be sure this is cleared in abort */
+		pgxact->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
+		pgxact->inCommit = false; /* be sure this is cleared in abort */
 		proc->recoveryConflictPending = false;
 
 		/* Clear the subtransaction-XID cache too while holding the lock */
-		proc->subxids.nxids = 0;
-		proc->subxids.overflowed = false;
+		pgxact->nxids = 0;
+		pgxact->overflowed = false;
 
 		/* Also advance global latestCompletedXid while holding the lock */
 		if (TransactionIdPrecedes(ShmemVariableCache->latestCompletedXid,
@@ -387,17 +419,17 @@ ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid)
 		 * anyone else's calculation of a snapshot.  We might change their
 		 * estimate of global xmin, but that's OK.
 		 */
-		Assert(!TransactionIdIsValid(proc->xid));
+		Assert(!TransactionIdIsValid(allPgXact[proc->pgprocno].xid));
 
 		proc->lxid = InvalidLocalTransactionId;
-		proc->xmin = InvalidTransactionId;
+		pgxact->xmin = InvalidTransactionId;
 		/* must be cleared with xid/xmin: */
-		proc->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
-		proc->inCommit = false; /* be sure this is cleared in abort */
+		pgxact->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
+		pgxact->inCommit = false; /* be sure this is cleared in abort */
 		proc->recoveryConflictPending = false;
 
-		Assert(proc->subxids.nxids == 0);
-		Assert(proc->subxids.overflowed == false);
+		Assert(pgxact->nxids == 0);
+		Assert(pgxact->overflowed == false);
 	}
 }
 
@@ -413,24 +445,26 @@ ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid)
 void
 ProcArrayClearTransaction(PGPROC *proc)
 {
+	PGXACT *pgxact = &allPgXact[proc->pgprocno];
+
 	/*
 	 * We can skip locking ProcArrayLock here, because this action does not
 	 * actually change anyone's view of the set of running XIDs: our entry is
 	 * duplicate with the gxact that has already been inserted into the
 	 * ProcArray.
 	 */
-	proc->xid = InvalidTransactionId;
+	pgxact->xid = InvalidTransactionId;
 	proc->lxid = InvalidLocalTransactionId;
-	proc->xmin = InvalidTransactionId;
+	pgxact->xmin = InvalidTransactionId;
 	proc->recoveryConflictPending = false;
 
 	/* redundant, but just in case */
-	proc->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
-	proc->inCommit = false;
+	pgxact->vacuumFlags &= ~PROC_VACUUM_STATE_MASK;
+	pgxact->inCommit = false;
 
 	/* Clear the subtransaction-XID cache too */
-	proc->subxids.nxids = 0;
-	proc->subxids.overflowed = false;
+	pgxact->nxids = 0;
+	pgxact->overflowed = false;
 }
 
 /*
@@ -811,15 +845,17 @@ TransactionIdIsInProgress(TransactionId xid)
 	/* No shortcuts, gotta grovel through the array */
 	for (i = 0; i < arrayP->numProcs; i++)
 	{
-		volatile PGPROC *proc = arrayP->procs[i];
-		TransactionId pxid;
+		int		pgprocno = arrayP->pgprocnos[i];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
+		TransactionId	pxid;
 
 		/* Ignore my own proc --- dealt with it above */
 		if (proc == MyProc)
 			continue;
 
 		/* Fetch xid just once - see GetNewTransactionId */
-		pxid = proc->xid;
+		pxid = pgxact->xid;
 
 		if (!TransactionIdIsValid(pxid))
 			continue;
@@ -844,7 +880,7 @@ TransactionIdIsInProgress(TransactionId xid)
 		/*
 		 * Step 2: check the cached child-Xids arrays
 		 */
-		for (j = proc->subxids.nxids - 1; j >= 0; j--)
+		for (j = pgxact->nxids - 1; j >= 0; j--)
 		{
 			/* Fetch xid just once - see GetNewTransactionId */
 			TransactionId cxid = proc->subxids.xids[j];
@@ -864,7 +900,7 @@ TransactionIdIsInProgress(TransactionId xid)
 		 * we hold ProcArrayLock.  So we can't miss an Xid that we need to
 		 * worry about.)
 		 */
-		if (proc->subxids.overflowed)
+		if (pgxact->overflowed)
 			xids[nxids++] = pxid;
 	}
 
@@ -965,10 +1001,13 @@ TransactionIdIsActive(TransactionId xid)
 
 	for (i = 0; i < arrayP->numProcs; i++)
 	{
-		volatile PGPROC *proc = arrayP->procs[i];
+		int		pgprocno = arrayP->pgprocnos[i];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
+		TransactionId	pxid;
 
 		/* Fetch xid just once - see GetNewTransactionId */
-		TransactionId pxid = proc->xid;
+		pxid = pgxact->xid;
 
 		if (!TransactionIdIsValid(pxid))
 			continue;
@@ -1060,9 +1099,11 @@ GetOldestXmin(bool allDbs, bool ignoreVacuum)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
 
-		if (ignoreVacuum && (proc->vacuumFlags & PROC_IN_VACUUM))
+		if (ignoreVacuum && (pgxact->vacuumFlags & PROC_IN_VACUUM))
 			continue;
 
 		if (allDbs ||
@@ -1070,7 +1111,7 @@ GetOldestXmin(bool allDbs, bool ignoreVacuum)
 			proc->databaseId == 0)		/* always include WalSender */
 		{
 			/* Fetch xid just once - see GetNewTransactionId */
-			TransactionId xid = proc->xid;
+			TransactionId xid = pgxact->xid;
 
 			/* First consider the transaction's own Xid, if any */
 			if (TransactionIdIsNormal(xid) &&
@@ -1084,7 +1125,7 @@ GetOldestXmin(bool allDbs, bool ignoreVacuum)
 			 * have an Xmin but not (yet) an Xid; conversely, if it has an
 			 * Xid, that could determine some not-yet-set Xmin.
 			 */
-			xid = proc->xmin;	/* Fetch just once */
+			xid = pgxact->xmin;	/* Fetch just once */
 			if (TransactionIdIsNormal(xid) &&
 				TransactionIdPrecedes(xid, result))
 				result = xid;
@@ -1261,31 +1302,33 @@ GetSnapshotData(Snapshot snapshot)
 
 	if (!snapshot->takenDuringRecovery)
 	{
+		int *pgprocnos = arrayP->pgprocnos;
+		int			numProcs;
+
 		/*
 		 * Spin over procArray checking xid, xmin, and subxids.  The goal is
 		 * to gather all active xids, find the lowest xmin, and try to record
-		 * subxids. During recovery no xids will be assigned, so all normal
-		 * backends can be ignored, nor are there any VACUUMs running. All
-		 * prepared transaction xids are held in KnownAssignedXids, so these
-		 * will be seen without needing to loop through procs here.
+		 * subxids.
 		 */
-		for (index = 0; index < arrayP->numProcs; index++)
+		numProcs = arrayP->numProcs;
+		for (index = 0; index < numProcs; index++)
 		{
-			volatile PGPROC *proc = arrayP->procs[index];
-			TransactionId xid;
+			int		pgprocno = pgprocnos[index];
+			volatile PGXACT    *pgxact = &allPgXact[pgprocno];
+			TransactionId	xid;
 
 			/* Ignore procs running LAZY VACUUM */
-			if (proc->vacuumFlags & PROC_IN_VACUUM)
+			if (pgxact->vacuumFlags & PROC_IN_VACUUM)
 				continue;
 
 			/* Update globalxmin to be the smallest valid xmin */
-			xid = proc->xmin;	/* fetch just once */
+			xid = pgxact->xmin;	/* fetch just once */
 			if (TransactionIdIsNormal(xid) &&
 				TransactionIdPrecedes(xid, globalxmin))
-				globalxmin = xid;
+					globalxmin = xid;
 
 			/* Fetch xid just once - see GetNewTransactionId */
-			xid = proc->xid;
+			xid = pgxact->xid;
 
 			/*
 			 * If the transaction has been assigned an xid < xmax we add it to
@@ -1300,7 +1343,7 @@ GetSnapshotData(Snapshot snapshot)
 			{
 				if (TransactionIdFollowsOrEquals(xid, xmax))
 					continue;
-				if (proc != MyProc)
+				if (pgxact != MyPgXact)
 					snapshot->xip[count++] = xid;
 				if (TransactionIdPrecedes(xid, xmin))
 					xmin = xid;
@@ -1321,16 +1364,17 @@ GetSnapshotData(Snapshot snapshot)
 			 *
 			 * Again, our own XIDs are not included in the snapshot.
 			 */
-			if (!suboverflowed && proc != MyProc)
+			if (!suboverflowed && pgxact != MyPgXact)
 			{
-				if (proc->subxids.overflowed)
+				if (pgxact->overflowed)
 					suboverflowed = true;
 				else
 				{
-					int			nxids = proc->subxids.nxids;
+					int			nxids = pgxact->nxids;
 
 					if (nxids > 0)
 					{
+						volatile PGPROC *proc = &allProcs[pgprocno];
 						memcpy(snapshot->subxip + subcount,
 							   (void *) proc->subxids.xids,
 							   nxids * sizeof(TransactionId));
@@ -1372,9 +1416,8 @@ GetSnapshotData(Snapshot snapshot)
 			suboverflowed = true;
 	}
 
-	if (!TransactionIdIsValid(MyProc->xmin))
-		MyProc->xmin = TransactionXmin = xmin;
-
+	if (!TransactionIdIsValid(MyPgXact->xmin))
+		MyPgXact->xmin = TransactionXmin = xmin;
 	LWLockRelease(ProcArrayLock);
 
 	/*
@@ -1436,14 +1479,16 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
-		TransactionId xid;
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
+		TransactionId	xid;
 
 		/* Ignore procs running LAZY VACUUM */
-		if (proc->vacuumFlags & PROC_IN_VACUUM)
+		if (pgxact->vacuumFlags & PROC_IN_VACUUM)
 			continue;
 
-		xid = proc->xid;	/* fetch just once */
+		xid = pgxact->xid;	/* fetch just once */
 		if (xid != sourcexid)
 			continue;
 
@@ -1459,7 +1504,7 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
 		/*
 		 * Likewise, let's just make real sure its xmin does cover us.
 		 */
-		xid = proc->xmin;	/* fetch just once */
+		xid = pgxact->xmin;	/* fetch just once */
 		if (!TransactionIdIsNormal(xid) ||
 			!TransactionIdPrecedesOrEquals(xid, xmin))
 			continue;
@@ -1470,7 +1515,7 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
 		 * GetSnapshotData first, we'll be overwriting a valid xmin here,
 		 * so we don't check that.)
 		 */
-		MyProc->xmin = TransactionXmin = xmin;
+		MyPgXact->xmin = TransactionXmin = xmin;
 
 		result = true;
 		break;
@@ -1562,12 +1607,14 @@ GetRunningTransactionData(void)
 	 */
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC *proc = &allProcs[pgprocno];
+		volatile PGXACT *pgxact = &allPgXact[pgprocno];
 		TransactionId xid;
 		int			nxids;
 
 		/* Fetch xid just once - see GetNewTransactionId */
-		xid = proc->xid;
+		xid = pgxact->xid;
 
 		/*
 		 * We don't need to store transactions that don't have a TransactionId
@@ -1585,7 +1632,7 @@ GetRunningTransactionData(void)
 		 * Save subtransaction XIDs. Other backends can't add or remove
 		 * entries while we're holding XidGenLock.
 		 */
-		nxids = proc->subxids.nxids;
+		nxids = pgxact->nxids;
 		if (nxids > 0)
 		{
 			memcpy(&xids[count], (void *) proc->subxids.xids,
@@ -1593,7 +1640,7 @@ GetRunningTransactionData(void)
 			count += nxids;
 			subcount += nxids;
 
-			if (proc->subxids.overflowed)
+			if (pgxact->overflowed)
 				suboverflowed = true;
 
 			/*
@@ -1653,11 +1700,12 @@ GetOldestActiveTransactionId(void)
 	 */
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGXACT *pgxact = &allPgXact[pgprocno];
 		TransactionId xid;
 
 		/* Fetch xid just once - see GetNewTransactionId */
-		xid = proc->xid;
+		xid = pgxact->xid;
 
 		if (!TransactionIdIsNormal(xid))
 			continue;
@@ -1709,12 +1757,14 @@ GetTransactionsInCommit(TransactionId **xids_p)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGXACT *pgxact = &allPgXact[pgprocno];
+		TransactionId pxid;
 
 		/* Fetch xid just once - see GetNewTransactionId */
-		TransactionId pxid = proc->xid;
+		pxid = pgxact->xid;
 
-		if (proc->inCommit && TransactionIdIsValid(pxid))
+		if (pgxact->inCommit && TransactionIdIsValid(pxid))
 			xids[nxids++] = pxid;
 	}
 
@@ -1744,12 +1794,14 @@ HaveTransactionsInCommit(TransactionId *xids, int nxids)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
+		TransactionId	pxid;
 
 		/* Fetch xid just once - see GetNewTransactionId */
-		TransactionId pxid = proc->xid;
+		pxid = pgxact->xid;
 
-		if (proc->inCommit && TransactionIdIsValid(pxid))
+		if (pgxact->inCommit && TransactionIdIsValid(pxid))
 		{
 			int			i;
 
@@ -1792,7 +1844,7 @@ BackendPidGetProc(int pid)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		PGPROC	   *proc = arrayP->procs[index];
+		PGPROC	   *proc = &allProcs[arrayP->pgprocnos[index]];
 
 		if (proc->pid == pid)
 		{
@@ -1833,9 +1885,11 @@ BackendXidGetPid(TransactionId xid)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
 
-		if (proc->xid == xid)
+		if (pgxact->xid == xid)
 		{
 			result = proc->pid;
 			break;
@@ -1901,18 +1955,20 @@ GetCurrentVirtualXIDs(TransactionId limitXmin, bool excludeXmin0,
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
 
 		if (proc == MyProc)
 			continue;
 
-		if (excludeVacuum & proc->vacuumFlags)
+		if (excludeVacuum & pgxact->vacuumFlags)
 			continue;
 
 		if (allDbs || proc->databaseId == MyDatabaseId)
 		{
 			/* Fetch xmin just once - might change on us */
-			TransactionId pxmin = proc->xmin;
+			TransactionId pxmin = pgxact->xmin;
 
 			if (excludeXmin0 && !TransactionIdIsValid(pxmin))
 				continue;
@@ -1996,7 +2052,9 @@ GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
 
 		/* Exclude prepared transactions */
 		if (proc->pid == 0)
@@ -2006,7 +2064,7 @@ GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid)
 			proc->databaseId == dbOid)
 		{
 			/* Fetch xmin just once - can't change on us, but good coding */
-			TransactionId pxmin = proc->xmin;
+			TransactionId pxmin = pgxact->xmin;
 
 			/*
 			 * We ignore an invalid pxmin because this means that backend has
@@ -2050,8 +2108,9 @@ CancelVirtualTransaction(VirtualTransactionId vxid, ProcSignalReason sigmode)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		VirtualTransactionId procvxid;
-		PGPROC	   *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		VirtualTransactionId	procvxid;
 
 		GET_VXID_FROM_PGPROC(procvxid, *proc);
 
@@ -2104,7 +2163,9 @@ MinimumActiveBackends(int min)
 	 */
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int		pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC    *proc = &allProcs[pgprocno];
+		volatile PGXACT    *pgxact = &allPgXact[pgprocno];
 
 		/*
 		 * Since we're not holding a lock, need to check that the pointer is
@@ -2122,10 +2183,10 @@ MinimumActiveBackends(int min)
 
 		if (proc == MyProc)
 			continue;			/* do not count myself */
+		if (pgxact->xid == InvalidTransactionId)
+			continue;			/* do not count if no XID assigned */
 		if (proc->pid == 0)
 			continue;			/* do not count prepared xacts */
-		if (proc->xid == InvalidTransactionId)
-			continue;			/* do not count if no XID assigned */
 		if (proc->waitLock != NULL)
 			continue;			/* do not count if blocked on a lock */
 		count++;
@@ -2150,7 +2211,8 @@ CountDBBackends(Oid databaseid)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC *proc = &allProcs[pgprocno];
 
 		if (proc->pid == 0)
 			continue;			/* do not count prepared xacts */
@@ -2179,7 +2241,8 @@ CancelDBBackends(Oid databaseid, ProcSignalReason sigmode, bool conflictPending)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC *proc = &allProcs[pgprocno];
 
 		if (databaseid == InvalidOid || proc->databaseId == databaseid)
 		{
@@ -2217,7 +2280,8 @@ CountUserBackends(Oid roleid)
 
 	for (index = 0; index < arrayP->numProcs; index++)
 	{
-		volatile PGPROC *proc = arrayP->procs[index];
+		int pgprocno = arrayP->pgprocnos[index];
+		volatile PGPROC *proc = &allProcs[pgprocno];
 
 		if (proc->pid == 0)
 			continue;			/* do not count prepared xacts */
@@ -2277,7 +2341,9 @@ CountOtherDBBackends(Oid databaseId, int *nbackends, int *nprepared)
 
 		for (index = 0; index < arrayP->numProcs; index++)
 		{
-			volatile PGPROC *proc = arrayP->procs[index];
+			int pgprocno = arrayP->pgprocnos[index];
+			volatile PGPROC *proc = &allProcs[pgprocno];
+			volatile PGXACT *pgxact = &allPgXact[pgprocno];
 
 			if (proc->databaseId != databaseId)
 				continue;
@@ -2291,7 +2357,7 @@ CountOtherDBBackends(Oid databaseId, int *nbackends, int *nprepared)
 			else
 			{
 				(*nbackends)++;
-				if ((proc->vacuumFlags & PROC_IS_AUTOVACUUM) &&
+				if ((pgxact->vacuumFlags & PROC_IS_AUTOVACUUM) &&
 					nautovacs < MAXAUTOVACPIDS)
 					autovac_pids[nautovacs++] = proc->pid;
 			}
@@ -2321,8 +2387,8 @@ CountOtherDBBackends(Oid databaseId, int *nbackends, int *nprepared)
 
 #define XidCacheRemove(i) \
 	do { \
-		MyProc->subxids.xids[i] = MyProc->subxids.xids[MyProc->subxids.nxids - 1]; \
-		MyProc->subxids.nxids--; \
+		MyProc->subxids.xids[i] = MyProc->subxids.xids[MyPgXact->nxids - 1]; \
+		MyPgXact->nxids--; \
 	} while (0)
 
 /*
@@ -2361,7 +2427,7 @@ XidCacheRemoveRunningXids(TransactionId xid,
 	{
 		TransactionId anxid = xids[i];
 
-		for (j = MyProc->subxids.nxids - 1; j >= 0; j--)
+		for (j = MyPgXact->nxids - 1; j >= 0; j--)
 		{
 			if (TransactionIdEquals(MyProc->subxids.xids[j], anxid))
 			{
@@ -2377,11 +2443,11 @@ XidCacheRemoveRunningXids(TransactionId xid,
 		 * error during AbortSubTransaction.  So instead of Assert, emit a
 		 * debug warning.
 		 */
-		if (j < 0 && !MyProc->subxids.overflowed)
+		if (j < 0 && !MyPgXact->overflowed)
 			elog(WARNING, "did not find subXID %u in MyProc", anxid);
 	}
 
-	for (j = MyProc->subxids.nxids - 1; j >= 0; j--)
+	for (j = MyPgXact->nxids - 1; j >= 0; j--)
 	{
 		if (TransactionIdEquals(MyProc->subxids.xids[j], xid))
 		{
@@ -2390,7 +2456,7 @@ XidCacheRemoveRunningXids(TransactionId xid,
 		}
 	}
 	/* Ordinarily we should have found it, unless the cache has overflowed */
-	if (j < 0 && !MyProc->subxids.overflowed)
+	if (j < 0 && !MyPgXact->overflowed)
 		elog(WARNING, "did not find subXID %u in MyProc", xid);
 
 	/* Also advance global latestCompletedXid while holding the lock */
diff --git a/src/backend/storage/lmgr/deadlock.c b/src/backend/storage/lmgr/deadlock.c
index 7e7f6af21b7..63326b8770d 100644
--- a/src/backend/storage/lmgr/deadlock.c
+++ b/src/backend/storage/lmgr/deadlock.c
@@ -450,6 +450,7 @@ FindLockCycleRecurse(PGPROC *checkProc,
 					 int *nSoftEdges)	/* output argument */
 {
 	PGPROC	   *proc;
+	PGXACT	   *pgxact;
 	LOCK	   *lock;
 	PROCLOCK   *proclock;
 	SHM_QUEUE  *procLocks;
@@ -516,6 +517,7 @@ FindLockCycleRecurse(PGPROC *checkProc,
 	while (proclock)
 	{
 		proc = proclock->tag.myProc;
+		pgxact = &ProcGlobal->allPgXact[proc->pgprocno];
 
 		/* A proc never blocks itself */
 		if (proc != checkProc)
@@ -541,7 +543,7 @@ FindLockCycleRecurse(PGPROC *checkProc,
 					 * vacuumFlag bit), but we don't do that here to avoid
 					 * grabbing ProcArrayLock.
 					 */
-					if (proc->vacuumFlags & PROC_IS_AUTOVACUUM)
+					if (pgxact->vacuumFlags & PROC_IS_AUTOVACUUM)
 						blocking_autovacuum_proc = proc;
 
 					/* This proc hard-blocks checkProc */
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 905502f145b..3ba4671ac31 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -3188,9 +3188,10 @@ GetRunningTransactionLocks(int *nlocks)
 			proclock->tag.myLock->tag.locktag_type == LOCKTAG_RELATION)
 		{
 			PGPROC	   *proc = proclock->tag.myProc;
+			PGXACT	   *pgxact = &ProcGlobal->allPgXact[proc->pgprocno];
 			LOCK	   *lock = proclock->tag.myLock;
 
-			accessExclusiveLocks[index].xid = proc->xid;
+			accessExclusiveLocks[index].xid = pgxact->xid;
 			accessExclusiveLocks[index].dbOid = lock->tag.locktag_field1;
 			accessExclusiveLocks[index].relOid = lock->tag.locktag_field2;
 
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index eda3a98a85b..bcbc80222bb 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -36,6 +36,7 @@
 #include <sys/time.h>
 
 #include "access/transam.h"
+#include "access/twophase.h"
 #include "access/xact.h"
 #include "miscadmin.h"
 #include "postmaster/autovacuum.h"
@@ -57,6 +58,7 @@ bool		log_lock_waits = false;
 
 /* Pointer to this process's PGPROC struct, if any */
 PGPROC	   *MyProc = NULL;
+PGXACT	   *MyPgXact = NULL;
 
 /*
  * This spinlock protects the freelist of recycled PGPROC structures.
@@ -70,6 +72,7 @@ NON_EXEC_STATIC slock_t *ProcStructLock = NULL;
 /* Pointers to shared-memory structures */
 PROC_HDR *ProcGlobal = NULL;
 NON_EXEC_STATIC PGPROC *AuxiliaryProcs = NULL;
+PGPROC *PreparedXactProcs = NULL;
 
 /* If we are waiting for a lock, this points to the associated LOCALLOCK */
 static LOCALLOCK *lockAwaited = NULL;
@@ -106,13 +109,19 @@ ProcGlobalShmemSize(void)
 
 	/* ProcGlobal */
 	size = add_size(size, sizeof(PROC_HDR));
-	/* AuxiliaryProcs */
-	size = add_size(size, mul_size(NUM_AUXILIARY_PROCS, sizeof(PGPROC)));
 	/* MyProcs, including autovacuum workers and launcher */
 	size = add_size(size, mul_size(MaxBackends, sizeof(PGPROC)));
+	/* AuxiliaryProcs */
+	size = add_size(size, mul_size(NUM_AUXILIARY_PROCS, sizeof(PGPROC)));
+	/* Prepared xacts */
+	size = add_size(size, mul_size(max_prepared_xacts, sizeof(PGPROC)));
 	/* ProcStructLock */
 	size = add_size(size, sizeof(slock_t));
 
+	size = add_size(size, mul_size(MaxBackends, sizeof(PGXACT)));
+	size = add_size(size, mul_size(NUM_AUXILIARY_PROCS, sizeof(PGXACT)));
+	size = add_size(size, mul_size(max_prepared_xacts, sizeof(PGXACT)));
+
 	return size;
 }
 
@@ -157,10 +166,11 @@ void
 InitProcGlobal(void)
 {
 	PGPROC	   *procs;
+	PGXACT	   *pgxacts;
 	int			i,
 				j;
 	bool		found;
-	uint32		TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS;
+	uint32		TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS + max_prepared_xacts;
 
 	/* Create the ProcGlobal shared structure */
 	ProcGlobal = (PROC_HDR *)
@@ -182,10 +192,11 @@ InitProcGlobal(void)
 	 * those used for 2PC, which are embedded within a GlobalTransactionData
 	 * struct).
 	 *
-	 * There are three separate consumers of PGPROC structures: (1) normal
-	 * backends, (2) autovacuum workers and the autovacuum launcher, and (3)
-	 * auxiliary processes.  Each PGPROC structure is dedicated to exactly
-	 * one of these purposes, and they do not move between groups.
+	 * There are four separate consumers of PGPROC structures: (1) normal
+	 * backends, (2) autovacuum workers and the autovacuum launcher, (3)
+	 * auxiliary processes, and (4) prepared transactions.  Each PGPROC
+	 * structure is dedicated to exactly one of these purposes, and they do
+	 * not move between groups.
 	 */
 	procs = (PGPROC *) ShmemAlloc(TotalProcs * sizeof(PGPROC));
 	ProcGlobal->allProcs = procs;
@@ -195,21 +206,43 @@ InitProcGlobal(void)
 				(errcode(ERRCODE_OUT_OF_MEMORY),
 				 errmsg("out of shared memory")));
 	MemSet(procs, 0, TotalProcs * sizeof(PGPROC));
+
+	/*
+	 * Also allocate a separate array of PGXACT structures.  This is separate
+	 * from the main PGPROC array so that the most heavily accessed data is
+	 * stored contiguously in memory in as few cache lines as possible. This
+	 * provides significant performance benefits, especially on a
+	 * multiprocessor system.  Thereis one PGXACT structure for every PGPROC
+	 * structure.
+	 */
+	pgxacts = (PGXACT *) ShmemAlloc(TotalProcs * sizeof(PGXACT));
+	MemSet(pgxacts, 0, TotalProcs * sizeof(PGXACT));
+	ProcGlobal->allPgXact = pgxacts;
+
 	for (i = 0; i < TotalProcs; i++)
 	{
 		/* Common initialization for all PGPROCs, regardless of type. */
 
-		/* Set up per-PGPROC semaphore, latch, and backendLock */
-		PGSemaphoreCreate(&(procs[i].sem));
-		InitSharedLatch(&(procs[i].procLatch));
-		procs[i].backendLock = LWLockAssign();
+		/*
+		 * Set up per-PGPROC semaphore, latch, and backendLock. Prepared
+		 * xact dummy PGPROCs don't need these though - they're never
+		 * associated with a real process
+		 */
+		if (i < MaxBackends + NUM_AUXILIARY_PROCS)
+		{
+			PGSemaphoreCreate(&(procs[i].sem));
+			InitSharedLatch(&(procs[i].procLatch));
+			procs[i].backendLock = LWLockAssign();
+		}
+		procs[i].pgprocno = i;
 
 		/*
 		 * Newly created PGPROCs for normal backends or for autovacuum must
 		 * be queued up on the appropriate free list.  Because there can only
 		 * ever be a small, fixed number of auxiliary processes, no free
 		 * list is used in that case; InitAuxiliaryProcess() instead uses a
-		 * linear search.
+		 * linear search.  PGPROCs for prepared transactions are added to a
+		 * free list by TwoPhaseShmemInit().
 		 */
 		if (i < MaxConnections)
 		{
@@ -230,10 +263,11 @@ InitProcGlobal(void)
 	}
 
 	/*
-	 * Save a pointer to the block of PGPROC structures reserved for
-	 * auxiliary proceses.
+	 * Save pointers to the blocks of PGPROC structures reserved for
+	 * auxiliary processes and prepared transactions.
 	 */
 	AuxiliaryProcs = &procs[MaxBackends];
+	PreparedXactProcs = &procs[MaxBackends + NUM_AUXILIARY_PROCS];
 
 	/* Create ProcStructLock spinlock, too */
 	ProcStructLock = (slock_t *) ShmemAlloc(sizeof(slock_t));
@@ -296,6 +330,7 @@ InitProcess(void)
 				(errcode(ERRCODE_TOO_MANY_CONNECTIONS),
 				 errmsg("sorry, too many clients already")));
 	}
+	MyPgXact = &ProcGlobal->allPgXact[MyProc->pgprocno];
 
 	/*
 	 * Now that we have a PGPROC, mark ourselves as an active postmaster
@@ -313,18 +348,18 @@ InitProcess(void)
 	SHMQueueElemInit(&(MyProc->links));
 	MyProc->waitStatus = STATUS_OK;
 	MyProc->lxid = InvalidLocalTransactionId;
-	MyProc->xid = InvalidTransactionId;
-	MyProc->xmin = InvalidTransactionId;
+	MyPgXact->xid = InvalidTransactionId;
+	MyPgXact->xmin = InvalidTransactionId;
 	MyProc->pid = MyProcPid;
 	/* backendId, databaseId and roleId will be filled in later */
 	MyProc->backendId = InvalidBackendId;
 	MyProc->databaseId = InvalidOid;
 	MyProc->roleId = InvalidOid;
-	MyProc->inCommit = false;
-	MyProc->vacuumFlags = 0;
+	MyPgXact->inCommit = false;
+	MyPgXact->vacuumFlags = 0;
 	/* NB -- autovac launcher intentionally does not set IS_AUTOVACUUM */
 	if (IsAutoVacuumWorkerProcess())
-		MyProc->vacuumFlags |= PROC_IS_AUTOVACUUM;
+		MyPgXact->vacuumFlags |= PROC_IS_AUTOVACUUM;
 	MyProc->lwWaiting = false;
 	MyProc->lwExclusive = false;
 	MyProc->lwWaitLink = NULL;
@@ -462,6 +497,7 @@ InitAuxiliaryProcess(void)
 	((volatile PGPROC *) auxproc)->pid = MyProcPid;
 
 	MyProc = auxproc;
+	MyPgXact = &ProcGlobal->allPgXact[auxproc->pgprocno];
 
 	SpinLockRelease(ProcStructLock);
 
@@ -472,13 +508,13 @@ InitAuxiliaryProcess(void)
 	SHMQueueElemInit(&(MyProc->links));
 	MyProc->waitStatus = STATUS_OK;
 	MyProc->lxid = InvalidLocalTransactionId;
-	MyProc->xid = InvalidTransactionId;
-	MyProc->xmin = InvalidTransactionId;
+	MyPgXact->xid = InvalidTransactionId;
+	MyPgXact->xmin = InvalidTransactionId;
 	MyProc->backendId = InvalidBackendId;
 	MyProc->databaseId = InvalidOid;
 	MyProc->roleId = InvalidOid;
-	MyProc->inCommit = false;
-	MyProc->vacuumFlags = 0;
+	MyPgXact->inCommit = false;
+	MyPgXact->vacuumFlags = 0;
 	MyProc->lwWaiting = false;
 	MyProc->lwExclusive = false;
 	MyProc->lwWaitLink = NULL;
@@ -1045,6 +1081,7 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
 		if (deadlock_state == DS_BLOCKED_BY_AUTOVACUUM && allow_autovacuum_cancel)
 		{
 			PGPROC	   *autovac = GetBlockingAutoVacuumPgproc();
+			PGXACT	   *autovac_pgxact = &ProcGlobal->allPgXact[autovac->pgprocno];
 
 			LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
 
@@ -1053,8 +1090,8 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
 			 * wraparound.
 			 */
 			if ((autovac != NULL) &&
-				(autovac->vacuumFlags & PROC_IS_AUTOVACUUM) &&
-				!(autovac->vacuumFlags & PROC_VACUUM_FOR_WRAPAROUND))
+				(autovac_pgxact->vacuumFlags & PROC_IS_AUTOVACUUM) &&
+				!(autovac_pgxact->vacuumFlags & PROC_VACUUM_FOR_WRAPAROUND))
 			{
 				int			pid = autovac->pid;
 
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 50fb78057d8..814cd237399 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -577,7 +577,7 @@ static void
 SnapshotResetXmin(void)
 {
 	if (RegisteredSnapshots == 0 && ActiveSnapshot == NULL)
-		MyProc->xmin = InvalidTransactionId;
+		MyPgXact->xmin = InvalidTransactionId;
 }
 
 /*
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 6e798b1b2d9..c7cddc79931 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -35,8 +35,6 @@
 
 struct XidCache
 {
-	bool		overflowed;
-	int			nxids;
 	TransactionId xids[PGPROC_MAX_CACHED_SUBXIDS];
 };
 
@@ -86,27 +84,14 @@ struct PGPROC
 	LocalTransactionId lxid;	/* local id of top-level transaction currently
 								 * being executed by this proc, if running;
 								 * else InvalidLocalTransactionId */
-
-	TransactionId xid;			/* id of top-level transaction currently being
-								 * executed by this proc, if running and XID
-								 * is assigned; else InvalidTransactionId */
-
-	TransactionId xmin;			/* minimal running XID as it was when we were
-								 * starting our xact, excluding LAZY VACUUM:
-								 * vacuum must not remove tuples deleted by
-								 * xid >= xmin ! */
-
 	int			pid;			/* Backend's process ID; 0 if prepared xact */
+	int			pgprocno;
 
 	/* 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 */
 
-	bool		inCommit;		/* true if within commit critical section */
-
-	uint8		vacuumFlags;	/* vacuum-related flags, see above */
-
 	/*
 	 * While in hot standby mode, shows that a conflict signal has been sent
 	 * for the current transaction. Set/cleared while holding ProcArrayLock,
@@ -160,7 +145,33 @@ struct PGPROC
 
 
 extern PGDLLIMPORT PGPROC *MyProc;
+extern PGDLLIMPORT struct PGXACT *MyPgXact;
+
+/*
+ * Prior to PostgreSQL 9.2, the fieds below were stored as part of the
+ * PGPROC.  However, benchmarking revealed that packing these particular
+ * members into a separate array as tightly as possible sped up GetSnapshotData
+ * considerably on systems with many CPU cores, by reducing the number of
+ * cache lines needing to be fetched.  Thus, think very carefully before adding
+ * anything else here.
+ */
+typedef struct PGXACT
+{
+	TransactionId xid;			/* id of top-level transaction currently being
+								 * executed by this proc, if running and XID
+								 * is assigned; else InvalidTransactionId */
+
+	TransactionId xmin;			/* minimal running XID as it was when we were
+								 * starting our xact, excluding LAZY VACUUM:
+								 * vacuum must not remove tuples deleted by
+								 * xid >= xmin ! */
+
+	uint8		vacuumFlags;	/* vacuum-related flags, see above */
+	bool		overflowed;
+	bool		inCommit;		/* true if within commit critical section */
 
+	uint8		nxids;
+} PGXACT;
 
 /*
  * There is one ProcGlobal struct for the whole database cluster.
@@ -169,6 +180,8 @@ typedef struct PROC_HDR
 {
 	/* Array of PGPROC structures (not including dummies for prepared txns) */
 	PGPROC	   *allProcs;
+	/* Array of PGXACT structures (not including dummies for prepared txns */
+	PGXACT	   *allPgXact;
 	/* Length of allProcs array */
 	uint32		allProcCount;
 	/* Head of list of free PGPROC structures */
@@ -186,6 +199,8 @@ typedef struct PROC_HDR
 
 extern PROC_HDR *ProcGlobal;
 
+extern PGPROC *PreparedXactProcs;
+
 /*
  * We set aside some extra PGPROC structures for auxiliary processes,
  * ie things that aren't full-fledged backends but need shmem access.
-- 
GitLab