1
0
mirror of https://github.com/postgres/postgres.git synced 2025-07-05 07:21:24 +03:00

Allow read only connections during recovery, known as Hot Standby.

Enabled by recovery_connections = on (default) and forcing archive recovery using a recovery.conf. Recovery processing now emulates the original transactions as they are replayed, providing full locking and MVCC behaviour for read only queries. Recovery must enter consistent state before connections are allowed, so there is a delay, typically short, before connections succeed. Replay of recovering transactions can conflict and in some cases deadlock with queries during recovery; these result in query cancellation after max_standby_delay seconds have expired. Infrastructure changes have minor effects on normal running, though introduce four new types of WAL record.

New test mode "make standbycheck" allows regression tests of static command behaviour on a standby server while in recovery. Typical and extreme dynamic behaviours have been checked via code inspection and manual testing. Few port specific behaviours have been utilised, though primary testing has been on Linux only so far.

This commit is the basic patch. Additional changes will follow in this release to enhance some aspects of behaviour, notably improved handling of conflicts, deadlock detection and query cancellation. Changes to VACUUM FULL are also required.

Simon Riggs, with significant and lengthy review by Heikki Linnakangas, including streamlined redesign of snapshot creation and two-phase commit.

Important contributions from Florian Pflug, Mark Kirkwood, Merlin Moncure, Greg Stark, Gianni Ciolli, Gabriele Bartolini, Hannu Krosing, Robert Haas, Tatsuo Ishii, Hiroyuki Yamada plus support and feedback from many other community members.
This commit is contained in:
Simon Riggs
2009-12-19 01:32:45 +00:00
parent 78a09145e0
commit efc16ea520
87 changed files with 6165 additions and 428 deletions

View File

@ -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,13 +937,24 @@ 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;
/* Tell bufmgr and smgr to prepare for commit */
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
@ -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;
@ -1475,6 +1595,11 @@ StartTransaction(void)
currentCommandId = FirstCommandId;
currentCommandIdUsed = false;
/*
* initialize reported xid accounting
*/
nUnreportedXids = 0;
/*
* must initialize resource-management stuff first
*/
@ -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");
}