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(¶m->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, ¶m->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