postgresql/src/backend/access/transam/xact.c

3037 lines
76 KiB
C

/*-------------------------------------------------------------------------
*
* xact.c
* top level transaction system support routines
*
* Portions Copyright (c) 1996-2003, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.170 2004/07/01 20:11:02 tgl Exp $
*
* NOTES
* Transaction aborts can now occur two ways:
*
* 1) system dies from some internal cause (syntax error, etc..)
* 2) user types ABORT
*
* These two cases used to be treated identically, but now
* we need to distinguish them. Why? consider the following
* two situations:
*
* case 1 case 2
* ------ ------
* 1) user types BEGIN 1) user types BEGIN
* 2) user does something 2) user does something
* 3) user does not like what 3) system aborts for some reason
* she sees and types ABORT
*
* In case 1, we want to abort the transaction and return to the
* default state. In case 2, there may be more commands coming
* our way which are part of the same transaction block and we have
* to ignore these commands until we see a COMMIT transaction or
* ROLLBACK.
*
* Internal aborts are now handled by AbortTransactionBlock(), just as
* they always have been, and user aborts are now handled by
* UserAbortTransactionBlock(). Both of them rely on AbortTransaction()
* to do all the real work. The only difference is what state we
* enter after AbortTransaction() does its work:
*
* * AbortTransactionBlock() leaves us in TBLOCK_ABORT and
* * UserAbortTransactionBlock() leaves us in TBLOCK_ENDABORT
*
* Low-level transaction abort handling is divided into two phases:
* * AbortTransaction() executes as soon as we realize the transaction
* has failed. It should release all shared resources (locks etc)
* so that we do not delay other backends unnecessarily.
* * CleanupTransaction() executes when we finally see a user COMMIT
* or ROLLBACK command; it cleans things up and gets us out of
* the transaction internally. In particular, we mustn't destroy
* TopTransactionContext until this point.
*
* NOTES
* The essential aspects of the transaction system are:
*
* o transaction id generation
* o transaction log updating
* o memory cleanup
* o cache invalidation
* o lock cleanup
*
* Hence, the functional division of the transaction code is
* based on which of the above things need to be done during
* a start/commit/abort transaction. For instance, the
* routine AtCommit_Memory() takes care of all the memory
* cleanup stuff done at commit time.
*
* The code is layered as follows:
*
* StartTransaction
* CommitTransaction
* AbortTransaction
* CleanupTransaction
*
* are provided to do the lower level work like recording
* the transaction status in the log and doing memory cleanup.
* above these routines are another set of functions:
*
* StartTransactionCommand
* CommitTransactionCommand
* AbortCurrentTransaction
*
* These are the routines used in the postgres main processing
* loop. They are sensitive to the current transaction block state
* and make calls to the lower level routines appropriately.
*
* Support for transaction blocks is provided via the functions:
*
* BeginTransactionBlock
* CommitTransactionBlock
* AbortTransactionBlock
*
* These are invoked only in response to a user "BEGIN WORK", "COMMIT",
* or "ROLLBACK" command. The tricky part about these functions
* is that they are called within the postgres main loop, in between
* the StartTransactionCommand() and CommitTransactionCommand().
*
* For example, consider the following sequence of user commands:
*
* 1) begin
* 2) select * from foo
* 3) insert into foo (bar = baz)
* 4) commit
*
* in the main processing loop, this results in the following
* transaction sequence:
*
* / StartTransactionCommand();
* 1) / ProcessUtility(); << begin
* \ BeginTransactionBlock();
* \ CommitTransactionCommand();
*
* / StartTransactionCommand();
* 2) < ProcessQuery(); << select * from foo
* \ CommitTransactionCommand();
*
* / StartTransactionCommand();
* 3) < ProcessQuery(); << insert into foo (bar = baz)
* \ CommitTransactionCommand();
*
* / StartTransactionCommand();
* 4) / ProcessUtility(); << commit
* \ CommitTransactionBlock();
* \ CommitTransactionCommand();
*
* The point of this example is to demonstrate the need for
* StartTransactionCommand() and CommitTransactionCommand() to
* be state smart -- they should do nothing in between the calls
* to BeginTransactionBlock() and EndTransactionBlock() and
* outside these calls they need to do normal start/commit
* processing.
*
* Furthermore, suppose the "select * from foo" caused an abort
* condition. We would then want to abort the transaction and
* ignore all subsequent commands up to the "commit".
* -cim 3/23/90
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include <time.h>
#include <unistd.h>
#include "access/gistscan.h"
#include "access/hash.h"
#include "access/nbtree.h"
#include "access/rtree.h"
#include "access/subtrans.h"
#include "access/xact.h"
#include "catalog/heap.h"
#include "catalog/index.h"
#include "catalog/namespace.h"
#include "commands/async.h"
#include "commands/tablecmds.h"
#include "commands/trigger.h"
#include "commands/user.h"
#include "executor/spi.h"
#include "libpq/be-fsstubs.h"
#include "miscadmin.h"
#include "storage/fd.h"
#include "storage/proc.h"
#include "storage/sinval.h"
#include "storage/smgr.h"
#include "utils/guc.h"
#include "utils/inval.h"
#include "utils/memutils.h"
#include "utils/portal.h"
#include "utils/catcache.h"
#include "utils/relcache.h"
#include "pgstat.h"
static void AbortTransaction(void);
static void AtAbort_Cache(void);
static void AtAbort_Locks(void);
static void AtAbort_Memory(void);
static void AtCleanup_Memory(void);
static void AtCommit_Cache(void);
static void AtCommit_LocalCache(void);
static void AtCommit_Locks(void);
static void AtCommit_Memory(void);
static void AtStart_Cache(void);
static void AtStart_Locks(void);
static void AtStart_Memory(void);
static void CallEOXactCallbacks(bool isCommit);
static void CleanupTransaction(void);
static void CommitTransaction(void);
static void RecordTransactionAbort(void);
static void StartTransaction(void);
static void RecordSubTransactionCommit(void);
static void StartSubTransaction(void);
static void CommitSubTransaction(void);
static void AbortSubTransaction(void);
static void CleanupSubTransaction(void);
static void StartAbortedSubTransaction(void);
static void PushTransaction(void);
static void PopTransaction(void);
static void AtSubAbort_Locks(void);
static void AtSubAbort_Memory(void);
static void AtSubCleanup_Memory(void);
static void AtSubCommit_Memory(void);
static void AtSubStart_Memory(void);
static void ShowTransactionState(const char *str);
static void ShowTransactionStateRec(TransactionState state);
static const char *BlockStateAsString(TBlockState blockState);
static const char *TransStateAsString(TransState state);
/*
* CurrentTransactionState always points to the current transaction state
* block. It will point to TopTransactionStateData when not in a
* transaction at all, or when in a top-level transaction.
*/
static TransactionStateData TopTransactionStateData = {
0, /* transaction id */
FirstCommandId, /* command id */
TRANS_DEFAULT, /* transaction state */
TBLOCK_DEFAULT, /* transaction block state from the client
* perspective */
0, /* nesting level */
NULL, /* cur transaction context */
NIL, /* subcommitted child Xids */
0, /* entry-time current userid */
NULL /* link to parent state block */
};
static TransactionState CurrentTransactionState = &TopTransactionStateData;
/*
* These vars hold the value of now(), ie, the transaction start time.
* This does not change as we enter and exit subtransactions, so we don't
* keep it inside the TransactionState stack.
*/
static AbsoluteTime xactStartTime; /* integer part */
static int xactStartTimeUsec; /* microsecond part */
/*
* User-tweakable parameters
*/
int DefaultXactIsoLevel = XACT_READ_COMMITTED;
int XactIsoLevel;
bool DefaultXactReadOnly = false;
bool XactReadOnly;
int CommitDelay = 0; /* precommit delay in microseconds */
int CommitSiblings = 5; /* number of concurrent xacts needed to
* sleep */
/*
* List of add-on end-of-xact callbacks
*/
typedef struct EOXactCallbackItem
{
struct EOXactCallbackItem *next;
EOXactCallback callback;
void *arg;
} EOXactCallbackItem;
static EOXactCallbackItem *EOXact_callbacks = NULL;
static void (*_RollbackFunc) (void *) = NULL;
static void *_RollbackData = NULL;
/* ----------------------------------------------------------------
* transaction state accessors
* ----------------------------------------------------------------
*/
/*
* IsTransactionState
*
* This returns true if we are currently running a query
* within an executing transaction.
*/
bool
IsTransactionState(void)
{
TransactionState s = CurrentTransactionState;
switch (s->state)
{
case TRANS_DEFAULT:
return false;
case TRANS_START:
return true;
case TRANS_INPROGRESS:
return true;
case TRANS_COMMIT:
return true;
case TRANS_ABORT:
return true;
}
/*
* Shouldn't get here, but lint is not happy with this...
*/
return false;
}
/*
* IsAbortedTransactionBlockState
*
* This returns true if we are currently running a query
* within an aborted transaction block.
*/
bool
IsAbortedTransactionBlockState(void)
{
TransactionState s = CurrentTransactionState;
if (s->blockState == TBLOCK_ABORT ||
s->blockState == TBLOCK_SUBABORT)
return true;
return false;
}
/*
* GetTopTransactionId
*
* Get the ID of the main transaction, even if we are currently inside
* a subtransaction.
*/
TransactionId
GetTopTransactionId(void)
{
return TopTransactionStateData.transactionIdData;
}
/*
* GetCurrentTransactionId
*/
TransactionId
GetCurrentTransactionId(void)
{
TransactionState s = CurrentTransactionState;
return s->transactionIdData;
}
/*
* GetCurrentCommandId
*/
CommandId
GetCurrentCommandId(void)
{
TransactionState s = CurrentTransactionState;
return s->commandId;
}
/*
* GetCurrentTransactionStartTime
*/
AbsoluteTime
GetCurrentTransactionStartTime(void)
{
return xactStartTime;
}
/*
* GetCurrentTransactionStartTimeUsec
*/
AbsoluteTime
GetCurrentTransactionStartTimeUsec(int *msec)
{
*msec = xactStartTimeUsec;
return xactStartTime;
}
/*
* GetCurrentTransactionNestLevel
*
* Note: this will return zero when not inside any transaction, one when
* inside a top-level transaction, etc.
*/
int
GetCurrentTransactionNestLevel(void)
{
TransactionState s = CurrentTransactionState;
return s->nestingLevel;
}
/*
* TransactionIdIsCurrentTransactionId
*
* During bootstrap, we cheat and say "it's not my transaction ID" even though
* it is. Along with transam.c's cheat to say that the bootstrap XID is
* already committed, this causes the tqual.c routines to see previously
* inserted tuples as committed, which is what we need during bootstrap.
*/
bool
TransactionIdIsCurrentTransactionId(TransactionId xid)
{
TransactionState s = CurrentTransactionState;
if (AMI_OVERRIDE)
{
Assert(xid == BootstrapTransactionId);
return false;
}
/*
* We will return true for the Xid of the current subtransaction,
* any of its subcommitted children, any of its parents, or any of
* their previously subcommitted children.
*/
while (s != NULL)
{
ListCell *cell;
if (TransactionIdEquals(xid, s->transactionIdData))
return true;
foreach(cell, s->childXids)
{
if (TransactionIdEquals(xid, lfirst_int(cell)))
return true;
}
s = s->parent;
}
return false;
}
/*
* CommandCounterIncrement
*/
void
CommandCounterIncrement(void)
{
TransactionState s = CurrentTransactionState;
s->commandId += 1;
if (s->commandId == FirstCommandId) /* check for overflow */
ereport(ERROR,
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("cannot have more than 2^32-1 commands in a transaction")));
/* Propagate new command ID into query snapshots, if set */
if (QuerySnapshot)
QuerySnapshot->curcid = s->commandId;
if (SerializableSnapshot)
SerializableSnapshot->curcid = s->commandId;
/*
* make cache changes visible to me. AtCommit_LocalCache() instead of
* AtCommit_Cache() is called here.
*/
AtCommit_LocalCache();
AtStart_Cache();
}
/* ----------------------------------------------------------------
* StartTransaction stuff
* ----------------------------------------------------------------
*/
/*
* AtStart_Cache
*/
static void
AtStart_Cache(void)
{
AcceptInvalidationMessages();
}
/*
* AtStart_Locks
*/
static void
AtStart_Locks(void)
{
/*
* at present, it is unknown to me what belongs here -cim 3/18/90
*
* There isn't anything to do at the start of a xact for locks. -mer
* 5/24/92
*/
}
/*
* AtStart_Memory
*/
static void
AtStart_Memory(void)
{
TransactionState s = CurrentTransactionState;
/*
* We shouldn't have a transaction context already.
*/
Assert(TopTransactionContext == NULL);
/*
* Create a toplevel context for the transaction.
*/
TopTransactionContext =
AllocSetContextCreate(TopMemoryContext,
"TopTransactionContext",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
/*
* In a top-level transaction, CurTransactionContext is the same as
* TopTransactionContext.
*/
CurTransactionContext = TopTransactionContext;
s->curTransactionContext = CurTransactionContext;
/* Make the CurTransactionContext active. */
MemoryContextSwitchTo(CurTransactionContext);
}
/* ----------------------------------------------------------------
* StartSubTransaction stuff
* ----------------------------------------------------------------
*/
/*
* AtSubStart_Memory
*/
static void
AtSubStart_Memory(void)
{
TransactionState s = CurrentTransactionState;
Assert(CurTransactionContext != NULL);
/*
* Create a CurTransactionContext, which will be used to hold data that
* survives subtransaction commit but disappears on subtransaction abort.
* We make it a child of the immediate parent's CurTransactionContext.
*/
CurTransactionContext = AllocSetContextCreate(CurTransactionContext,
"CurTransactionContext",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
s->curTransactionContext = CurTransactionContext;
/* Make the CurTransactionContext active. */
MemoryContextSwitchTo(CurTransactionContext);
}
/* ----------------------------------------------------------------
* CommitTransaction stuff
* ----------------------------------------------------------------
*/
/*
* RecordTransactionCommit
*/
void
RecordTransactionCommit(void)
{
int nrels;
RelFileNode *rptr;
int nchildren;
TransactionId *children;
/* Get data needed for commit record */
nrels = smgrGetPendingDeletes(true, &rptr);
nchildren = xactGetCommittedChildren(&children, false);
/*
* If we made neither any XLOG entries nor any temp-rel updates,
* and have no files to be deleted, we can omit recording the transaction
* commit at all. (This test includes the effects of subtransactions,
* so the presence of committed subxacts need not alone force a write.)
*/
if (MyXactMadeXLogEntry || MyXactMadeTempRelUpdate || nrels > 0)
{
TransactionId xid = GetCurrentTransactionId();
bool madeTCentries;
XLogRecPtr recptr;
/* Tell bufmgr and smgr to prepare for commit */
BufmgrCommit();
START_CRIT_SECTION();
/*
* We only need to log the commit in XLOG if the transaction made
* any transaction-controlled XLOG entries or will delete files.
* (If it made no transaction-controlled XLOG entries, its XID
* appears nowhere in permanent storage, so no one else will ever care
* if it committed.)
*/
madeTCentries = (MyLastRecPtr.xrecoff != 0);
if (madeTCentries || nrels > 0)
{
XLogRecData rdata[3];
int lastrdata = 0;
xl_xact_commit xlrec;
xlrec.xtime = time(NULL);
xlrec.nrels = nrels;
xlrec.nsubxacts = nchildren;
rdata[0].buffer = InvalidBuffer;
rdata[0].data = (char *) (&xlrec);
rdata[0].len = MinSizeOfXactCommit;
/* dump rels to delete */
if (nrels > 0)
{
rdata[0].next = &(rdata[1]);
rdata[1].buffer = InvalidBuffer;
rdata[1].data = (char *) rptr;
rdata[1].len = nrels * sizeof(RelFileNode);
lastrdata = 1;
}
/* dump committed child Xids */
if (nchildren > 0)
{
rdata[lastrdata].next = &(rdata[2]);
rdata[2].buffer = InvalidBuffer;
rdata[2].data = (char *) children;
rdata[2].len = nchildren * sizeof(TransactionId);
lastrdata = 2;
}
rdata[lastrdata].next = NULL;
recptr = XLogInsert(RM_XACT_ID, XLOG_XACT_COMMIT, rdata);
}
else
{
/* Just flush through last record written by me */
recptr = ProcLastRecEnd;
}
/*
* We must flush our XLOG entries to disk if we made any XLOG
* entries, whether in or out of transaction control. For
* example, if we reported a nextval() result to the client, this
* ensures that any XLOG record generated by nextval will hit the
* disk before we report the transaction committed.
*
* Note: if we generated a commit record above, MyXactMadeXLogEntry
* will certainly be set now.
*/
if (MyXactMadeXLogEntry)
{
/*
* Sleep before flush! So we can flush more than one commit
* records per single fsync. (The idea is some other backend
* may do the XLogFlush while we're sleeping. This needs work
* still, because on most Unixen, the minimum select() delay
* is 10msec or more, which is way too long.)
*
* We do not sleep if enableFsync is not turned on, nor if there
* are fewer than CommitSiblings other backends with active
* transactions.
*/
if (CommitDelay > 0 && enableFsync &&
CountActiveBackends() >= CommitSiblings)
pg_usleep(CommitDelay);
XLogFlush(recptr);
}
/*
* We must mark the transaction committed in clog if its XID
* appears either in permanent rels or in local temporary rels. We
* test this by seeing if we made transaction-controlled entries
* *OR* local-rel tuple updates. Note that if we made only the
* latter, we have not emitted an XLOG record for our commit, and
* so in the event of a crash the clog update might be lost. This
* is okay because no one else will ever care whether we
* committed.
*/
if (madeTCentries || MyXactMadeTempRelUpdate)
{
TransactionIdCommit(xid);
/* to avoid race conditions, the parent must commit first */
TransactionIdCommitTree(nchildren, children);
}
END_CRIT_SECTION();
}
/* Break the chain of back-links in the XLOG records I output */
MyLastRecPtr.xrecoff = 0;
MyXactMadeXLogEntry = false;
MyXactMadeTempRelUpdate = false;
/* Show myself as out of the transaction in PGPROC array */
MyProc->logRec.xrecoff = 0;
/* And clean up local data */
if (rptr)
pfree(rptr);
if (children)
pfree(children);
}
/*
* AtCommit_Cache
*/
static void
AtCommit_Cache(void)
{
/*
* Clean up the relation cache.
*/
AtEOXact_RelationCache(true);
/*
* Make catalog changes visible to all backends.
*/
AtEOXact_Inval(true);
}
/*
* AtCommit_LocalCache
*/
static void
AtCommit_LocalCache(void)
{
/*
* Make catalog changes visible to me for the next command.
*/
CommandEndInvalidationMessages();
}
/*
* AtCommit_Locks
*/
static void
AtCommit_Locks(void)
{
/*
* XXX What if ProcReleaseLocks fails? (race condition?)
*
* Then you're up a creek! -mer 5/24/92
*/
ProcReleaseLocks(ReleaseAllExceptSession, 0, NULL);
}
/*
* AtCommit_Memory
*/
static void
AtCommit_Memory(void)
{
/*
* Now that we're "out" of a transaction, have the system allocate
* things in the top memory context instead of per-transaction
* contexts.
*/
MemoryContextSwitchTo(TopMemoryContext);
/*
* Release all transaction-local memory.
*/
Assert(TopTransactionContext != NULL);
MemoryContextDelete(TopTransactionContext);
TopTransactionContext = NULL;
CurTransactionContext = NULL;
CurrentTransactionState->curTransactionContext = NULL;
}
/* ----------------------------------------------------------------
* CommitSubTransaction stuff
* ----------------------------------------------------------------
*/
/*
* AtSubCommit_Memory
*
* We do not throw away the child's CurTransactionContext, since the data
* it contains will be needed at upper commit.
*/
static void
AtSubCommit_Memory(void)
{
TransactionState s = CurrentTransactionState;
Assert(s->parent != NULL);
/* Return to parent transaction level's memory context. */
CurTransactionContext = s->parent->curTransactionContext;
MemoryContextSwitchTo(CurTransactionContext);
}
/*
* AtSubCommit_childXids
*
* Pass my own XID and my child XIDs up to my parent as committed children.
*/
static void
AtSubCommit_childXids(void)
{
TransactionState s = CurrentTransactionState;
MemoryContext old_cxt;
Assert(s->parent != NULL);
old_cxt = MemoryContextSwitchTo(s->parent->curTransactionContext);
s->parent->childXids = list_concat(s->parent->childXids, s->childXids);
s->childXids = NIL; /* ensure list not doubly referenced */
s->parent->childXids = lappend_int(s->parent->childXids,
s->transactionIdData);
MemoryContextSwitchTo(old_cxt);
}
/*
* RecordSubTransactionCommit
*/
static void
RecordSubTransactionCommit(void)
{
/*
* We do not log the subcommit in XLOG; it doesn't matter until
* the top-level transaction commits.
*
* We must mark the subtransaction subcommitted in clog if its XID
* appears either in permanent rels or in local temporary rels. We
* test this by seeing if we made transaction-controlled entries
* *OR* local-rel tuple updates. (The test here actually covers the
* entire transaction tree so far, so it may mark subtransactions that
* don't really need it, but it's probably not worth being tenser.
* Note that if a prior subtransaction dirtied these variables, then
* RecordTransactionCommit will have to do the full pushup anyway...)
*/
if (MyLastRecPtr.xrecoff != 0 || MyXactMadeTempRelUpdate)
{
TransactionId xid = GetCurrentTransactionId();
/* XXX does this really need to be a critical section? */
START_CRIT_SECTION();
/* Record subtransaction subcommit */
TransactionIdSubCommit(xid);
END_CRIT_SECTION();
}
}
/* ----------------------------------------------------------------
* AbortTransaction stuff
* ----------------------------------------------------------------
*/
/*
* RecordTransactionAbort
*/
static void
RecordTransactionAbort(void)
{
int nrels;
RelFileNode *rptr;
int nchildren;
TransactionId *children;
/* Get data needed for abort record */
nrels = smgrGetPendingDeletes(false, &rptr);
nchildren = xactGetCommittedChildren(&children, false);
/*
* If we made neither any transaction-controlled XLOG entries nor any
* temp-rel updates, and are not going to delete any files, we can omit
* recording the transaction abort at all. No one will ever care that
* it aborted. (These tests cover our whole transaction tree.)
*/
if (MyLastRecPtr.xrecoff != 0 || MyXactMadeTempRelUpdate || nrels > 0)
{
TransactionId xid = GetCurrentTransactionId();
/*
* Catch the scenario where we aborted partway through
* RecordTransactionCommit ...
*/
if (TransactionIdDidCommit(xid))
elog(PANIC, "cannot abort transaction %u, it was already committed", xid);
START_CRIT_SECTION();
/*
* We only need to log the abort in XLOG if the transaction made
* any transaction-controlled XLOG entries or will delete files.
* (If it made no transaction-controlled XLOG entries, its XID
* appears nowhere in permanent storage, so no one else will ever care
* if it committed.)
*
* We do not flush XLOG to disk unless deleting files, since the
* default assumption after a crash would be that we aborted, anyway.
*/
if (MyLastRecPtr.xrecoff != 0 || nrels > 0)
{
XLogRecData rdata[3];
int lastrdata = 0;
xl_xact_abort xlrec;
XLogRecPtr recptr;
xlrec.xtime = time(NULL);
xlrec.nrels = nrels;
xlrec.nsubxacts = nchildren;
rdata[0].buffer = InvalidBuffer;
rdata[0].data = (char *) (&xlrec);
rdata[0].len = MinSizeOfXactAbort;
/* dump rels to delete */
if (nrels > 0)
{
rdata[0].next = &(rdata[1]);
rdata[1].buffer = InvalidBuffer;
rdata[1].data = (char *) rptr;
rdata[1].len = nrels * sizeof(RelFileNode);
lastrdata = 1;
}
/* dump committed child Xids */
if (nchildren > 0)
{
rdata[lastrdata].next = &(rdata[2]);
rdata[2].buffer = InvalidBuffer;
rdata[2].data = (char *) children;
rdata[2].len = nchildren * sizeof(TransactionId);
lastrdata = 2;
}
rdata[lastrdata].next = NULL;
recptr = XLogInsert(RM_XACT_ID, XLOG_XACT_ABORT, rdata);
/* Must flush if we are deleting files... */
if (nrels > 0)
XLogFlush(recptr);
}
/*
* Mark the transaction aborted in clog. This is not absolutely
* necessary but we may as well do it while we are here.
*
* The ordering here isn't critical but it seems best to mark the
* parent last. That reduces the chance that concurrent
* TransactionIdDidAbort calls will decide they need to do redundant
* work.
*/
TransactionIdAbortTree(nchildren, children);
TransactionIdAbort(xid);
END_CRIT_SECTION();
}
/* Break the chain of back-links in the XLOG records I output */
MyLastRecPtr.xrecoff = 0;
MyXactMadeXLogEntry = false;
MyXactMadeTempRelUpdate = false;
/* Show myself as out of the transaction in PGPROC array */
MyProc->logRec.xrecoff = 0;
/* And clean up local data */
if (rptr)
pfree(rptr);
if (children)
pfree(children);
}
/*
* AtAbort_Cache
*/
static void
AtAbort_Cache(void)
{
AtEOXact_RelationCache(false);
AtEOXact_Inval(false);
}
/*
* AtAbort_Locks
*/
static void
AtAbort_Locks(void)
{
/*
* XXX What if ProcReleaseLocks() fails? (race condition?)
*
* Then you're up a creek without a paddle! -mer
*/
ProcReleaseLocks(ReleaseAll, 0, NULL);
}
/*
* AtAbort_Memory
*/
static void
AtAbort_Memory(void)
{
/*
* Make sure we are in a valid context (not a child of
* TopTransactionContext...). Note that it is possible for this code
* to be called when we aren't in a transaction at all; go directly to
* TopMemoryContext in that case.
*/
if (TopTransactionContext != NULL)
{
MemoryContextSwitchTo(TopTransactionContext);
/*
* We do not want to destroy the transaction's global state yet,
* so we can't free any memory here.
*/
}
else
MemoryContextSwitchTo(TopMemoryContext);
}
/*
* AtSubAbort_Locks
*/
static void
AtSubAbort_Locks(void)
{
int nxids;
TransactionId *xids;
nxids = xactGetCommittedChildren(&xids, true);
ProcReleaseLocks(ReleaseGivenXids, nxids, xids);
pfree(xids);
}
/*
* AtSubAbort_Memory
*/
static void
AtSubAbort_Memory(void)
{
Assert(TopTransactionContext != NULL);
MemoryContextSwitchTo(TopTransactionContext);
}
/*
* RecordSubTransactionAbort
*/
static void
RecordSubTransactionAbort(void)
{
int nrels;
RelFileNode *rptr;
int nchildren;
TransactionId *children;
/* Get data needed for abort record */
nrels = smgrGetPendingDeletes(false, &rptr);
nchildren = xactGetCommittedChildren(&children, false);
/*
* If we made neither any transaction-controlled XLOG entries nor any
* temp-rel updates, and are not going to delete any files, we can omit
* recording the transaction abort at all. No one will ever care that
* it aborted. (These tests cover our whole transaction tree, and
* therefore may mark subxacts that don't really need it, but it's
* probably not worth being tenser.)
*
* In this case we needn't worry about marking subcommitted children as
* aborted, because they didn't mark themselves as subcommitted in the
* first place; see the optimization in RecordSubTransactionCommit.
*/
if (MyLastRecPtr.xrecoff != 0 || MyXactMadeTempRelUpdate || nrels > 0)
{
TransactionId xid = GetCurrentTransactionId();
START_CRIT_SECTION();
/*
* We only need to log the abort in XLOG if the transaction made
* any transaction-controlled XLOG entries or will delete files.
*/
if (MyLastRecPtr.xrecoff != 0 || nrels > 0)
{
XLogRecData rdata[3];
int lastrdata = 0;
xl_xact_abort xlrec;
XLogRecPtr recptr;
xlrec.xtime = time(NULL);
xlrec.nrels = nrels;
xlrec.nsubxacts = nchildren;
rdata[0].buffer = InvalidBuffer;
rdata[0].data = (char *) (&xlrec);
rdata[0].len = MinSizeOfXactAbort;
/* dump rels to delete */
if (nrels > 0)
{
rdata[0].next = &(rdata[1]);
rdata[1].buffer = InvalidBuffer;
rdata[1].data = (char *) rptr;
rdata[1].len = nrels * sizeof(RelFileNode);
lastrdata = 1;
}
/* dump committed child Xids */
if (nchildren > 0)
{
rdata[lastrdata].next = &(rdata[2]);
rdata[2].buffer = InvalidBuffer;
rdata[2].data = (char *) children;
rdata[2].len = nchildren * sizeof(TransactionId);
lastrdata = 2;
}
rdata[lastrdata].next = NULL;
recptr = XLogInsert(RM_XACT_ID, XLOG_XACT_ABORT, rdata);
/* Must flush if we are deleting files... */
if (nrels > 0)
XLogFlush(recptr);
}
/*
* Mark the transaction aborted in clog. This is not absolutely
* necessary but we may as well do it while we are here.
*/
TransactionIdAbortTree(nchildren, children);
TransactionIdAbort(xid);
END_CRIT_SECTION();
}
/* And clean up local data */
if (rptr)
pfree(rptr);
if (children)
pfree(children);
}
/* ----------------------------------------------------------------
* CleanupTransaction stuff
* ----------------------------------------------------------------
*/
/*
* AtCleanup_Memory
*/
static void
AtCleanup_Memory(void)
{
/*
* Now that we're "out" of a transaction, have the system allocate
* things in the top memory context instead of per-transaction
* contexts.
*/
MemoryContextSwitchTo(TopMemoryContext);
Assert(CurrentTransactionState->parent == NULL);
/*
* Release all transaction-local memory.
*/
if (TopTransactionContext != NULL)
MemoryContextDelete(TopTransactionContext);
TopTransactionContext = NULL;
CurTransactionContext = NULL;
CurrentTransactionState->curTransactionContext = NULL;
}
/* ----------------------------------------------------------------
* CleanupSubTransaction stuff
* ----------------------------------------------------------------
*/
/*
* AtSubCleanup_Memory
*/
static void
AtSubCleanup_Memory(void)
{
TransactionState s = CurrentTransactionState;
Assert(s->parent != NULL);
/* Make sure we're not in an about-to-be-deleted context */
MemoryContextSwitchTo(s->parent->curTransactionContext);
CurTransactionContext = s->parent->curTransactionContext;
/*
* Delete the subxact local memory contexts. Its CurTransactionContext
* can go too (note this also kills CurTransactionContexts from any
* children of the subxact).
*/
MemoryContextDelete(s->curTransactionContext);
}
/* ----------------------------------------------------------------
* interface routines
* ----------------------------------------------------------------
*/
/*
* StartTransaction
*/
static void
StartTransaction(void)
{
TransactionState s = CurrentTransactionState;
/*
* check the current transaction state
*/
if (s->state != TRANS_DEFAULT)
elog(WARNING, "StartTransaction and not in default state");
/*
* set the current transaction state information appropriately during
* start processing
*/
s->state = TRANS_START;
/*
* Make sure we've freed any old snapshot, and reset xact state variables
*/
FreeXactSnapshot();
XactIsoLevel = DefaultXactIsoLevel;
XactReadOnly = DefaultXactReadOnly;
/*
* generate a new transaction id
*/
s->transactionIdData = GetNewTransactionId(false);
XactLockTableInsert(s->transactionIdData);
/*
* set now()
*/
xactStartTime = GetCurrentAbsoluteTimeUsec(&(xactStartTimeUsec));
/*
* initialize current transaction state fields
*/
s->commandId = FirstCommandId;
s->nestingLevel = 1;
s->childXids = NIL;
/*
* You might expect to see "s->currentUser = GetUserId();" here, but
* you won't because it doesn't work during startup; the userid isn't
* set yet during a backend's first transaction start. We only use
* the currentUser field in sub-transaction state structs.
*/
/*
* initialize the various transaction subsystems
*/
AtStart_Memory();
AtStart_Inval();
AtStart_Cache();
AtStart_Locks();
/*
* Tell the trigger manager we're starting a transaction
*/
DeferredTriggerBeginXact();
/*
* done with start processing, set current transaction state to "in
* progress"
*/
s->state = TRANS_INPROGRESS;
ShowTransactionState("StartTransaction");
}
/*
* CommitTransaction
*/
static void
CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
ShowTransactionState("CommitTransaction");
/*
* check the current transaction state
*/
if (s->state != TRANS_INPROGRESS)
elog(WARNING, "CommitTransaction and not in in-progress state");
Assert(s->parent == NULL);
/*
* Tell the trigger manager that this transaction is about to be
* committed. He'll invoke all trigger deferred until XACT before we
* really start on committing the transaction.
*/
DeferredTriggerEndXact();
/*
* Similarly, let ON COMMIT management do its thing before we start to
* commit.
*/
PreCommit_on_commit_actions();
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
/*
* set the current transaction state information appropriately during
* the abort processing
*/
s->state = TRANS_COMMIT;
/*
* Do pre-commit processing (most of this stuff requires database
* access, and in fact could still cause an error...)
*/
AtCommit_Portals();
/* handle commit for large objects [ PA, 7/17/98 ] */
/* XXX probably this does not belong here */
lo_commit(true);
/* NOTIFY commit must come before lower-level cleanup */
AtCommit_Notify();
/* Update the flat password file if we changed pg_shadow or pg_group */
AtEOXact_UpdatePasswordFile(true);
/*
* Here is where we really truly commit.
*/
RecordTransactionCommit();
/*
* Let others know about no transaction in progress by me. Note that
* this must be done _before_ releasing locks we hold and _after_
* RecordTransactionCommit.
*
* LWLockAcquire(SInvalLock) is required: UPDATE with xid 0 is blocked by
* xid 1' UPDATE, xid 1 is doing commit while xid 2 gets snapshot - if
* xid 2' GetSnapshotData sees xid 1 as running then it must see xid 0
* as running as well or it will see two tuple versions - one deleted
* by xid 1 and one inserted by xid 0. See notes in GetSnapshotData.
*/
if (MyProc != NULL)
{
/* Lock SInvalLock because that's what GetSnapshotData uses. */
LWLockAcquire(SInvalLock, LW_EXCLUSIVE);
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
LWLockRelease(SInvalLock);
}
/*
* This is all post-commit cleanup. Note that if an error is raised
* here, it's too late to abort the transaction. This should be just
* noncritical resource releasing.
*
* The ordering of operations is not entirely random. The idea is:
* release resources visible to other backends (eg, files, buffer
* pins); then release locks; then release backend-local resources. We
* want to release locks at the point where any backend waiting for us
* will see our transaction as being fully cleaned up.
*/
smgrDoPendingDeletes(true);
AtCommit_Cache();
AtEOXact_Buffers(true);
/* smgrcommit already done */
AtCommit_Locks();
CallEOXactCallbacks(true);
AtEOXact_GUC(true, false);
AtEOXact_SPI(true);
AtEOXact_gist();
AtEOXact_hash();
AtEOXact_nbtree();
AtEOXact_rtree();
AtEOXact_on_commit_actions(true, s->transactionIdData);
AtEOXact_Namespace(true);
AtEOXact_CatCache(true);
AtEOXact_Files();
pgstat_count_xact_commit();
AtCommit_Memory();
s->nestingLevel = 0;
s->childXids = NIL;
/*
* done with commit processing, set current transaction state back to
* default
*/
s->state = TRANS_DEFAULT;
RESUME_INTERRUPTS();
}
/*
* AbortTransaction
*/
static void
AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
/*
* Release any LW locks we might be holding as quickly as possible.
* (Regular locks, however, must be held till we finish aborting.)
* Releasing LW locks is critical since we might try to grab them
* again while cleaning up!
*/
LWLockReleaseAll();
/* Clean up buffer I/O and buffer context locks, too */
AbortBufferIO();
UnlockBuffers();
/*
* Also clean up any open wait for lock, since the lock manager will
* choke if we try to wait for another lock before doing this.
*/
LockWaitCancel();
/*
* check the current transaction state
*/
if (s->state != TRANS_INPROGRESS)
elog(WARNING, "AbortTransaction and not in in-progress state");
Assert(s->parent == NULL);
/*
* set the current transaction state information appropriately during
* the abort processing
*/
s->state = TRANS_ABORT;
/* Make sure we are in a valid memory context */
AtAbort_Memory();
/*
* Reset user id which might have been changed transiently. We cannot
* use s->currentUser, but must get the session userid from miscinit.c.
*
* (Note: it is not necessary to restore session authorization here
* because that can only be changed via GUC, and GUC will take care of
* rolling it back if need be. However, an error within a SECURITY
* DEFINER function could send control here with the wrong current
* userid.)
*/
SetUserId(GetSessionUserId());
/*
* do abort processing
*/
DeferredTriggerAbortXact();
AtAbort_Portals();
lo_commit(false); /* 'false' means it's abort */
AtAbort_Notify();
AtEOXact_UpdatePasswordFile(false);
/* Advertise the fact that we aborted in pg_clog. */
RecordTransactionAbort();
/*
* Let others know about no transaction in progress by me. Note that
* this must be done _before_ releasing locks we hold and _after_
* RecordTransactionAbort.
*/
if (MyProc != NULL)
{
/* Lock SInvalLock because that's what GetSnapshotData uses. */
LWLockAcquire(SInvalLock, LW_EXCLUSIVE);
MyProc->xid = InvalidTransactionId;
MyProc->xmin = InvalidTransactionId;
LWLockRelease(SInvalLock);
}
/*
* Post-abort cleanup. See notes in CommitTransaction() concerning
* ordering.
*/
smgrDoPendingDeletes(false);
AtAbort_Cache();
AtEOXact_Buffers(false);
smgrabort();
AtAbort_Locks();
CallEOXactCallbacks(false);
AtEOXact_GUC(false, false);
AtEOXact_SPI(false);
AtEOXact_gist();
AtEOXact_hash();
AtEOXact_nbtree();
AtEOXact_rtree();
AtEOXact_on_commit_actions(false, s->transactionIdData);
AtEOXact_Namespace(false);
AtEOXact_CatCache(false);
AtEOXact_Files();
SetReindexProcessing(InvalidOid, InvalidOid);
pgstat_count_xact_rollback();
/*
* State remains TRANS_ABORT until CleanupTransaction().
*/
RESUME_INTERRUPTS();
}
/*
* CleanupTransaction
*/
static void
CleanupTransaction(void)
{
TransactionState s = CurrentTransactionState;
/*
* State should still be TRANS_ABORT from AbortTransaction().
*/
if (s->state != TRANS_ABORT)
elog(FATAL, "CleanupTransaction and not in abort state");
/*
* do abort cleanup processing
*/
AtCleanup_Portals(); /* now safe to release portal memory */
AtCleanup_Memory(); /* and transaction memory */
s->nestingLevel = 0;
s->childXids = NIL;
/*
* done with abort processing, set current transaction state back to
* default
*/
s->state = TRANS_DEFAULT;
}
/*
* StartTransactionCommand
*/
void
StartTransactionCommand(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState)
{
/*
* if we aren't in a transaction block, we just do our usual
* start transaction.
*/
case TBLOCK_DEFAULT:
StartTransaction();
s->blockState = TBLOCK_STARTED;
break;
/*
* This is the case when we are somewhere in a transaction block
* and about to start a new command. For now we do nothing
* but someday we may do command-local resource initialization.
*/
case TBLOCK_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
break;
/*
* Here we are in the middle of a transaction block but one of
* the commands caused an abort so we do nothing but remain in
* the abort state. Eventually we will get to the "END
* TRANSACTION" which will set things straight.
*/
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
break;
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
case TBLOCK_END:
case TBLOCK_SUBEND:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
case TBLOCK_ENDABORT:
elog(FATAL, "StartTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
}
/*
* We must switch to CurTransactionContext before returning. This is
* already done if we called StartTransaction, otherwise not.
*/
Assert(CurTransactionContext != NULL);
MemoryContextSwitchTo(CurTransactionContext);
}
/*
* CommitTransactionCommand
*/
void
CommitTransactionCommand(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState)
{
/*
* This shouldn't happen, because it means the previous
* StartTransactionCommand didn't set the STARTED state
* appropiately.
*/
case TBLOCK_DEFAULT:
elog(FATAL, "CommitTransactionCommand: unexpected TBLOCK_DEFAULT");
break;
/*
* If we aren't in a transaction block, just do our usual
* transaction commit.
*/
case TBLOCK_STARTED:
CommitTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
/*
* This is the case right after we get a "BEGIN TRANSACTION"
* command, but the user hasn't done anything else yet, so we
* change to the "transaction block in progress" state and
* return.
*/
case TBLOCK_BEGIN:
s->blockState = TBLOCK_INPROGRESS;
break;
/*
* This is the case when we have finished executing a command
* someplace within a transaction block. We increment the
* command counter and return.
*/
case TBLOCK_INPROGRESS:
CommandCounterIncrement();
break;
/*
* This is the case when we just got the "END TRANSACTION"
* statement, so we commit the transaction and go back to the
* default state.
*/
case TBLOCK_END:
CommitTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
/*
* Here we are in the middle of a transaction block but one of
* the commands caused an abort so we do nothing but remain in
* the abort state. Eventually we will get to the "END
* TRANSACTION" which will set things straight.
*/
case TBLOCK_ABORT:
break;
/*
* Here we were in an aborted transaction block which just
* processed the "END TRANSACTION" command from the user, so
* clean up and return to the default state.
*/
case TBLOCK_ENDABORT:
CleanupTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
/*
* We were just issued a BEGIN inside a transaction block.
* Start a subtransaction. (BeginTransactionBlock already
* did PushTransaction, so as to have someplace to put the
* SUBBEGIN state.)
*/
case TBLOCK_SUBBEGIN:
StartSubTransaction();
s->blockState = TBLOCK_SUBINPROGRESS;
break;
/*
* We were issued a BEGIN inside an aborted transaction block.
* Start a subtransaction, and put it in aborted state.
*/
case TBLOCK_SUBBEGINABORT:
StartAbortedSubTransaction();
s->blockState = TBLOCK_SUBABORT;
break;
/*
* Inside a subtransaction, increment the command counter.
*/
case TBLOCK_SUBINPROGRESS:
CommandCounterIncrement();
break;
/*
* We were issued a COMMIT command, so we end the current
* subtransaction and return to the parent transaction.
*/
case TBLOCK_SUBEND:
CommitSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
break;
/*
* If we are in an aborted subtransaction, do nothing.
*/
case TBLOCK_SUBABORT:
break;
/*
* We are ending an aborted subtransaction via ROLLBACK,
* so the parent can be allowed to live.
*/
case TBLOCK_SUBENDABORT_OK:
CleanupSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
break;
/*
* We are ending an aborted subtransaction via COMMIT.
* End the subtransaction, and abort the parent too.
*/
case TBLOCK_SUBENDABORT_ERROR:
CleanupSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
Assert(s->blockState != TBLOCK_SUBENDABORT_ERROR);
AbortCurrentTransaction();
break;
}
}
/*
* AbortCurrentTransaction
*/
void
AbortCurrentTransaction(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState)
{
/*
* we aren't in a transaction, so we do nothing.
*/
case TBLOCK_DEFAULT:
break;
/*
* if we aren't in a transaction block, we just do the basic
* abort & cleanup transaction.
*/
case TBLOCK_STARTED:
AbortTransaction();
CleanupTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
/*
* If we are in TBLOCK_BEGIN it means something screwed up
* right after reading "BEGIN TRANSACTION" so we enter the
* abort state. Eventually an "END TRANSACTION" will fix
* things.
*/
case TBLOCK_BEGIN:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ENDABORT */
break;
/*
* This is the case when we are somewhere in a transaction block
* and we've gotten a failure, so we abort the transaction and
* set up the persistent ABORT state. We will stay in ABORT
* until we get an "END TRANSACTION".
*/
case TBLOCK_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ENDABORT */
break;
/*
* Here, the system was fouled up just after the user wanted
* to end the transaction block so we abort the transaction
* and return to the default state.
*/
case TBLOCK_END:
AbortTransaction();
CleanupTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
/*
* Here, we are already in an aborted transaction state and
* are waiting for an "END TRANSACTION" to come along and lo
* and behold, we abort again! So we just remain in the abort
* state.
*/
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
break;
/*
* Here we were in an aborted transaction block which just
* processed the "END TRANSACTION" command but somehow aborted
* again.. since we must have done the abort processing, we
* clean up and return to the default state.
*/
case TBLOCK_ENDABORT:
CleanupTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
/*
* If we are just starting a subtransaction, put it
* in aborted state.
*/
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
StartAbortedSubTransaction();
s->blockState = TBLOCK_SUBABORT;
break;
case TBLOCK_SUBINPROGRESS:
AbortSubTransaction();
s->blockState = TBLOCK_SUBABORT;
break;
/*
* If we are aborting an ending transaction,
* we have to abort the parent transaction too.
*/
case TBLOCK_SUBEND:
AbortSubTransaction();
CleanupSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
Assert(s->blockState != TBLOCK_SUBEND &&
s->blockState != TBLOCK_SUBENDABORT_OK &&
s->blockState != TBLOCK_SUBENDABORT_ERROR);
AbortCurrentTransaction();
break;
/*
* Same as above, except the Abort() was already done.
*/
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
CleanupSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
Assert(s->blockState != TBLOCK_SUBEND &&
s->blockState != TBLOCK_SUBENDABORT_OK &&
s->blockState != TBLOCK_SUBENDABORT_ERROR);
AbortCurrentTransaction();
break;
}
}
/*
* PreventTransactionChain
*
* This routine is to be called by statements that must not run inside
* a transaction block, typically because they have non-rollback-able
* side effects or do internal commits.
*
* If we have already started a transaction block, issue an error; also issue
* an error if we appear to be running inside a user-defined function (which
* could issue more commands and possibly cause a failure after the statement
* completes). Subtransactions are verboten too.
*
* stmtNode: pointer to parameter block for statement; this is used in
* a very klugy way to determine whether we are inside a function.
* stmtType: statement type name for error messages.
*/
void
PreventTransactionChain(void *stmtNode, const char *stmtType)
{
/*
* xact block already started?
*/
if (IsTransactionBlock())
ereport(ERROR,
(errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
/* translator: %s represents an SQL statement name */
errmsg("%s cannot run inside a transaction block",
stmtType)));
/*
* subtransaction?
*/
if (IsSubTransaction())
ereport(ERROR,
(errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
/* translator: %s represents an SQL statement name */
errmsg("%s cannot run inside a subtransaction",
stmtType)));
/*
* Are we inside a function call? If the statement's parameter block
* was allocated in QueryContext, assume it is an interactive command.
* Otherwise assume it is coming from a function.
*/
if (!MemoryContextContains(QueryContext, stmtNode))
ereport(ERROR,
(errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
/* translator: %s represents an SQL statement name */
errmsg("%s cannot be executed from a function", stmtType)));
/* If we got past IsTransactionBlock test, should be in default state */
if (CurrentTransactionState->blockState != TBLOCK_DEFAULT &&
CurrentTransactionState->blockState != TBLOCK_STARTED)
elog(FATAL, "cannot prevent transaction chain");
/* all okay */
}
/*
* RequireTransactionChain
*
* This routine is to be called by statements that must run inside
* a transaction block, because they have no effects that persist past
* transaction end (and so calling them outside a transaction block
* is presumably an error). DECLARE CURSOR is an example.
*
* If we appear to be running inside a user-defined function, we do not
* issue an error, since the function could issue more commands that make
* use of the current statement's results. Likewise subtransactions.
* Thus this is an inverse for PreventTransactionChain.
*
* stmtNode: pointer to parameter block for statement; this is used in
* a very klugy way to determine whether we are inside a function.
* stmtType: statement type name for error messages.
*/
void
RequireTransactionChain(void *stmtNode, const char *stmtType)
{
/*
* xact block already started?
*/
if (IsTransactionBlock())
return;
/*
* subtransaction?
*/
if (IsSubTransaction())
return;
/*
* Are we inside a function call? If the statement's parameter block
* was allocated in QueryContext, assume it is an interactive command.
* Otherwise assume it is coming from a function.
*/
if (!MemoryContextContains(QueryContext, stmtNode))
return;
ereport(ERROR,
(errcode(ERRCODE_NO_ACTIVE_SQL_TRANSACTION),
/* translator: %s represents an SQL statement name */
errmsg("%s may only be used in transaction blocks",
stmtType)));
}
/*
* IsInTransactionChain
*
* This routine is for statements that need to behave differently inside
* a transaction block than when running as single commands. ANALYZE is
* currently the only example.
*
* stmtNode: pointer to parameter block for statement; this is used in
* a very klugy way to determine whether we are inside a function.
*/
bool
IsInTransactionChain(void *stmtNode)
{
/*
* Return true on same conditions that would make PreventTransactionChain
* error out
*/
if (IsTransactionBlock())
return true;
if (IsSubTransaction())
return true;
if (!MemoryContextContains(QueryContext, stmtNode))
return true;
if (CurrentTransactionState->blockState != TBLOCK_DEFAULT &&
CurrentTransactionState->blockState != TBLOCK_STARTED)
return true;
return false;
}
/*
* Register or deregister callback functions for end-of-xact cleanup
*
* These functions are intended for use by dynamically loaded modules.
* For built-in modules we generally just hardwire the appropriate calls
* (mainly because it's easier to control the order that way, where needed).
*
* Note that the callback occurs post-commit or post-abort, so the callback
* functions can only do noncritical cleanup.
*/
void
RegisterEOXactCallback(EOXactCallback callback, void *arg)
{
EOXactCallbackItem *item;
item = (EOXactCallbackItem *)
MemoryContextAlloc(TopMemoryContext, sizeof(EOXactCallbackItem));
item->callback = callback;
item->arg = arg;
item->next = EOXact_callbacks;
EOXact_callbacks = item;
}
void
UnregisterEOXactCallback(EOXactCallback callback, void *arg)
{
EOXactCallbackItem *item;
EOXactCallbackItem *prev;
prev = NULL;
for (item = EOXact_callbacks; item; prev = item, item = item->next)
{
if (item->callback == callback && item->arg == arg)
{
if (prev)
prev->next = item->next;
else
EOXact_callbacks = item->next;
pfree(item);
break;
}
}
}
static void
CallEOXactCallbacks(bool isCommit)
{
EOXactCallbackItem *item;
for (item = EOXact_callbacks; item; item = item->next)
{
(*item->callback) (isCommit, item->arg);
}
}
/* ----------------------------------------------------------------
* transaction block support
* ----------------------------------------------------------------
*/
/*
* BeginTransactionBlock
* This executes a BEGIN command.
*/
void
BeginTransactionBlock(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState) {
/*
* We are not inside a transaction block, so allow one
* to begin.
*/
case TBLOCK_STARTED:
s->blockState = TBLOCK_BEGIN;
break;
/*
* Already a transaction block in progress.
* Start a subtransaction.
*/
case TBLOCK_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
PushTransaction();
s = CurrentTransactionState; /* changed by push */
s->blockState = TBLOCK_SUBBEGIN;
break;
/*
* An aborted transaction block should be allowed to start
* a subtransaction, but it must put it in aborted state.
*/
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
PushTransaction();
s = CurrentTransactionState; /* changed by push */
s->blockState = TBLOCK_SUBBEGINABORT;
break;
/* These cases are invalid. Reject them altogether. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
case TBLOCK_ENDABORT:
case TBLOCK_END:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
case TBLOCK_SUBEND:
elog(FATAL, "BeginTransactionBlock: unexpected state %s",
BlockStateAsString(s->blockState));
break;
}
}
/*
* EndTransactionBlock
* This executes a COMMIT command.
*/
void
EndTransactionBlock(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState) {
/*
* here we are in a transaction block which should commit when we
* get to the upcoming CommitTransactionCommand() so we set the
* state to "END". CommitTransactionCommand() will recognize this
* and commit the transaction and return us to the default state
*/
case TBLOCK_INPROGRESS:
s->blockState = TBLOCK_END;
break;
/*
* here we are in a subtransaction block. Signal
* CommitTransactionCommand() to end it and return to the
* parent transaction.
*/
case TBLOCK_SUBINPROGRESS:
s->blockState = TBLOCK_SUBEND;
break;
/*
* here, we are in a transaction block which aborted. Since the
* AbortTransaction() was already done, we need only
* change to the special "END ABORT" state. The upcoming
* CommitTransactionCommand() will recognise this and then put us
* back in the default state.
*/
case TBLOCK_ABORT:
s->blockState = TBLOCK_ENDABORT;
break;
/*
* here we are in an aborted subtransaction. Signal
* CommitTransactionCommand() to clean up and return to the
* parent transaction. Since the user said COMMIT, we must
* fail the parent transaction.
*/
case TBLOCK_SUBABORT:
s->blockState = TBLOCK_SUBENDABORT_ERROR;
break;
case TBLOCK_STARTED:
/*
* here, the user issued COMMIT when not inside a
* transaction. Issue a WARNING and go to abort state. The
* upcoming call to CommitTransactionCommand() will then put us
* back into the default state.
*/
ereport(WARNING,
(errcode(ERRCODE_NO_ACTIVE_SQL_TRANSACTION),
errmsg("there is no transaction in progress")));
AbortTransaction();
s->blockState = TBLOCK_ENDABORT;
break;
/* these cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
case TBLOCK_ENDABORT:
case TBLOCK_END:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
case TBLOCK_SUBEND:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
elog(FATAL, "EndTransactionBlock: unexpected state %s",
BlockStateAsString(s->blockState));
break;
}
}
/*
* UserAbortTransactionBlock
* This executes a ROLLBACK command.
*/
void
UserAbortTransactionBlock(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState) {
/*
* here we are inside a failed transaction block and we got an abort
* command from the user. Abort processing is already done, we just
* need to move to the ENDABORT state so we will end up in the default
* state after the upcoming CommitTransactionCommand().
*/
case TBLOCK_ABORT:
s->blockState = TBLOCK_ENDABORT;
break;
/*
* Ditto, for a subtransaction. Here it is okay to allow the
* parent transaction to continue.
*/
case TBLOCK_SUBABORT:
s->blockState = TBLOCK_SUBENDABORT_OK;
break;
/*
* here we are inside a transaction block and we got an abort
* command from the user, so we move to the ENDABORT state and
* do abort processing so we will end up in the default state
* after the upcoming CommitTransactionCommand().
*/
case TBLOCK_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ENDABORT;
break;
/* Ditto, for a subtransaction. */
case TBLOCK_SUBINPROGRESS:
AbortSubTransaction();
s->blockState = TBLOCK_SUBENDABORT_OK;
break;
/*
* here, the user issued ABORT when not inside a
* transaction. Issue a WARNING and go to abort state. The
* upcoming call to CommitTransactionCommand() will then put us
* back into the default state.
*/
case TBLOCK_STARTED:
ereport(WARNING,
(errcode(ERRCODE_NO_ACTIVE_SQL_TRANSACTION),
errmsg("there is no transaction in progress")));
AbortTransaction();
s->blockState = TBLOCK_ENDABORT;
break;
/* these cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
case TBLOCK_END:
case TBLOCK_ENDABORT:
case TBLOCK_SUBEND:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
elog(FATAL, "UserAbortTransactionBlock: unexpected state %s",
BlockStateAsString(s->blockState));
break;
}
}
/*
* AbortOutOfAnyTransaction
*
* This routine is provided for error recovery purposes. It aborts any
* active transaction or transaction block, leaving the system in a known
* idle state.
*/
void
AbortOutOfAnyTransaction(void)
{
TransactionState s = CurrentTransactionState;
/*
* Get out of any transaction or nested transaction
*/
do {
switch (s->blockState)
{
case TBLOCK_DEFAULT:
/* Not in a transaction, do nothing */
break;
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
/* In a transaction, so clean up */
AbortTransaction();
CleanupTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
case TBLOCK_ABORT:
case TBLOCK_ENDABORT:
/* AbortTransaction already done, still need Cleanup */
CleanupTransaction();
s->blockState = TBLOCK_DEFAULT;
break;
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
/*
* We didn't get as far as starting the subxact, so there's
* nothing to abort. Just pop back to parent.
*/
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
break;
case TBLOCK_SUBINPROGRESS:
case TBLOCK_SUBEND:
/* In a subtransaction, so clean it up and abort parent too */
AbortSubTransaction();
CleanupSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
break;
case TBLOCK_SUBABORT:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
/* As above, but AbortSubTransaction already done */
CleanupSubTransaction();
PopTransaction();
s = CurrentTransactionState; /* changed by pop */
break;
}
} while (s->blockState != TBLOCK_DEFAULT);
/* Should be out of all subxacts now */
Assert(s->parent == NULL);
}
/*
* IsTransactionBlock --- are we within a transaction block?
*/
bool
IsTransactionBlock(void)
{
TransactionState s = CurrentTransactionState;
if (s->blockState == TBLOCK_DEFAULT || s->blockState == TBLOCK_STARTED)
return false;
return true;
}
/*
* IsTransactionOrTransactionBlock --- are we within either a transaction
* or a transaction block? (The backend is only really "idle" when this
* returns false.)
*
* This should match up with IsTransactionBlock and IsTransactionState.
*/
bool
IsTransactionOrTransactionBlock(void)
{
TransactionState s = CurrentTransactionState;
if (s->blockState == TBLOCK_DEFAULT)
return false;
return true;
}
/*
* TransactionBlockStatusCode - return status code to send in ReadyForQuery
*/
char
TransactionBlockStatusCode(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState)
{
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
return 'I'; /* idle --- not in transaction */
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBEND:
return 'T'; /* in transaction */
case TBLOCK_ABORT:
case TBLOCK_ENDABORT:
case TBLOCK_SUBABORT:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
case TBLOCK_SUBBEGINABORT:
return 'E'; /* in failed transaction */
}
/* should never get here */
elog(FATAL, "invalid transaction block state: %s",
BlockStateAsString(s->blockState));
return 0; /* keep compiler quiet */
}
/*
* IsSubTransaction
*/
bool
IsSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
switch (s->blockState) {
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT:
case TBLOCK_ENDABORT:
return false;
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBBEGINABORT:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_SUBABORT:
case TBLOCK_SUBEND:
case TBLOCK_SUBENDABORT_OK:
case TBLOCK_SUBENDABORT_ERROR:
return true;
}
/* should never get here */
elog(FATAL, "invalid transaction block state: %s",
BlockStateAsString(s->blockState));
return false; /* keep compiler quiet */
}
/*
* StartSubTransaction
*/
static void
StartSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
if (s->state != TRANS_DEFAULT)
elog(WARNING, "StartSubTransaction and not in default state");
s->state = TRANS_START;
/*
* Generate a new Xid and record it in pg_subtrans.
*/
s->transactionIdData = GetNewTransactionId(true);
SubTransSetParent(s->transactionIdData, s->parent->transactionIdData);
/*
* Finish setup of other transaction state fields.
*/
s->currentUser = GetUserId();
/* Initialize the various transaction subsystems */
AtSubStart_Memory();
AtSubStart_Inval();
AtSubStart_RelationCache();
AtSubStart_CatCache();
AtSubStart_Buffers();
AtSubStart_smgr();
AtSubStart_Notify();
DeferredTriggerBeginSubXact();
s->state = TRANS_INPROGRESS;
ShowTransactionState("StartSubTransaction");
}
/*
* CommitSubTransaction
*/
static void
CommitSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
ShowTransactionState("CommitSubTransaction");
if (s->state != TRANS_INPROGRESS)
elog(WARNING, "CommitSubTransaction and not in in-progress state");
/* Pre-commit processing */
AtSubCommit_Portals(s->parent->transactionIdData);
DeferredTriggerEndSubXact(true);
s->state = TRANS_COMMIT;
/* Mark subtransaction as subcommitted */
CommandCounterIncrement();
RecordSubTransactionCommit();
AtSubCommit_childXids();
/* Post-commit cleanup */
AtSubCommit_smgr();
AtSubEOXact_Inval(true);
AtEOSubXact_SPI(true, s->transactionIdData);
AtSubCommit_Notify();
AtEOXact_GUC(true, true);
AtEOSubXact_gist(s->transactionIdData);
AtEOSubXact_hash(s->transactionIdData);
AtEOSubXact_rtree(s->transactionIdData);
AtEOSubXact_on_commit_actions(true, s->transactionIdData,
s->parent->transactionIdData);
AtEOSubXact_CatCache(true);
AtEOSubXact_RelationCache(true);
AtEOSubXact_Buffers(true);
AtSubCommit_Memory();
s->state = TRANS_DEFAULT;
}
/*
* AbortSubTransaction
*/
static void
AbortSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
ShowTransactionState("AbortSubTransaction");
HOLD_INTERRUPTS();
s->state = TRANS_ABORT;
/*
* Release any LW locks we might be holding as quickly as possible.
* (Regular locks, however, must be held till we finish aborting.)
* Releasing LW locks is critical since we might try to grab them
* again while cleaning up!
*
* FIXME This may be incorrect --- Are there some locks we should keep?
* Buffer locks, for example? I don't think so but I'm not sure.
*/
LWLockReleaseAll();
AbortBufferIO();
UnlockBuffers();
LockWaitCancel();
AtSubAbort_Memory();
/*
* do abort processing
*/
RecordSubTransactionAbort();
/* Post-abort cleanup */
AtSubAbort_smgr();
DeferredTriggerEndSubXact(false);
AtSubAbort_Portals();
AtSubEOXact_Inval(false);
AtSubAbort_Locks();
AtEOSubXact_SPI(false, s->transactionIdData);
AtSubAbort_Notify();
AtEOXact_GUC(false, true);
AtEOSubXact_gist(s->transactionIdData);
AtEOSubXact_hash(s->transactionIdData);
AtEOSubXact_rtree(s->transactionIdData);
AtEOSubXact_on_commit_actions(false, s->transactionIdData,
s->parent->transactionIdData);
AtEOSubXact_RelationCache(false);
AtEOSubXact_CatCache(false);
AtEOSubXact_Buffers(false);
/*
* Reset user id which might have been changed transiently. Here we
* want to restore to the userid that was current at subxact entry.
* (As in AbortTransaction, we need not worry about the session userid.)
*
* Must do this after AtEOXact_GUC to handle the case where we entered
* the subxact inside a SECURITY DEFINER function (hence current and
* session userids were different) and then session auth was changed
* inside the subxact. GUC will reset both current and session userids
* to the entry-time session userid. This is right in every other
* scenario so it seems simplest to let GUC do that and fix it here.
*/
SetUserId(s->currentUser);
CommandCounterIncrement();
RESUME_INTERRUPTS();
}
/*
* CleanupSubTransaction
*/
static void
CleanupSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
ShowTransactionState("CleanupSubTransaction");
if (s->state != TRANS_ABORT)
elog(WARNING, "CleanupSubTransaction and not in aborted state");
AtSubCleanup_Portals();
AtSubCleanup_Memory();
s->state = TRANS_DEFAULT;
}
/*
* StartAbortedSubTransaction
*
* This function is used to start a subtransaction and put it immediately
* into aborted state. The end result should be equivalent to
* StartSubTransaction immediately followed by AbortSubTransaction.
* The reason we don't implement it just that way is that many of the backend
* modules aren't designed to handle starting a subtransaction when not
* inside a valid transaction. Rather than making them all capable of
* doing that, we just omit the paired start and abort calls in this path.
*/
static void
StartAbortedSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
if (s->state != TRANS_DEFAULT)
elog(WARNING, "StartAbortedSubTransaction and not in default state");
s->state = TRANS_START;
/*
* We don't bother to generate a new Xid, so the end state is not
* *exactly* like we had done a full Start/AbortSubTransaction...
*/
s->transactionIdData = InvalidTransactionId;
/* Make sure currentUser is reasonably valid */
Assert(s->parent != NULL);
s->currentUser = s->parent->currentUser;
/*
* Initialize only what has to be there for CleanupSubTransaction to work.
*/
AtSubStart_Memory();
s->state = TRANS_ABORT;
AtSubAbort_Memory();
ShowTransactionState("StartAbortedSubTransaction");
}
/*
* PushTransaction
* Set up transaction state for a subtransaction
*/
static void
PushTransaction(void)
{
TransactionState p = CurrentTransactionState;
TransactionState s;
/*
* We keep subtransaction state nodes in TopTransactionContext.
*/
s = (TransactionState)
MemoryContextAllocZero(TopTransactionContext,
sizeof(TransactionStateData));
s->parent = p;
s->nestingLevel = p->nestingLevel + 1;
s->state = TRANS_DEFAULT;
s->blockState = TBLOCK_SUBBEGIN;
/* Command IDs count in a continuous sequence through subtransactions */
s->commandId = p->commandId;
/*
* Copy down some other data so that we will have valid state until
* StartSubTransaction runs.
*/
s->transactionIdData = p->transactionIdData;
s->curTransactionContext = p->curTransactionContext;
s->currentUser = p->currentUser;
CurrentTransactionState = s;
}
/*
* PopTransaction
* Pop back to parent transaction state
*/
static void
PopTransaction(void)
{
TransactionState s = CurrentTransactionState;
if (s->state != TRANS_DEFAULT)
elog(WARNING, "PopTransaction and not in default state");
if (s->parent == NULL)
elog(FATAL, "PopTransaction with no parent");
/* Command IDs count in a continuous sequence through subtransactions */
s->parent->commandId = s->commandId;
CurrentTransactionState = s->parent;
/* Let's just make sure CurTransactionContext is good */
CurTransactionContext = s->parent->curTransactionContext;
MemoryContextSwitchTo(CurTransactionContext);
/* Free the old child structure */
pfree(s);
}
/*
* ShowTransactionState
* Debug support
*/
static void
ShowTransactionState(const char *str)
{
/* skip work if message will definitely not be printed */
if (log_min_messages <= DEBUG2 || client_min_messages <= DEBUG2)
{
elog(DEBUG2, "%s", str);
ShowTransactionStateRec(CurrentTransactionState);
}
}
/*
* ShowTransactionStateRec
* Recursive subroutine for ShowTransactionState
*/
static void
ShowTransactionStateRec(TransactionState s)
{
if (s->parent)
ShowTransactionStateRec(s->parent);
/* use ereport to suppress computation if msg will not be printed */
ereport(DEBUG2,
(errmsg_internal("blockState: %13s; state: %7s, xid/cid: %u/%02u, nestlvl: %d, children: %s",
BlockStateAsString(s->blockState),
TransStateAsString(s->state),
(unsigned int) s->transactionIdData,
(unsigned int) s->commandId,
s->nestingLevel,
nodeToString(s->childXids))));
}
/*
* BlockStateAsString
* Debug support
*/
static const char *
BlockStateAsString(TBlockState blockState)
{
switch (blockState) {
case TBLOCK_DEFAULT:
return "DEFAULT";
case TBLOCK_STARTED:
return "STARTED";
case TBLOCK_BEGIN:
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
return "ABORT";
case TBLOCK_ENDABORT:
return "ENDABORT";
case TBLOCK_SUBBEGIN:
return "SUB BEGIN";
case TBLOCK_SUBBEGINABORT:
return "SUB BEGIN AB";
case TBLOCK_SUBINPROGRESS:
return "SUB INPROGRS";
case TBLOCK_SUBEND:
return "SUB END";
case TBLOCK_SUBABORT:
return "SUB ABORT";
case TBLOCK_SUBENDABORT_OK:
return "SUB ENDAB OK";
case TBLOCK_SUBENDABORT_ERROR:
return "SUB ENDAB ERR";
}
return "UNRECOGNIZED";
}
/*
* TransStateAsString
* Debug support
*/
static const char *
TransStateAsString(TransState state)
{
switch (state) {
case TRANS_DEFAULT:
return "DEFAULT";
case TRANS_START:
return "START";
case TRANS_COMMIT:
return "COMMIT";
case TRANS_ABORT:
return "ABORT";
case TRANS_INPROGRESS:
return "INPROGR";
}
return "UNRECOGNIZED";
}
/*
* xactGetCommittedChildren
*
* Gets the list of committed children of the current transaction. The return
* value is the number of child transactions. *children is set to point to a
* palloc'd array of TransactionIds. If there are no subxacts, *children is
* set to NULL.
*
* If metoo is true, include the current TransactionId.
*/
int
xactGetCommittedChildren(TransactionId **ptr, bool metoo)
{
TransactionState s = CurrentTransactionState;
int nchildren;
TransactionId *children;
ListCell *p;
nchildren = list_length(s->childXids);
if (metoo)
nchildren++;
if (nchildren == 0)
{
*ptr = NULL;
return 0;
}
children = (TransactionId *) palloc(nchildren * sizeof(TransactionId));
*ptr = children;
foreach(p, s->childXids)
{
TransactionId child = lfirst_int(p);
*children++ = (TransactionId)child;
}
if (metoo)
*children = s->transactionIdData;
return nchildren;
}
/*
* XLOG support routines
*/
void
xact_redo(XLogRecPtr lsn, XLogRecord *record)
{
uint8 info = record->xl_info & ~XLR_INFO_MASK;
if (info == XLOG_XACT_COMMIT)
{
xl_xact_commit *xlrec = (xl_xact_commit *) XLogRecGetData(record);
int i;
TransactionIdCommit(record->xl_xid);
/* Mark committed subtransactions as committed */
TransactionIdCommitTree(xlrec->nsubxacts,
(TransactionId *) &(xlrec->xnodes[xlrec->nrels]));
/* Make sure files supposed to be dropped are dropped */
for (i = 0; i < xlrec->nrels; i++)
{
XLogCloseRelation(xlrec->xnodes[i]);
smgrdounlink(smgropen(xlrec->xnodes[i]), false, true);
}
}
else if (info == XLOG_XACT_ABORT)
{
xl_xact_abort *xlrec = (xl_xact_abort *) XLogRecGetData(record);
int i;
TransactionIdAbort(record->xl_xid);
/* mark subtransactions as aborted */
TransactionIdAbortTree(xlrec->nsubxacts,
(TransactionId *) &(xlrec->xnodes[xlrec->nrels]));
/* Make sure files supposed to be dropped are dropped */
for (i = 0; i < xlrec->nrels; i++)
{
XLogCloseRelation(xlrec->xnodes[i]);
smgrdounlink(smgropen(xlrec->xnodes[i]), false, true);
}
}
else
elog(PANIC, "xact_redo: unknown op code %u", info);
}
void
xact_undo(XLogRecPtr lsn, XLogRecord *record)
{
uint8 info = record->xl_info & ~XLR_INFO_MASK;
if (info == XLOG_XACT_COMMIT) /* shouldn't be called by XLOG */
elog(PANIC, "xact_undo: can't undo committed xaction");
else if (info != XLOG_XACT_ABORT)
elog(PANIC, "xact_redo: unknown op code %u", info);
}
void
xact_desc(char *buf, uint8 xl_info, char *rec)
{
uint8 info = xl_info & ~XLR_INFO_MASK;
int i;
if (info == XLOG_XACT_COMMIT)
{
xl_xact_commit *xlrec = (xl_xact_commit *) rec;
struct tm *tm = localtime(&xlrec->xtime);
sprintf(buf + strlen(buf), "commit: %04u-%02u-%02u %02u:%02u:%02u",
tm->tm_year + 1900, tm->tm_mon + 1, tm->tm_mday,
tm->tm_hour, tm->tm_min, tm->tm_sec);
if (xlrec->nrels > 0)
{
sprintf(buf + strlen(buf), "; rels:");
for (i = 0; i < xlrec->nrels; i++)
{
RelFileNode rnode = xlrec->xnodes[i];
sprintf(buf + strlen(buf), " %u/%u/%u",
rnode.spcNode, rnode.dbNode, rnode.relNode);
}
}
if (xlrec->nsubxacts > 0)
{
TransactionId *xacts = (TransactionId *)
&xlrec->xnodes[xlrec->nrels];
sprintf(buf + strlen(buf), "; subxacts:");
for (i = 0; i < xlrec->nsubxacts; i++)
sprintf(buf + strlen(buf), " %u", xacts[i]);
}
}
else if (info == XLOG_XACT_ABORT)
{
xl_xact_abort *xlrec = (xl_xact_abort *) rec;
struct tm *tm = localtime(&xlrec->xtime);
sprintf(buf + strlen(buf), "abort: %04u-%02u-%02u %02u:%02u:%02u",
tm->tm_year + 1900, tm->tm_mon + 1, tm->tm_mday,
tm->tm_hour, tm->tm_min, tm->tm_sec);
if (xlrec->nrels > 0)
{
sprintf(buf + strlen(buf), "; rels:");
for (i = 0; i < xlrec->nrels; i++)
{
RelFileNode rnode = xlrec->xnodes[i];
sprintf(buf + strlen(buf), " %u/%u/%u",
rnode.spcNode, rnode.dbNode, rnode.relNode);
}
}
if (xlrec->nsubxacts > 0)
{
TransactionId *xacts = (TransactionId *)
&xlrec->xnodes[xlrec->nrels];
sprintf(buf + strlen(buf), "; subxacts:");
for (i = 0; i < xlrec->nsubxacts; i++)
sprintf(buf + strlen(buf), " %u", xacts[i]);
}
}
else
strcat(buf, "UNKNOWN");
}
void
XactPushRollback(void (*func) (void *), void *data)
{
#ifdef XLOG_II
if (_RollbackFunc != NULL)
elog(PANIC, "XactPushRollback: already installed");
#endif
_RollbackFunc = func;
_RollbackData = data;
}
void
XactPopRollback(void)
{
_RollbackFunc = NULL;
}