diff options
Diffstat (limited to 'src/backend/access/transam/xact.c')
-rw-r--r-- | src/backend/access/transam/xact.c | 378 |
1 files changed, 344 insertions, 34 deletions
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c index f9a71760d38..a1656922779 100644 --- a/src/backend/access/transam/xact.c +++ b/src/backend/access/transam/xact.c @@ -10,7 +10,7 @@ * * * IDENTIFICATION - * $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.277 2009/12/09 21:57:50 tgl Exp $ + * $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.278 2009/12/19 01:32:33 sriggs Exp $ * *------------------------------------------------------------------------- */ @@ -42,6 +42,7 @@ #include "storage/procarray.h" #include "storage/sinvaladt.h" #include "storage/smgr.h" +#include "storage/standby.h" #include "utils/combocid.h" #include "utils/guc.h" #include "utils/inval.h" @@ -139,6 +140,7 @@ typedef struct TransactionStateData Oid prevUser; /* previous CurrentUserId setting */ int prevSecContext; /* previous SecurityRestrictionContext */ bool prevXactReadOnly; /* entry-time xact r/o state */ + bool startedInRecovery; /* did we start in recovery? */ struct TransactionStateData *parent; /* back link to parent */ } TransactionStateData; @@ -167,9 +169,17 @@ static TransactionStateData TopTransactionStateData = { InvalidOid, /* previous CurrentUserId setting */ 0, /* previous SecurityRestrictionContext */ false, /* entry-time xact r/o state */ + false, /* startedInRecovery */ NULL /* link to parent state block */ }; +/* + * unreportedXids holds XIDs of all subtransactions that have not yet been + * reported in a XLOG_XACT_ASSIGNMENT record. + */ +static int nUnreportedXids; +static TransactionId unreportedXids[PGPROC_MAX_CACHED_SUBXIDS]; + static TransactionState CurrentTransactionState = &TopTransactionStateData; /* @@ -392,6 +402,9 @@ AssignTransactionId(TransactionState s) bool isSubXact = (s->parent != NULL); ResourceOwner currentOwner; + if (RecoveryInProgress()) + elog(ERROR, "cannot assign TransactionIds during recovery"); + /* Assert that caller didn't screw up */ Assert(!TransactionIdIsValid(s->transactionId)); Assert(s->state == TRANS_INPROGRESS); @@ -414,7 +427,7 @@ AssignTransactionId(TransactionState s) s->transactionId = GetNewTransactionId(isSubXact); if (isSubXact) - SubTransSetParent(s->transactionId, s->parent->transactionId); + SubTransSetParent(s->transactionId, s->parent->transactionId, false); /* * Acquire lock on the transaction XID. (We assume this cannot block.) We @@ -435,8 +448,57 @@ AssignTransactionId(TransactionState s) } PG_END_TRY(); CurrentResourceOwner = currentOwner; -} + /* + * Every PGPROC_MAX_CACHED_SUBXIDS assigned transaction ids within each + * top-level transaction we issue a WAL record for the assignment. We + * include the top-level xid and all the subxids that have not yet been + * reported using XLOG_XACT_ASSIGNMENT records. + * + * This is required to limit the amount of shared memory required in a + * hot standby server to keep track of in-progress XIDs. See notes for + * RecordKnownAssignedTransactionIds(). + * + * We don't keep track of the immediate parent of each subxid, + * only the top-level transaction that each subxact belongs to. This + * is correct in recovery only because aborted subtransactions are + * separately WAL logged. + */ + if (isSubXact && XLogStandbyInfoActive()) + { + unreportedXids[nUnreportedXids] = s->transactionId; + nUnreportedXids++; + + /* ensure this test matches similar one in RecoverPreparedTransactions() */ + if (nUnreportedXids >= PGPROC_MAX_CACHED_SUBXIDS) + { + XLogRecData rdata[2]; + xl_xact_assignment xlrec; + + /* + * xtop is always set by now because we recurse up transaction + * stack to the highest unassigned xid and then come back down + */ + xlrec.xtop = GetTopTransactionId(); + Assert(TransactionIdIsValid(xlrec.xtop)); + xlrec.nsubxacts = nUnreportedXids; + + rdata[0].data = (char *) &xlrec; + rdata[0].len = MinSizeOfXactAssignment; + rdata[0].buffer = InvalidBuffer; + rdata[0].next = &rdata[1]; + + rdata[1].data = (char *) unreportedXids; + rdata[1].len = PGPROC_MAX_CACHED_SUBXIDS * sizeof(TransactionId); + rdata[1].buffer = InvalidBuffer; + rdata[1].next = NULL; + + (void) XLogInsert(RM_XACT_ID, XLOG_XACT_ASSIGNMENT, rdata); + + nUnreportedXids = 0; + } + } +} /* * GetCurrentSubTransactionId @@ -596,6 +658,18 @@ TransactionIdIsCurrentTransactionId(TransactionId xid) return false; } +/* + * TransactionStartedDuringRecovery + * + * Returns true if the current transaction started while recovery was still + * in progress. Recovery might have ended since so RecoveryInProgress() might + * return false already. + */ +bool +TransactionStartedDuringRecovery(void) +{ + return CurrentTransactionState->startedInRecovery; +} /* * CommandCounterIncrement @@ -811,7 +885,7 @@ AtSubStart_ResourceOwner(void) * This is exported only to support an ugly hack in VACUUM FULL. */ TransactionId -RecordTransactionCommit(void) +RecordTransactionCommit(bool isVacuumFull) { TransactionId xid = GetTopTransactionIdIfAny(); bool markXidCommitted = TransactionIdIsValid(xid); @@ -821,11 +895,15 @@ RecordTransactionCommit(void) bool haveNonTemp; int nchildren; TransactionId *children; + int nmsgs; + SharedInvalidationMessage *invalMessages = NULL; + bool RelcacheInitFileInval; /* Get data needed for commit record */ nrels = smgrGetPendingDeletes(true, &rels, &haveNonTemp); nchildren = xactGetCommittedChildren(&children); - + nmsgs = xactGetCommittedInvalidationMessages(&invalMessages, + &RelcacheInitFileInval); /* * If we haven't been assigned an XID yet, we neither can, nor do we want * to write a COMMIT record. @@ -859,7 +937,7 @@ RecordTransactionCommit(void) /* * Begin commit critical section and insert the commit XLOG record. */ - XLogRecData rdata[3]; + XLogRecData rdata[4]; int lastrdata = 0; xl_xact_commit xlrec; @@ -867,6 +945,17 @@ RecordTransactionCommit(void) BufmgrCommit(); /* + * Set flags required for recovery processing of commits. + */ + xlrec.xinfo = 0; + if (RelcacheInitFileInval) + xlrec.xinfo |= XACT_COMPLETION_UPDATE_RELCACHE_FILE; + if (isVacuumFull) + xlrec.xinfo |= XACT_COMPLETION_VACUUM_FULL; + if (forceSyncCommit) + xlrec.xinfo |= XACT_COMPLETION_FORCE_SYNC_COMMIT; + + /* * Mark ourselves as within our "commit critical section". This * forces any concurrent checkpoint to wait until we've updated * pg_clog. Without this, it is possible for the checkpoint to set @@ -890,6 +979,7 @@ RecordTransactionCommit(void) xlrec.xact_time = xactStopTimestamp; xlrec.nrels = nrels; xlrec.nsubxacts = nchildren; + xlrec.nmsgs = nmsgs; rdata[0].data = (char *) (&xlrec); rdata[0].len = MinSizeOfXactCommit; rdata[0].buffer = InvalidBuffer; @@ -911,6 +1001,15 @@ RecordTransactionCommit(void) rdata[2].buffer = InvalidBuffer; lastrdata = 2; } + /* dump shared cache invalidation messages */ + if (nmsgs > 0) + { + rdata[lastrdata].next = &(rdata[3]); + rdata[3].data = (char *) invalMessages; + rdata[3].len = nmsgs * sizeof(SharedInvalidationMessage); + rdata[3].buffer = InvalidBuffer; + lastrdata = 3; + } rdata[lastrdata].next = NULL; (void) XLogInsert(RM_XACT_ID, XLOG_XACT_COMMIT, rdata); @@ -1352,6 +1451,13 @@ AtSubAbort_childXids(void) s->childXids = NULL; s->nChildXids = 0; s->maxChildXids = 0; + + /* + * We could prune the unreportedXids array here. But we don't bother. + * That would potentially reduce number of XLOG_XACT_ASSIGNMENT records + * but it would likely introduce more CPU time into the more common + * paths, so we choose not to do that. + */ } /* ---------------------------------------------------------------- @@ -1461,9 +1567,23 @@ StartTransaction(void) /* * Make sure we've reset xact state variables + * + * If recovery is still in progress, mark this transaction as read-only. + * We have lower level defences in XLogInsert and elsewhere to stop us + * from modifying data during recovery, but this gives the normal + * indication to the user that the transaction is read-only. */ + if (RecoveryInProgress()) + { + s->startedInRecovery = true; + XactReadOnly = true; + } + else + { + s->startedInRecovery = false; + XactReadOnly = DefaultXactReadOnly; + } XactIsoLevel = DefaultXactIsoLevel; - XactReadOnly = DefaultXactReadOnly; forceSyncCommit = false; MyXactAccessedTempRel = false; @@ -1476,6 +1596,11 @@ StartTransaction(void) currentCommandIdUsed = false; /* + * initialize reported xid accounting + */ + nUnreportedXids = 0; + + /* * must initialize resource-management stuff first */ AtStart_Memory(); @@ -1619,7 +1744,7 @@ CommitTransaction(void) /* * Here is where we really truly commit. */ - latestXid = RecordTransactionCommit(); + latestXid = RecordTransactionCommit(false); TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid); @@ -1853,7 +1978,6 @@ PrepareTransaction(void) StartPrepare(gxact); AtPrepare_Notify(); - AtPrepare_Inval(); AtPrepare_Locks(); AtPrepare_PgStat(); AtPrepare_MultiXact(); @@ -4199,29 +4323,108 @@ xactGetCommittedChildren(TransactionId **ptr) * XLOG support routines */ +/* + * Before 8.5 this was a fairly short function, but now it performs many + * actions for which the order of execution is critical. + */ static void -xact_redo_commit(xl_xact_commit *xlrec, TransactionId xid) +xact_redo_commit(xl_xact_commit *xlrec, TransactionId xid, XLogRecPtr lsn) { TransactionId *sub_xids; + SharedInvalidationMessage *inval_msgs; TransactionId max_xid; int i; - /* Mark the transaction committed in pg_clog */ + /* subxid array follows relfilenodes */ sub_xids = (TransactionId *) &(xlrec->xnodes[xlrec->nrels]); - TransactionIdCommitTree(xid, xlrec->nsubxacts, sub_xids); + /* invalidation messages array follows subxids */ + inval_msgs = (SharedInvalidationMessage *) &(sub_xids[xlrec->nsubxacts]); - /* Make sure nextXid is beyond any XID mentioned in the record */ - max_xid = xid; - for (i = 0; i < xlrec->nsubxacts; i++) - { - if (TransactionIdPrecedes(max_xid, sub_xids[i])) - max_xid = sub_xids[i]; - } + max_xid = TransactionIdLatest(xid, xlrec->nsubxacts, sub_xids); + + /* + * Make sure nextXid is beyond any XID mentioned in the record. + * + * We don't expect anyone else to modify nextXid, hence we + * don't need to hold a lock while checking this. We still acquire + * the lock to modify it, though. + */ if (TransactionIdFollowsOrEquals(max_xid, ShmemVariableCache->nextXid)) { + LWLockAcquire(XidGenLock, LW_EXCLUSIVE); ShmemVariableCache->nextXid = max_xid; TransactionIdAdvance(ShmemVariableCache->nextXid); + LWLockRelease(XidGenLock); + } + + if (!InHotStandby || XactCompletionVacuumFull(xlrec)) + { + /* + * Mark the transaction committed in pg_clog. + * + * If InHotStandby and this is the first commit of a VACUUM FULL INPLACE + * we perform only the actual commit to clog. Strangely, there are two + * commits that share the same xid for every VFI, so we need to skip + * some steps for the first commit. It's OK to repeat the clog update + * when we see the second commit on a VFI. + */ + TransactionIdCommitTree(xid, xlrec->nsubxacts, sub_xids); + } + else + { + /* + * If a transaction completion record arrives that has as-yet unobserved + * subtransactions then this will not have been fully handled by the call + * to RecordKnownAssignedTransactionIds() in the main recovery loop in + * xlog.c. So we need to do bookkeeping again to cover that case. This is + * confusing and it is easy to think this call is irrelevant, which has + * happened three times in development already. Leave it in. + */ + RecordKnownAssignedTransactionIds(max_xid); + + /* + * Mark the transaction committed in pg_clog. We use async commit + * protocol during recovery to provide information on database + * consistency for when users try to set hint bits. It is important + * that we do not set hint bits until the minRecoveryPoint is past + * this commit record. This ensures that if we crash we don't see + * hint bits set on changes made by transactions that haven't yet + * recovered. It's unlikely but it's good to be safe. + */ + TransactionIdAsyncCommitTree(xid, xlrec->nsubxacts, sub_xids, lsn); + + /* + * We must mark clog before we update the ProcArray. + */ + ExpireTreeKnownAssignedTransactionIds(xid, xlrec->nsubxacts, sub_xids); + + /* + * Send any cache invalidations attached to the commit. We must + * maintain the same order of invalidation then release locks + * as occurs in . + */ + if (xlrec->nmsgs > 0) + { + /* + * Relcache init file invalidation requires processing both + * before and after we send the SI messages. See AtEOXact_Inval() + */ + if (XactCompletionRelcacheInitFileInval(xlrec)) + RelationCacheInitFileInvalidate(true); + + SendSharedInvalidMessages(inval_msgs, xlrec->nmsgs); + + if (XactCompletionRelcacheInitFileInval(xlrec)) + RelationCacheInitFileInvalidate(false); + } + + /* + * Release locks, if any. We do this for both two phase and normal + * one phase transactions. In effect we are ignoring the prepare + * phase and just going straight to lock release. + */ + StandbyReleaseLockTree(xid, xlrec->nsubxacts, sub_xids); } /* Make sure files supposed to be dropped are dropped */ @@ -4240,8 +4443,31 @@ xact_redo_commit(xl_xact_commit *xlrec, TransactionId xid) } smgrclose(srel); } + + /* + * We issue an XLogFlush() for the same reason we emit ForceSyncCommit() in + * normal operation. For example, in DROP DATABASE, we delete all the files + * belonging to the database, and then commit the transaction. If we crash + * after all the files have been deleted but before the commit, you have an + * entry in pg_database without any files. To minimize the window for that, + * we use ForceSyncCommit() to rush the commit record to disk as quick as + * possible. We have the same window during recovery, and forcing an + * XLogFlush() (which updates minRecoveryPoint during recovery) helps + * to reduce that problem window, for any user that requested ForceSyncCommit(). + */ + if (XactCompletionForceSyncCommit(xlrec)) + XLogFlush(lsn); } +/* + * Be careful with the order of execution, as with xact_redo_commit(). + * The two functions are similar but differ in key places. + * + * Note also that an abort can be for a subtransaction and its children, + * not just for a top level abort. That means we have to consider + * topxid != xid, whereas in commit we would find topxid == xid always + * because subtransaction commit is never WAL logged. + */ static void xact_redo_abort(xl_xact_abort *xlrec, TransactionId xid) { @@ -4249,22 +4475,55 @@ xact_redo_abort(xl_xact_abort *xlrec, TransactionId xid) TransactionId max_xid; int i; - /* Mark the transaction aborted in pg_clog */ sub_xids = (TransactionId *) &(xlrec->xnodes[xlrec->nrels]); - TransactionIdAbortTree(xid, xlrec->nsubxacts, sub_xids); + max_xid = TransactionIdLatest(xid, xlrec->nsubxacts, sub_xids); /* Make sure nextXid is beyond any XID mentioned in the record */ - max_xid = xid; - for (i = 0; i < xlrec->nsubxacts; i++) - { - if (TransactionIdPrecedes(max_xid, sub_xids[i])) - max_xid = sub_xids[i]; - } + /* We don't expect anyone else to modify nextXid, hence we + * don't need to hold a lock while checking this. We still acquire + * the lock to modify it, though. + */ if (TransactionIdFollowsOrEquals(max_xid, ShmemVariableCache->nextXid)) { + LWLockAcquire(XidGenLock, LW_EXCLUSIVE); ShmemVariableCache->nextXid = max_xid; TransactionIdAdvance(ShmemVariableCache->nextXid); + LWLockRelease(XidGenLock); + } + + if (InHotStandby) + { + /* + * If a transaction completion record arrives that has as-yet unobserved + * subtransactions then this will not have been fully handled by the call + * to RecordKnownAssignedTransactionIds() in the main recovery loop in + * xlog.c. So we need to do bookkeeping again to cover that case. This is + * confusing and it is easy to think this call is irrelevant, which has + * happened three times in development already. Leave it in. + */ + RecordKnownAssignedTransactionIds(max_xid); + } + + /* Mark the transaction aborted in pg_clog, no need for async stuff */ + TransactionIdAbortTree(xid, xlrec->nsubxacts, sub_xids); + + if (InHotStandby) + { + /* + * We must mark clog before we update the ProcArray. + */ + ExpireTreeKnownAssignedTransactionIds(xid, xlrec->nsubxacts, sub_xids); + + /* + * There are no flat files that need updating, nor invalidation + * messages to send or undo. + */ + + /* + * Release locks, if any. There are no invalidations to send. + */ + StandbyReleaseLockTree(xid, xlrec->nsubxacts, sub_xids); } /* Make sure files supposed to be dropped are dropped */ @@ -4297,7 +4556,7 @@ xact_redo(XLogRecPtr lsn, XLogRecord *record) { xl_xact_commit *xlrec = (xl_xact_commit *) XLogRecGetData(record); - xact_redo_commit(xlrec, record->xl_xid); + xact_redo_commit(xlrec, record->xl_xid, lsn); } else if (info == XLOG_XACT_ABORT) { @@ -4315,7 +4574,7 @@ xact_redo(XLogRecPtr lsn, XLogRecord *record) { xl_xact_commit_prepared *xlrec = (xl_xact_commit_prepared *) XLogRecGetData(record); - xact_redo_commit(&xlrec->crec, xlrec->xid); + xact_redo_commit(&xlrec->crec, xlrec->xid, lsn); RemoveTwoPhaseFile(xlrec->xid, false); } else if (info == XLOG_XACT_ABORT_PREPARED) @@ -4325,6 +4584,14 @@ xact_redo(XLogRecPtr lsn, XLogRecord *record) xact_redo_abort(&xlrec->arec, xlrec->xid); RemoveTwoPhaseFile(xlrec->xid, false); } + else if (info == XLOG_XACT_ASSIGNMENT) + { + xl_xact_assignment *xlrec = (xl_xact_assignment *) XLogRecGetData(record); + + if (InHotStandby) + ProcArrayApplyXidAssignment(xlrec->xtop, + xlrec->nsubxacts, xlrec->xsub); + } else elog(PANIC, "xact_redo: unknown op code %u", info); } @@ -4333,6 +4600,14 @@ static void xact_desc_commit(StringInfo buf, xl_xact_commit *xlrec) { int i; + TransactionId *xacts; + SharedInvalidationMessage *msgs; + + xacts = (TransactionId *) &xlrec->xnodes[xlrec->nrels]; + msgs = (SharedInvalidationMessage *) &xacts[xlrec->nsubxacts]; + + if (XactCompletionRelcacheInitFileInval(xlrec)) + appendStringInfo(buf, "; relcache init file inval"); appendStringInfoString(buf, timestamptz_to_str(xlrec->xact_time)); if (xlrec->nrels > 0) @@ -4348,13 +4623,25 @@ xact_desc_commit(StringInfo buf, xl_xact_commit *xlrec) } if (xlrec->nsubxacts > 0) { - TransactionId *xacts = (TransactionId *) - &xlrec->xnodes[xlrec->nrels]; - appendStringInfo(buf, "; subxacts:"); for (i = 0; i < xlrec->nsubxacts; i++) appendStringInfo(buf, " %u", xacts[i]); } + if (xlrec->nmsgs > 0) + { + appendStringInfo(buf, "; inval msgs:"); + for (i = 0; i < xlrec->nmsgs; i++) + { + SharedInvalidationMessage *msg = &msgs[i]; + + if (msg->id >= 0) + appendStringInfo(buf, "catcache id%d ", msg->id); + else if (msg->id == SHAREDINVALRELCACHE_ID) + appendStringInfo(buf, "relcache "); + else if (msg->id == SHAREDINVALSMGR_ID) + appendStringInfo(buf, "smgr "); + } + } } static void @@ -4385,6 +4672,17 @@ xact_desc_abort(StringInfo buf, xl_xact_abort *xlrec) } } +static void +xact_desc_assignment(StringInfo buf, xl_xact_assignment *xlrec) +{ + int i; + + appendStringInfo(buf, "subxacts:"); + + for (i = 0; i < xlrec->nsubxacts; i++) + appendStringInfo(buf, " %u", xlrec->xsub[i]); +} + void xact_desc(StringInfo buf, uint8 xl_info, char *rec) { @@ -4412,16 +4710,28 @@ xact_desc(StringInfo buf, uint8 xl_info, char *rec) { xl_xact_commit_prepared *xlrec = (xl_xact_commit_prepared *) rec; - appendStringInfo(buf, "commit %u: ", xlrec->xid); + appendStringInfo(buf, "commit prepared %u: ", xlrec->xid); xact_desc_commit(buf, &xlrec->crec); } else if (info == XLOG_XACT_ABORT_PREPARED) { xl_xact_abort_prepared *xlrec = (xl_xact_abort_prepared *) rec; - appendStringInfo(buf, "abort %u: ", xlrec->xid); + appendStringInfo(buf, "abort prepared %u: ", xlrec->xid); xact_desc_abort(buf, &xlrec->arec); } + else if (info == XLOG_XACT_ASSIGNMENT) + { + xl_xact_assignment *xlrec = (xl_xact_assignment *) rec; + + /* + * Note that we ignore the WAL record's xid, since we're more + * interested in the top-level xid that issued the record + * and which xids are being reported here. + */ + appendStringInfo(buf, "xid assignment xtop %u: ", xlrec->xtop); + xact_desc_assignment(buf, xlrec); + } else appendStringInfo(buf, "UNKNOWN"); } |