
problems with starting subtransactions inside already-failed transactions. Clean up some comments.
3037 lines
76 KiB
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;
|
|
}
|