Commit ed0b409d authored by Robert Haas's avatar Robert Haas

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
parent 9ed439a9
This diff is collapsed.
...@@ -54,7 +54,7 @@ GetNewTransactionId(bool isSubXact) ...@@ -54,7 +54,7 @@ GetNewTransactionId(bool isSubXact)
if (IsBootstrapProcessingMode()) if (IsBootstrapProcessingMode())
{ {
Assert(!isSubXact); Assert(!isSubXact);
MyProc->xid = BootstrapTransactionId; MyPgXact->xid = BootstrapTransactionId;
return BootstrapTransactionId; return BootstrapTransactionId;
} }
...@@ -208,20 +208,21 @@ GetNewTransactionId(bool isSubXact) ...@@ -208,20 +208,21 @@ GetNewTransactionId(bool isSubXact)
* TransactionId and int fetch/store are atomic. * TransactionId and int fetch/store are atomic.
*/ */
volatile PGPROC *myproc = MyProc; volatile PGPROC *myproc = MyProc;
volatile PGXACT *mypgxact = MyPgXact;
if (!isSubXact) if (!isSubXact)
myproc->xid = xid; mypgxact->xid = xid;
else else
{ {
int nxids = myproc->subxids.nxids; int nxids = mypgxact->nxids;
if (nxids < PGPROC_MAX_CACHED_SUBXIDS) if (nxids < PGPROC_MAX_CACHED_SUBXIDS)
{ {
myproc->subxids.xids[nxids] = xid; myproc->subxids.xids[nxids] = xid;
myproc->subxids.nxids = nxids + 1; mypgxact->nxids = nxids + 1;
} }
else else
myproc->subxids.overflowed = true; mypgxact->overflowed = true;
} }
} }
......
...@@ -981,7 +981,7 @@ RecordTransactionCommit(void) ...@@ -981,7 +981,7 @@ RecordTransactionCommit(void)
* bit fuzzy, but it doesn't matter. * bit fuzzy, but it doesn't matter.
*/ */
START_CRIT_SECTION(); START_CRIT_SECTION();
MyProc->inCommit = true; MyPgXact->inCommit = true;
SetCurrentTransactionStopTimestamp(); SetCurrentTransactionStopTimestamp();
...@@ -1155,7 +1155,7 @@ RecordTransactionCommit(void) ...@@ -1155,7 +1155,7 @@ RecordTransactionCommit(void)
*/ */
if (markXidCommitted) if (markXidCommitted)
{ {
MyProc->inCommit = false; MyPgXact->inCommit = false;
END_CRIT_SECTION(); END_CRIT_SECTION();
} }
......
...@@ -223,7 +223,7 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy) ...@@ -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. * OK, let's do it. First let other backends know I'm in ANALYZE.
*/ */
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE); LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
MyProc->vacuumFlags |= PROC_IN_ANALYZE; MyPgXact->vacuumFlags |= PROC_IN_ANALYZE;
LWLockRelease(ProcArrayLock); LWLockRelease(ProcArrayLock);
/* /*
...@@ -250,7 +250,7 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy) ...@@ -250,7 +250,7 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
* because the vacuum flag is cleared by the end-of-xact code. * because the vacuum flag is cleared by the end-of-xact code.
*/ */
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE); LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
MyProc->vacuumFlags &= ~PROC_IN_ANALYZE; MyPgXact->vacuumFlags &= ~PROC_IN_ANALYZE;
LWLockRelease(ProcArrayLock); LWLockRelease(ProcArrayLock);
} }
......
...@@ -893,9 +893,9 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound) ...@@ -893,9 +893,9 @@ vacuum_rel(Oid relid, VacuumStmt *vacstmt, bool do_toast, bool for_wraparound)
* which is probably Not Good. * which is probably Not Good.
*/ */
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE); LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
MyProc->vacuumFlags |= PROC_IN_VACUUM; MyPgXact->vacuumFlags |= PROC_IN_VACUUM;
if (for_wraparound) if (for_wraparound)
MyProc->vacuumFlags |= PROC_VACUUM_FOR_WRAPAROUND; MyPgXact->vacuumFlags |= PROC_VACUUM_FOR_WRAPAROUND;
LWLockRelease(ProcArrayLock); LWLockRelease(ProcArrayLock);
} }
......
...@@ -430,6 +430,7 @@ typedef struct ...@@ -430,6 +430,7 @@ typedef struct
slock_t *ProcStructLock; slock_t *ProcStructLock;
PROC_HDR *ProcGlobal; PROC_HDR *ProcGlobal;
PGPROC *AuxiliaryProcs; PGPROC *AuxiliaryProcs;
PGPROC *PreparedXactProcs;
PMSignalData *PMSignalState; PMSignalData *PMSignalState;
InheritableSocket pgStatSock; InheritableSocket pgStatSock;
pid_t PostmasterPid; pid_t PostmasterPid;
...@@ -4724,6 +4725,7 @@ save_backend_variables(BackendParameters *param, Port *port, ...@@ -4724,6 +4725,7 @@ save_backend_variables(BackendParameters *param, Port *port,
param->ProcStructLock = ProcStructLock; param->ProcStructLock = ProcStructLock;
param->ProcGlobal = ProcGlobal; param->ProcGlobal = ProcGlobal;
param->AuxiliaryProcs = AuxiliaryProcs; param->AuxiliaryProcs = AuxiliaryProcs;
param->PreparedXactProcs = PreparedXactProcs;
param->PMSignalState = PMSignalState; param->PMSignalState = PMSignalState;
if (!write_inheritable_socket(&param->pgStatSock, pgStatSock, childPid)) if (!write_inheritable_socket(&param->pgStatSock, pgStatSock, childPid))
return false; return false;
...@@ -4947,6 +4949,7 @@ restore_backend_variables(BackendParameters *param, Port *port) ...@@ -4947,6 +4949,7 @@ restore_backend_variables(BackendParameters *param, Port *port)
ProcStructLock = param->ProcStructLock; ProcStructLock = param->ProcStructLock;
ProcGlobal = param->ProcGlobal; ProcGlobal = param->ProcGlobal;
AuxiliaryProcs = param->AuxiliaryProcs; AuxiliaryProcs = param->AuxiliaryProcs;
PreparedXactProcs = param->PreparedXactProcs;
PMSignalState = param->PMSignalState; PMSignalState = param->PMSignalState;
read_inheritable_socket(&pgStatSock, &param->pgStatSock); read_inheritable_socket(&pgStatSock, &param->pgStatSock);
......
...@@ -702,7 +702,7 @@ ProcessStandbyHSFeedbackMessage(void) ...@@ -702,7 +702,7 @@ ProcessStandbyHSFeedbackMessage(void)
* safe, and if we're moving it backwards, well, the data is at risk * safe, and if we're moving it backwards, well, the data is at risk
* already since a VACUUM could have just finished calling GetOldestXmin.) * already since a VACUUM could have just finished calling GetOldestXmin.)
*/ */
MyProc->xmin = reply.xmin; MyPgXact->xmin = reply.xmin;
} }
/* Main loop of walsender process */ /* Main loop of walsender process */
......
...@@ -192,7 +192,6 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port) ...@@ -192,7 +192,6 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
XLOGShmemInit(); XLOGShmemInit();
CLOGShmemInit(); CLOGShmemInit();
SUBTRANSShmemInit(); SUBTRANSShmemInit();
TwoPhaseShmemInit();
MultiXactShmemInit(); MultiXactShmemInit();
InitBufferPool(); InitBufferPool();
...@@ -213,6 +212,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port) ...@@ -213,6 +212,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
InitProcGlobal(); InitProcGlobal();
CreateSharedProcArray(); CreateSharedProcArray();
CreateSharedBackendStatus(); CreateSharedBackendStatus();
TwoPhaseShmemInit();
/* /*
* Set up shared-inval messaging * Set up shared-inval messaging
......
This diff is collapsed.
...@@ -450,6 +450,7 @@ FindLockCycleRecurse(PGPROC *checkProc, ...@@ -450,6 +450,7 @@ FindLockCycleRecurse(PGPROC *checkProc,
int *nSoftEdges) /* output argument */ int *nSoftEdges) /* output argument */
{ {
PGPROC *proc; PGPROC *proc;
PGXACT *pgxact;
LOCK *lock; LOCK *lock;
PROCLOCK *proclock; PROCLOCK *proclock;
SHM_QUEUE *procLocks; SHM_QUEUE *procLocks;
...@@ -516,6 +517,7 @@ FindLockCycleRecurse(PGPROC *checkProc, ...@@ -516,6 +517,7 @@ FindLockCycleRecurse(PGPROC *checkProc,
while (proclock) while (proclock)
{ {
proc = proclock->tag.myProc; proc = proclock->tag.myProc;
pgxact = &ProcGlobal->allPgXact[proc->pgprocno];
/* A proc never blocks itself */ /* A proc never blocks itself */
if (proc != checkProc) if (proc != checkProc)
...@@ -541,7 +543,7 @@ FindLockCycleRecurse(PGPROC *checkProc, ...@@ -541,7 +543,7 @@ FindLockCycleRecurse(PGPROC *checkProc,
* vacuumFlag bit), but we don't do that here to avoid * vacuumFlag bit), but we don't do that here to avoid
* grabbing ProcArrayLock. * grabbing ProcArrayLock.
*/ */
if (proc->vacuumFlags & PROC_IS_AUTOVACUUM) if (pgxact->vacuumFlags & PROC_IS_AUTOVACUUM)
blocking_autovacuum_proc = proc; blocking_autovacuum_proc = proc;
/* This proc hard-blocks checkProc */ /* This proc hard-blocks checkProc */
......
...@@ -3188,9 +3188,10 @@ GetRunningTransactionLocks(int *nlocks) ...@@ -3188,9 +3188,10 @@ GetRunningTransactionLocks(int *nlocks)
proclock->tag.myLock->tag.locktag_type == LOCKTAG_RELATION) proclock->tag.myLock->tag.locktag_type == LOCKTAG_RELATION)
{ {
PGPROC *proc = proclock->tag.myProc; PGPROC *proc = proclock->tag.myProc;
PGXACT *pgxact = &ProcGlobal->allPgXact[proc->pgprocno];
LOCK *lock = proclock->tag.myLock; LOCK *lock = proclock->tag.myLock;
accessExclusiveLocks[index].xid = proc->xid; accessExclusiveLocks[index].xid = pgxact->xid;
accessExclusiveLocks[index].dbOid = lock->tag.locktag_field1; accessExclusiveLocks[index].dbOid = lock->tag.locktag_field1;
accessExclusiveLocks[index].relOid = lock->tag.locktag_field2; accessExclusiveLocks[index].relOid = lock->tag.locktag_field2;
......
...@@ -36,6 +36,7 @@ ...@@ -36,6 +36,7 @@
#include <sys/time.h> #include <sys/time.h>
#include "access/transam.h" #include "access/transam.h"
#include "access/twophase.h"
#include "access/xact.h" #include "access/xact.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "postmaster/autovacuum.h" #include "postmaster/autovacuum.h"
...@@ -57,6 +58,7 @@ bool log_lock_waits = false; ...@@ -57,6 +58,7 @@ bool log_lock_waits = false;
/* Pointer to this process's PGPROC struct, if any */ /* Pointer to this process's PGPROC struct, if any */
PGPROC *MyProc = NULL; PGPROC *MyProc = NULL;
PGXACT *MyPgXact = NULL;
/* /*
* This spinlock protects the freelist of recycled PGPROC structures. * This spinlock protects the freelist of recycled PGPROC structures.
...@@ -70,6 +72,7 @@ NON_EXEC_STATIC slock_t *ProcStructLock = NULL; ...@@ -70,6 +72,7 @@ NON_EXEC_STATIC slock_t *ProcStructLock = NULL;
/* Pointers to shared-memory structures */ /* Pointers to shared-memory structures */
PROC_HDR *ProcGlobal = NULL; PROC_HDR *ProcGlobal = NULL;
NON_EXEC_STATIC PGPROC *AuxiliaryProcs = NULL; NON_EXEC_STATIC PGPROC *AuxiliaryProcs = NULL;
PGPROC *PreparedXactProcs = NULL;
/* If we are waiting for a lock, this points to the associated LOCALLOCK */ /* If we are waiting for a lock, this points to the associated LOCALLOCK */
static LOCALLOCK *lockAwaited = NULL; static LOCALLOCK *lockAwaited = NULL;
...@@ -106,13 +109,19 @@ ProcGlobalShmemSize(void) ...@@ -106,13 +109,19 @@ ProcGlobalShmemSize(void)
/* ProcGlobal */ /* ProcGlobal */
size = add_size(size, sizeof(PROC_HDR)); 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 */ /* MyProcs, including autovacuum workers and launcher */
size = add_size(size, mul_size(MaxBackends, sizeof(PGPROC))); 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 */ /* ProcStructLock */
size = add_size(size, sizeof(slock_t)); 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; return size;
} }
...@@ -157,10 +166,11 @@ void ...@@ -157,10 +166,11 @@ void
InitProcGlobal(void) InitProcGlobal(void)
{ {
PGPROC *procs; PGPROC *procs;
PGXACT *pgxacts;
int i, int i,
j; j;
bool found; bool found;
uint32 TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS; uint32 TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS + max_prepared_xacts;
/* Create the ProcGlobal shared structure */ /* Create the ProcGlobal shared structure */
ProcGlobal = (PROC_HDR *) ProcGlobal = (PROC_HDR *)
...@@ -182,10 +192,11 @@ InitProcGlobal(void) ...@@ -182,10 +192,11 @@ InitProcGlobal(void)
* those used for 2PC, which are embedded within a GlobalTransactionData * those used for 2PC, which are embedded within a GlobalTransactionData
* struct). * struct).
* *
* There are three separate consumers of PGPROC structures: (1) normal * There are four separate consumers of PGPROC structures: (1) normal
* backends, (2) autovacuum workers and the autovacuum launcher, and (3) * backends, (2) autovacuum workers and the autovacuum launcher, (3)
* auxiliary processes. Each PGPROC structure is dedicated to exactly * auxiliary processes, and (4) prepared transactions. Each PGPROC
* one of these purposes, and they do not move between groups. * structure is dedicated to exactly one of these purposes, and they do
* not move between groups.
*/ */
procs = (PGPROC *) ShmemAlloc(TotalProcs * sizeof(PGPROC)); procs = (PGPROC *) ShmemAlloc(TotalProcs * sizeof(PGPROC));
ProcGlobal->allProcs = procs; ProcGlobal->allProcs = procs;
...@@ -195,21 +206,43 @@ InitProcGlobal(void) ...@@ -195,21 +206,43 @@ InitProcGlobal(void)
(errcode(ERRCODE_OUT_OF_MEMORY), (errcode(ERRCODE_OUT_OF_MEMORY),
errmsg("out of shared memory"))); errmsg("out of shared memory")));
MemSet(procs, 0, TotalProcs * sizeof(PGPROC)); 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++) for (i = 0; i < TotalProcs; i++)
{ {
/* Common initialization for all PGPROCs, regardless of type. */ /* Common initialization for all PGPROCs, regardless of type. */
/* Set up per-PGPROC semaphore, latch, and backendLock */ /*
PGSemaphoreCreate(&(procs[i].sem)); * Set up per-PGPROC semaphore, latch, and backendLock. Prepared
InitSharedLatch(&(procs[i].procLatch)); * xact dummy PGPROCs don't need these though - they're never
procs[i].backendLock = LWLockAssign(); * 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 * Newly created PGPROCs for normal backends or for autovacuum must
* be queued up on the appropriate free list. Because there can only * be queued up on the appropriate free list. Because there can only
* ever be a small, fixed number of auxiliary processes, no free * ever be a small, fixed number of auxiliary processes, no free
* list is used in that case; InitAuxiliaryProcess() instead uses a * 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) if (i < MaxConnections)
{ {
...@@ -230,10 +263,11 @@ InitProcGlobal(void) ...@@ -230,10 +263,11 @@ InitProcGlobal(void)
} }
/* /*
* Save a pointer to the block of PGPROC structures reserved for * Save pointers to the blocks of PGPROC structures reserved for
* auxiliary proceses. * auxiliary processes and prepared transactions.
*/ */
AuxiliaryProcs = &procs[MaxBackends]; AuxiliaryProcs = &procs[MaxBackends];
PreparedXactProcs = &procs[MaxBackends + NUM_AUXILIARY_PROCS];
/* Create ProcStructLock spinlock, too */ /* Create ProcStructLock spinlock, too */
ProcStructLock = (slock_t *) ShmemAlloc(sizeof(slock_t)); ProcStructLock = (slock_t *) ShmemAlloc(sizeof(slock_t));
...@@ -296,6 +330,7 @@ InitProcess(void) ...@@ -296,6 +330,7 @@ InitProcess(void)
(errcode(ERRCODE_TOO_MANY_CONNECTIONS), (errcode(ERRCODE_TOO_MANY_CONNECTIONS),
errmsg("sorry, too many clients already"))); errmsg("sorry, too many clients already")));
} }
MyPgXact = &ProcGlobal->allPgXact[MyProc->pgprocno];
/* /*
* Now that we have a PGPROC, mark ourselves as an active postmaster * Now that we have a PGPROC, mark ourselves as an active postmaster
...@@ -313,18 +348,18 @@ InitProcess(void) ...@@ -313,18 +348,18 @@ InitProcess(void)
SHMQueueElemInit(&(MyProc->links)); SHMQueueElemInit(&(MyProc->links));
MyProc->waitStatus = STATUS_OK; MyProc->waitStatus = STATUS_OK;
MyProc->lxid = InvalidLocalTransactionId; MyProc->lxid = InvalidLocalTransactionId;
MyProc->xid = InvalidTransactionId; MyPgXact->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId; MyPgXact->xmin = InvalidTransactionId;
MyProc->pid = MyProcPid; MyProc->pid = MyProcPid;
/* backendId, databaseId and roleId will be filled in later */ /* backendId, databaseId and roleId will be filled in later */
MyProc->backendId = InvalidBackendId; MyProc->backendId = InvalidBackendId;
MyProc->databaseId = InvalidOid; MyProc->databaseId = InvalidOid;
MyProc->roleId = InvalidOid; MyProc->roleId = InvalidOid;
MyProc->inCommit = false; MyPgXact->inCommit = false;
MyProc->vacuumFlags = 0; MyPgXact->vacuumFlags = 0;
/* NB -- autovac launcher intentionally does not set IS_AUTOVACUUM */ /* NB -- autovac launcher intentionally does not set IS_AUTOVACUUM */
if (IsAutoVacuumWorkerProcess()) if (IsAutoVacuumWorkerProcess())
MyProc->vacuumFlags |= PROC_IS_AUTOVACUUM; MyPgXact->vacuumFlags |= PROC_IS_AUTOVACUUM;
MyProc->lwWaiting = false; MyProc->lwWaiting = false;
MyProc->lwExclusive = false; MyProc->lwExclusive = false;
MyProc->lwWaitLink = NULL; MyProc->lwWaitLink = NULL;
...@@ -462,6 +497,7 @@ InitAuxiliaryProcess(void) ...@@ -462,6 +497,7 @@ InitAuxiliaryProcess(void)
((volatile PGPROC *) auxproc)->pid = MyProcPid; ((volatile PGPROC *) auxproc)->pid = MyProcPid;
MyProc = auxproc; MyProc = auxproc;
MyPgXact = &ProcGlobal->allPgXact[auxproc->pgprocno];
SpinLockRelease(ProcStructLock); SpinLockRelease(ProcStructLock);
...@@ -472,13 +508,13 @@ InitAuxiliaryProcess(void) ...@@ -472,13 +508,13 @@ InitAuxiliaryProcess(void)
SHMQueueElemInit(&(MyProc->links)); SHMQueueElemInit(&(MyProc->links));
MyProc->waitStatus = STATUS_OK; MyProc->waitStatus = STATUS_OK;
MyProc->lxid = InvalidLocalTransactionId; MyProc->lxid = InvalidLocalTransactionId;
MyProc->xid = InvalidTransactionId; MyPgXact->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId; MyPgXact->xmin = InvalidTransactionId;
MyProc->backendId = InvalidBackendId; MyProc->backendId = InvalidBackendId;
MyProc->databaseId = InvalidOid; MyProc->databaseId = InvalidOid;
MyProc->roleId = InvalidOid; MyProc->roleId = InvalidOid;
MyProc->inCommit = false; MyPgXact->inCommit = false;
MyProc->vacuumFlags = 0; MyPgXact->vacuumFlags = 0;
MyProc->lwWaiting = false; MyProc->lwWaiting = false;
MyProc->lwExclusive = false; MyProc->lwExclusive = false;
MyProc->lwWaitLink = NULL; MyProc->lwWaitLink = NULL;
...@@ -1045,6 +1081,7 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable) ...@@ -1045,6 +1081,7 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
if (deadlock_state == DS_BLOCKED_BY_AUTOVACUUM && allow_autovacuum_cancel) if (deadlock_state == DS_BLOCKED_BY_AUTOVACUUM && allow_autovacuum_cancel)
{ {
PGPROC *autovac = GetBlockingAutoVacuumPgproc(); PGPROC *autovac = GetBlockingAutoVacuumPgproc();
PGXACT *autovac_pgxact = &ProcGlobal->allPgXact[autovac->pgprocno];
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE); LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
...@@ -1053,8 +1090,8 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable) ...@@ -1053,8 +1090,8 @@ ProcSleep(LOCALLOCK *locallock, LockMethod lockMethodTable)
* wraparound. * wraparound.
*/ */
if ((autovac != NULL) && if ((autovac != NULL) &&
(autovac->vacuumFlags & PROC_IS_AUTOVACUUM) && (autovac_pgxact->vacuumFlags & PROC_IS_AUTOVACUUM) &&
!(autovac->vacuumFlags & PROC_VACUUM_FOR_WRAPAROUND)) !(autovac_pgxact->vacuumFlags & PROC_VACUUM_FOR_WRAPAROUND))
{ {
int pid = autovac->pid; int pid = autovac->pid;
......
...@@ -577,7 +577,7 @@ static void ...@@ -577,7 +577,7 @@ static void
SnapshotResetXmin(void) SnapshotResetXmin(void)
{ {
if (RegisteredSnapshots == 0 && ActiveSnapshot == NULL) if (RegisteredSnapshots == 0 && ActiveSnapshot == NULL)
MyProc->xmin = InvalidTransactionId; MyPgXact->xmin = InvalidTransactionId;
} }
/* /*
......
...@@ -35,8 +35,6 @@ ...@@ -35,8 +35,6 @@
struct XidCache struct XidCache
{ {
bool overflowed;
int nxids;
TransactionId xids[PGPROC_MAX_CACHED_SUBXIDS]; TransactionId xids[PGPROC_MAX_CACHED_SUBXIDS];
}; };
...@@ -86,27 +84,14 @@ struct PGPROC ...@@ -86,27 +84,14 @@ struct PGPROC
LocalTransactionId lxid; /* local id of top-level transaction currently LocalTransactionId lxid; /* local id of top-level transaction currently
* being executed by this proc, if running; * being executed by this proc, if running;
* else InvalidLocalTransactionId */ * 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 pid; /* Backend's process ID; 0 if prepared xact */
int pgprocno;
/* These fields are zero while a backend is still starting up: */ /* These fields are zero while a backend is still starting up: */
BackendId backendId; /* This backend's backend ID (if assigned) */ BackendId backendId; /* This backend's backend ID (if assigned) */
Oid databaseId; /* OID of database this backend is using */ Oid databaseId; /* OID of database this backend is using */
Oid roleId; /* OID of role using this backend */ 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 * While in hot standby mode, shows that a conflict signal has been sent
* for the current transaction. Set/cleared while holding ProcArrayLock, * for the current transaction. Set/cleared while holding ProcArrayLock,
...@@ -160,7 +145,33 @@ struct PGPROC ...@@ -160,7 +145,33 @@ struct PGPROC
extern PGDLLIMPORT PGPROC *MyProc; 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. * There is one ProcGlobal struct for the whole database cluster.
...@@ -169,6 +180,8 @@ typedef struct PROC_HDR ...@@ -169,6 +180,8 @@ typedef struct PROC_HDR
{ {
/* Array of PGPROC structures (not including dummies for prepared txns) */ /* Array of PGPROC structures (not including dummies for prepared txns) */
PGPROC *allProcs; PGPROC *allProcs;
/* Array of PGXACT structures (not including dummies for prepared txns */
PGXACT *allPgXact;
/* Length of allProcs array */ /* Length of allProcs array */
uint32 allProcCount; uint32 allProcCount;
/* Head of list of free PGPROC structures */ /* Head of list of free PGPROC structures */
...@@ -186,6 +199,8 @@ typedef struct PROC_HDR ...@@ -186,6 +199,8 @@ typedef struct PROC_HDR
extern PROC_HDR *ProcGlobal; extern PROC_HDR *ProcGlobal;
extern PGPROC *PreparedXactProcs;
/* /*
* We set aside some extra PGPROC structures for auxiliary processes, * We set aside some extra PGPROC structures for auxiliary processes,
* ie things that aren't full-fledged backends but need shmem access. * ie things that aren't full-fledged backends but need shmem access.
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment