*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.263 2008/05/12 00:00:46 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/access/transam/xact.c,v 1.264 2008/05/12 20:01:58 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
}
currentCommandIdUsed = false;
- /* Propagate new command ID into static snapshots, if set */
- if (SerializableSnapshot)
- SerializableSnapshot->curcid = currentCommandId;
- if (LatestSnapshot)
- LatestSnapshot->curcid = currentCommandId;
-
+ /* Propagate new command ID into static snapshots */
+ SnapshotSetCommandId(currentCommandId);
+
/*
* Make any catalog changes done by the just-completed command
* visible in the local syscache. We obviously don't need to do
s->transactionId = InvalidTransactionId; /* until assigned */
/*
- * Make sure we've freed any old snapshot, and reset xact state variables
+ * Make sure we've reset xact state variables
*/
- FreeXactSnapshot();
XactIsoLevel = DefaultXactIsoLevel;
XactReadOnly = DefaultXactReadOnly;
forceSyncCommit = false;
AtEOXact_ComboCid();
AtEOXact_HashTables(true);
AtEOXact_PgStat(true);
+ AtEOXact_Snapshot(true);
pgstat_report_xact_timestamp(0);
CurrentResourceOwner = NULL;
AtEOXact_ComboCid();
AtEOXact_HashTables(true);
/* don't call AtEOXact_PgStat here */
+ AtEOXact_Snapshot(true);
CurrentResourceOwner = NULL;
ResourceOwnerDelete(TopTransactionResourceOwner);
AtEOXact_ComboCid();
AtEOXact_HashTables(false);
AtEOXact_PgStat(false);
+ AtEOXact_Snapshot(false);
pgstat_report_xact_timestamp(0);
/*
s->parent->subTransactionId);
AtEOSubXact_HashTables(true, s->nestingLevel);
AtEOSubXact_PgStat(true, s->nestingLevel);
+ AtSubCommit_Snapshot(s->nestingLevel);
/*
* We need to restore the upper transaction's read-only state, in case the
s->parent->subTransactionId);
AtEOSubXact_HashTables(false, s->nestingLevel);
AtEOSubXact_PgStat(false, s->nestingLevel);
+ AtSubAbort_Snapshot(s->nestingLevel);
}
/*
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/catalog/index.c,v 1.298 2008/05/12 00:00:47 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/catalog/index.c,v 1.299 2008/05/12 20:01:59 alvherre Exp $
*
*
* INTERFACE ROUTINES
}
else if (indexInfo->ii_Concurrent)
{
- snapshot = CopySnapshot(GetTransactionSnapshot());
+ snapshot = RegisterSnapshot(GetTransactionSnapshot());
OldestXmin = InvalidTransactionId; /* not used */
}
else
OldestXmin = GetOldestXmin(heapRelation->rd_rel->relisshared, true);
}
- scan = heap_beginscan(heapRelation, /* relation */
- snapshot, /* seeself */
- 0, /* number of keys */
- NULL); /* scan key */
+ scan = heap_beginscan(heapRelation, snapshot, 0, NULL);
reltuples = 0;
heap_endscan(scan);
+ /* we can now forget our snapshot, if set */
+ if (indexInfo->ii_Concurrent)
+ UnregisterSnapshot(snapshot);
+
ExecDropSingleTupleTableSlot(slot);
FreeExecutorState(estate);
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/cluster.c,v 1.175 2008/05/12 00:00:47 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/cluster.c,v 1.176 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
rvs = get_tables_to_cluster(cluster_context);
/* Commit to get out of starting transaction */
+ PopActiveSnapshot();
CommitTransactionCommand();
/* Ok, now that we've got them all, cluster them one by one */
/* Start a new transaction for each relation. */
StartTransactionCommand();
/* functions in indexes may want a snapshot set */
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
cluster_rel(rvtc, true);
+ PopActiveSnapshot();
CommitTransactionCommand();
}
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/copy.c,v 1.298 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/copy.c,v 1.299 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
plan = planner(query, 0, NULL);
/*
- * Update snapshot command ID to ensure this query sees results of any
- * previously executed queries. (It's a bit cheesy to modify
- * ActiveSnapshot without making a copy, but for the limited ways in
- * which COPY can be invoked, I think it's OK, because the active
- * snapshot shouldn't be shared with anything else anyway.)
+ * Use a snapshot with an updated command ID to ensure this query sees
+ * results of any previously executed queries.
*/
- ActiveSnapshot->curcid = GetCurrentCommandId(false);
+ PushUpdatedSnapshot(GetActiveSnapshot());
/* Create dest receiver for COPY OUT */
dest = CreateDestReceiver(DestCopyOut, NULL);
((DR_copy *) dest)->cstate = cstate;
/* Create a QueryDesc requesting no output */
- cstate->queryDesc = CreateQueryDesc(plan,
- ActiveSnapshot, InvalidSnapshot,
+ cstate->queryDesc = CreateQueryDesc(plan, GetActiveSnapshot(),
+ InvalidSnapshot,
dest, NULL, false);
/*
/* Close down the query and free resources. */
ExecutorEnd(cstate->queryDesc);
FreeQueryDesc(cstate->queryDesc);
+ PopActiveSnapshot();
}
/* Clean up storage (probably not really necessary) */
values = (Datum *) palloc(num_phys_attrs * sizeof(Datum));
nulls = (bool *) palloc(num_phys_attrs * sizeof(bool));
- scandesc = heap_beginscan(cstate->rel, ActiveSnapshot, 0, NULL);
+ scandesc = heap_beginscan(cstate->rel, GetActiveSnapshot(), 0, NULL);
while ((tuple = heap_getnext(scandesc, ForwardScanDirection)) != NULL)
{
* Portions Copyright (c) 1994-5, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/explain.c,v 1.173 2008/04/18 01:42:17 tgl Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/explain.c,v 1.174 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
int eflags;
/*
- * Update snapshot command ID to ensure this query sees results of any
- * previously executed queries. (It's a bit cheesy to modify
- * ActiveSnapshot without making a copy, but for the limited ways in which
- * EXPLAIN can be invoked, I think it's OK, because the active snapshot
- * shouldn't be shared with anything else anyway.)
+ * Use a snapshot with an updated command ID to ensure this query sees
+ * results of any previously executed queries.
*/
- ActiveSnapshot->curcid = GetCurrentCommandId(false);
+ PushUpdatedSnapshot(GetActiveSnapshot());
/* Create a QueryDesc requesting no output */
queryDesc = CreateQueryDesc(plannedstmt,
- ActiveSnapshot, InvalidSnapshot,
+ GetActiveSnapshot(), InvalidSnapshot,
None_Receiver, params,
stmt->analyze);
FreeQueryDesc(queryDesc);
+ PopActiveSnapshot();
+
/* We need a CCI just in case query expanded to multiple plans */
if (stmt->analyze)
CommandCounterIncrement();
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/indexcmds.c,v 1.175 2008/05/12 00:00:47 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/indexcmds.c,v 1.176 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
*/
LockRelationIdForSession(&heaprelid, ShareUpdateExclusiveLock);
+ PopActiveSnapshot();
CommitTransactionCommand();
StartTransactionCommand();
indexRelation = index_open(indexRelationId, RowExclusiveLock);
/* Set ActiveSnapshot since functions in the indexes may need it */
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
/* We have to re-build the IndexInfo struct, since it was lost in commit */
indexInfo = BuildIndexInfo(indexRelation);
heap_close(pg_index, RowExclusiveLock);
+ /* we can do away with our snapshot */
+ PopActiveSnapshot();
+
/*
* Commit this transaction to make the indisready update visible.
*/
* We also set ActiveSnapshot to this snap, since functions in indexes may
* need a snapshot.
*/
- snapshot = CopySnapshot(GetTransactionSnapshot());
- ActiveSnapshot = snapshot;
+ snapshot = RegisterSnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(snapshot);
/*
* Scan the index and the heap, insert any missing index entries.
* Also, GetCurrentVirtualXIDs never reports our own vxid, so we need not
* check for that.
*/
- old_snapshots = GetCurrentVirtualXIDs(ActiveSnapshot->xmax, false,
+ old_snapshots = GetCurrentVirtualXIDs(snapshot->xmax, false,
PROC_IS_AUTOVACUUM | PROC_IN_VACUUM);
while (VirtualTransactionIdIsValid(*old_snapshots))
*/
CacheInvalidateRelcacheByRelid(heaprelid.relId);
+ /* we can now do away with our active snapshot */
+ PopActiveSnapshot();
+
+ /* And we can remove the validating snapshot too */
+ UnregisterSnapshot(snapshot);
+
/*
* Last thing to do is release the session-level lock on the parent table.
*/
heap_close(relationRelation, AccessShareLock);
/* Now reindex each rel in a separate transaction */
+ PopActiveSnapshot();
CommitTransactionCommand();
foreach(l, relids)
{
StartTransactionCommand();
/* functions in indexes may want a snapshot set */
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
if (reindex_relation(relid, true))
ereport(NOTICE,
(errmsg("table \"%s\" was reindexed",
get_rel_name(relid))));
+ PopActiveSnapshot();
CommitTransactionCommand();
}
StartTransactionCommand();
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/portalcmds.c,v 1.73 2008/04/02 18:31:50 tgl Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/portalcmds.c,v 1.74 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
/*
* Start execution, inserting parameters if any.
*/
- PortalStart(portal, params, ActiveSnapshot);
+ PortalStart(portal, params, GetActiveSnapshot());
Assert(portal->strategy == PORTAL_ONE_SELECT);
{
QueryDesc *queryDesc = PortalGetQueryDesc(portal);
Portal saveActivePortal;
- Snapshot saveActiveSnapshot;
ResourceOwner saveResourceOwner;
MemoryContext savePortalContext;
MemoryContext oldcxt;
* Set up global portal context pointers.
*/
saveActivePortal = ActivePortal;
- saveActiveSnapshot = ActiveSnapshot;
saveResourceOwner = CurrentResourceOwner;
savePortalContext = PortalContext;
PG_TRY();
{
ActivePortal = portal;
- ActiveSnapshot = queryDesc->snapshot;
CurrentResourceOwner = portal->resowner;
PortalContext = PortalGetHeapMemory(portal);
MemoryContextSwitchTo(PortalContext);
+ PushActiveSnapshot(queryDesc->snapshot);
+
/*
* Rewind the executor: we need to store the entire result set in the
* tuplestore, so that subsequent backward FETCHs can be processed.
/* Restore global vars and propagate error */
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
CurrentResourceOwner = saveResourceOwner;
PortalContext = savePortalContext;
portal->status = PORTAL_READY;
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
CurrentResourceOwner = saveResourceOwner;
PortalContext = savePortalContext;
+ PopActiveSnapshot();
+
/*
* We can now release any subsidiary memory of the portal's heap context;
* we'll never use it again. The executor already dropped its context,
* Copyright (c) 2002-2008, PostgreSQL Global Development Group
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/prepare.c,v 1.86 2008/05/12 00:00:47 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/prepare.c,v 1.87 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
/*
* Run the portal to completion.
*/
- PortalStart(portal, paramLI, ActiveSnapshot);
+ PortalStart(portal, paramLI, GetActiveSnapshot());
(void) PortalRun(portal, FETCH_ALL, false, dest, dest, completionTag);
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/trigger.c,v 1.232 2008/05/12 00:00:48 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/trigger.c,v 1.233 2008/05/12 20:01:59 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
AfterTriggerFireDeferred(void)
{
AfterTriggerEventList *events;
+ bool snap_pushed = false;
/* Must be inside a transaction */
Assert(afterTriggers != NULL);
*/
events = &afterTriggers->events;
if (events->head != NULL)
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ {
+ PushActiveSnapshot(GetTransactionSnapshot());
+ snap_pushed = true;
+ }
/*
* Run all the remaining triggers. Loop until they are all gone, in case
afterTriggerInvokeEvents(events, firing_id, NULL, true);
}
+ if (snap_pushed)
+ PopActiveSnapshot();
+
Assert(events->head == NULL);
}
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.372 2008/05/12 00:00:48 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.373 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
*/
if (use_own_xacts)
{
+ /* ActiveSnapshot is not set by autovacuum */
+ if (ActiveSnapshotSet())
+ PopActiveSnapshot();
+
/* matches the StartTransaction in PostgresMain() */
CommitTransactionCommand();
}
{
StartTransactionCommand();
/* functions in indexes may want a snapshot set */
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
}
else
old_context = MemoryContextSwitchTo(anl_context);
analyze_rel(relid, vacstmt, vac_strategy);
if (use_own_xacts)
+ {
+ PopActiveSnapshot();
CommitTransactionCommand();
+ }
else
{
MemoryContextSwitchTo(old_context);
if (vacstmt->full)
{
/* functions in indexes may want a snapshot set */
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
}
else
{
if (!onerel)
{
+ if (vacstmt->full)
+ PopActiveSnapshot();
CommitTransactionCommand();
return;
}
(errmsg("skipping \"%s\" --- only table or database owner can vacuum it",
RelationGetRelationName(onerel))));
relation_close(onerel, lmode);
+ if (vacstmt->full)
+ PopActiveSnapshot();
CommitTransactionCommand();
return;
}
(errmsg("skipping \"%s\" --- cannot vacuum indexes, views, or special system tables",
RelationGetRelationName(onerel))));
relation_close(onerel, lmode);
+ if (vacstmt->full)
+ PopActiveSnapshot();
CommitTransactionCommand();
return;
}
if (isOtherTempNamespace(RelationGetNamespace(onerel)))
{
relation_close(onerel, lmode);
+ if (vacstmt->full)
+ PopActiveSnapshot();
CommitTransactionCommand();
return;
}
/*
* Complete the transaction and free all temporary memory used.
*/
+ if (vacstmt->full)
+ PopActiveSnapshot();
CommitTransactionCommand();
/*
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/commands/variable.c,v 1.127 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/commands/variable.c,v 1.128 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
const char *
assign_XactIsoLevel(const char *value, bool doit, GucSource source)
{
- if (SerializableSnapshot != NULL)
+ if (FirstSnapshotSet)
{
ereport(GUC_complaint_elevel(source),
(errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/executor/execMain.c,v 1.308 2008/05/12 00:00:48 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/executor/execMain.c,v 1.309 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
#include "utils/acl.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/snapmgr.h"
#include "utils/tqual.h"
/*
* Copy other important information into the EState
*/
- estate->es_snapshot = queryDesc->snapshot;
- estate->es_crosscheck_snapshot = queryDesc->crosscheck_snapshot;
+ estate->es_snapshot = RegisterSnapshot(queryDesc->snapshot);
+ estate->es_crosscheck_snapshot = RegisterSnapshot(queryDesc->crosscheck_snapshot);
estate->es_instrument = queryDesc->doInstrument;
/*
if (estate->es_select_into)
CloseIntoRel(queryDesc);
+ /* do away with our snapshots */
+ UnregisterSnapshot(estate->es_snapshot);
+ UnregisterSnapshot(estate->es_crosscheck_snapshot);
+
/*
* Must switch out of context before destroying it
*/
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/executor/functions.c,v 1.124 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/executor/functions.c,v 1.125 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
* In a read-only function, use the surrounding query's snapshot;
* otherwise take a new snapshot for each query. The snapshot should
* include a fresh command ID so that all work to date in this transaction
- * is visible. We copy in both cases so that postquel_end can
- * unconditionally do FreeSnapshot.
+ * is visible.
*/
if (fcache->readonly_func)
- snapshot = CopySnapshot(ActiveSnapshot);
+ snapshot = GetActiveSnapshot();
else
{
CommandCounterIncrement();
- snapshot = CopySnapshot(GetTransactionSnapshot());
+ snapshot = GetTransactionSnapshot();
}
if (IsA(es->stmt, PlannedStmt))
postquel_getnext(execution_state *es, SQLFunctionCachePtr fcache)
{
TupleTableSlot *result;
- Snapshot saveActiveSnapshot;
long count;
/* Make our snapshot the active one for any called functions */
- saveActiveSnapshot = ActiveSnapshot;
- PG_TRY();
- {
- ActiveSnapshot = es->qd->snapshot;
-
- if (es->qd->utilitystmt)
- {
- /* ProcessUtility needs the PlannedStmt for DECLARE CURSOR */
- ProcessUtility((es->qd->plannedstmt ?
- (Node *) es->qd->plannedstmt :
- es->qd->utilitystmt),
- fcache->src,
- es->qd->params,
- false, /* not top level */
- es->qd->dest,
- NULL);
- result = NULL;
- }
- else
- {
- /*
- * If it's the function's last command, and it's a SELECT, fetch
- * one row at a time so we can return the results. Otherwise just
- * run it to completion. (If we run to completion then
- * ExecutorRun is guaranteed to return NULL.)
- */
- if (LAST_POSTQUEL_COMMAND(es) &&
- es->qd->operation == CMD_SELECT &&
- es->qd->plannedstmt->utilityStmt == NULL &&
- es->qd->plannedstmt->intoClause == NULL)
- count = 1L;
- else
- count = 0L;
+ PushActiveSnapshot(es->qd->snapshot);
- result = ExecutorRun(es->qd, ForwardScanDirection, count);
- }
+ if (es->qd->utilitystmt)
+ {
+ /* ProcessUtility needs the PlannedStmt for DECLARE CURSOR */
+ ProcessUtility((es->qd->plannedstmt ?
+ (Node *) es->qd->plannedstmt :
+ es->qd->utilitystmt),
+ fcache->src,
+ es->qd->params,
+ false, /* not top level */
+ es->qd->dest,
+ NULL);
+ result = NULL;
}
- PG_CATCH();
+ else
{
- /* Restore global vars and propagate error */
- ActiveSnapshot = saveActiveSnapshot;
- PG_RE_THROW();
+ /*
+ * If it's the function's last command, and it's a SELECT, fetch
+ * one row at a time so we can return the results. Otherwise just
+ * run it to completion. (If we run to completion then
+ * ExecutorRun is guaranteed to return NULL.)
+ */
+ if (LAST_POSTQUEL_COMMAND(es) &&
+ es->qd->operation == CMD_SELECT &&
+ es->qd->plannedstmt->utilityStmt == NULL &&
+ es->qd->plannedstmt->intoClause == NULL)
+ count = 1L;
+ else
+ count = 0L;
+
+ result = ExecutorRun(es->qd, ForwardScanDirection, count);
}
- PG_END_TRY();
- ActiveSnapshot = saveActiveSnapshot;
+ PopActiveSnapshot();
return result;
}
static void
postquel_end(execution_state *es)
{
- Snapshot saveActiveSnapshot;
-
/* mark status done to ensure we don't do ExecutorEnd twice */
es->status = F_EXEC_DONE;
if (es->qd->utilitystmt == NULL)
{
/* Make our snapshot the active one for any called functions */
- saveActiveSnapshot = ActiveSnapshot;
- PG_TRY();
- {
- ActiveSnapshot = es->qd->snapshot;
+ PushActiveSnapshot(es->qd->snapshot);
- if (es->qd->operation != CMD_SELECT)
- AfterTriggerEndQuery(es->qd->estate);
- ExecutorEnd(es->qd);
- }
- PG_CATCH();
- {
- /* Restore global vars and propagate error */
- ActiveSnapshot = saveActiveSnapshot;
- PG_RE_THROW();
- }
- PG_END_TRY();
- ActiveSnapshot = saveActiveSnapshot;
+ if (es->qd->operation != CMD_SELECT)
+ AfterTriggerEndQuery(es->qd->estate);
+ ExecutorEnd(es->qd);
+
+ PopActiveSnapshot();
}
- FreeSnapshot(es->qd->snapshot);
FreeQueryDesc(es->qd);
es->qd = NULL;
}
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/executor/spi.c,v 1.194 2008/05/12 00:00:49 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/executor/spi.c,v 1.195 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
/*
* SPI_execute_snapshot -- identical to SPI_execute_plan, except that we allow
- * the caller to specify exactly which snapshots to use. Also, the caller
- * may specify that AFTER triggers should be queued as part of the outer
- * query rather than being fired immediately at the end of the command.
+ * the caller to specify exactly which snapshots to use, which will be
+ * registered here. Also, the caller may specify that AFTER triggers should be
+ * queued as part of the outer query rather than being fired immediately at the
+ * end of the command.
*
* This is currently not documented in spi.sgml because it is only intended
* for use by RI triggers.
}
/*
- * Set up the snapshot to use. (PortalStart will do CopySnapshot, so we
- * skip that here.)
+ * Set up the snapshot to use. (PortalStart will do PushActiveSnapshot, so
+ * we skip that here.)
*/
if (read_only)
- snapshot = ActiveSnapshot;
+ snapshot = GetActiveSnapshot();
else
{
CommandCounterIncrement();
Snapshot snapshot, Snapshot crosscheck_snapshot,
bool read_only, bool fire_triggers, long tcount)
{
- volatile int my_res = 0;
- volatile uint32 my_processed = 0;
- volatile Oid my_lastoid = InvalidOid;
- SPITupleTable *volatile my_tuptable = NULL;
- volatile int res = 0;
- Snapshot saveActiveSnapshot;
-
- /* Be sure to restore ActiveSnapshot on error exit */
- saveActiveSnapshot = ActiveSnapshot;
- PG_TRY();
+ int my_res = 0;
+ uint32 my_processed = 0;
+ Oid my_lastoid = InvalidOid;
+ SPITupleTable *my_tuptable = NULL;
+ int res = 0;
+ bool have_active_snap = ActiveSnapshotSet();
+ ErrorContextCallback spierrcontext;
+ CachedPlan *cplan = NULL;
+ ListCell *lc1;
+
+ /*
+ * Setup error traceback support for ereport()
+ */
+ spierrcontext.callback = _SPI_error_callback;
+ spierrcontext.arg = NULL;
+ spierrcontext.previous = error_context_stack;
+ error_context_stack = &spierrcontext;
+
+ foreach(lc1, plan->plancache_list)
{
- ErrorContextCallback spierrcontext;
- CachedPlan *cplan = NULL;
- ListCell *lc1;
+ CachedPlanSource *plansource = (CachedPlanSource *) lfirst(lc1);
+ List *stmt_list;
+ ListCell *lc2;
- /*
- * Setup error traceback support for ereport()
- */
- spierrcontext.callback = _SPI_error_callback;
- spierrcontext.arg = NULL;
- spierrcontext.previous = error_context_stack;
- error_context_stack = &spierrcontext;
+ spierrcontext.arg = (void *) plansource->query_string;
- foreach(lc1, plan->plancache_list)
+ if (plan->saved)
{
- CachedPlanSource *plansource = (CachedPlanSource *) lfirst(lc1);
- List *stmt_list;
- ListCell *lc2;
+ /* Replan if needed, and increment plan refcount locally */
+ cplan = RevalidateCachedPlan(plansource, true);
+ stmt_list = cplan->stmt_list;
+ }
+ else
+ {
+ /* No replan here */
+ cplan = NULL;
+ stmt_list = plansource->plan->stmt_list;
+ }
+
+ foreach(lc2, stmt_list)
+ {
+ Node *stmt = (Node *) lfirst(lc2);
+ bool canSetTag;
+ DestReceiver *dest;
+ bool pushed_active_snap = false;
- spierrcontext.arg = (void *) plansource->query_string;
+ _SPI_current->processed = 0;
+ _SPI_current->lastoid = InvalidOid;
+ _SPI_current->tuptable = NULL;
- if (plan->saved)
+ if (IsA(stmt, PlannedStmt))
{
- /* Replan if needed, and increment plan refcount locally */
- cplan = RevalidateCachedPlan(plansource, true);
- stmt_list = cplan->stmt_list;
+ canSetTag = ((PlannedStmt *) stmt)->canSetTag;
}
else
{
- /* No replan here */
- cplan = NULL;
- stmt_list = plansource->plan->stmt_list;
- }
-
- foreach(lc2, stmt_list)
- {
- Node *stmt = (Node *) lfirst(lc2);
- bool canSetTag;
- DestReceiver *dest;
+ /* utilities are canSetTag if only thing in list */
+ canSetTag = (list_length(stmt_list) == 1);
- _SPI_current->processed = 0;
- _SPI_current->lastoid = InvalidOid;
- _SPI_current->tuptable = NULL;
-
- if (IsA(stmt, PlannedStmt))
+ if (IsA(stmt, CopyStmt))
{
- canSetTag = ((PlannedStmt *) stmt)->canSetTag;
- }
- else
- {
- /* utilities are canSetTag if only thing in list */
- canSetTag = (list_length(stmt_list) == 1);
-
- if (IsA(stmt, CopyStmt))
- {
- CopyStmt *cstmt = (CopyStmt *) stmt;
+ CopyStmt *cstmt = (CopyStmt *) stmt;
- if (cstmt->filename == NULL)
- {
- my_res = SPI_ERROR_COPY;
- goto fail;
- }
- }
- else if (IsA(stmt, TransactionStmt))
+ if (cstmt->filename == NULL)
{
- my_res = SPI_ERROR_TRANSACTION;
+ my_res = SPI_ERROR_COPY;
goto fail;
}
}
+ else if (IsA(stmt, TransactionStmt))
+ {
+ my_res = SPI_ERROR_TRANSACTION;
+ goto fail;
+ }
+ }
- if (read_only && !CommandIsReadOnly(stmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(stmt))));
-
- /*
- * If not read-only mode, advance the command counter before
- * each command.
- */
- if (!read_only)
- CommandCounterIncrement();
+ if (read_only && !CommandIsReadOnly(stmt))
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(stmt))));
- dest = CreateDestReceiver(canSetTag ? DestSPI : DestNone,
- NULL);
+ /*
+ * If not read-only mode, advance the command counter before
+ * each command.
+ */
+ if (!read_only)
+ CommandCounterIncrement();
- if (snapshot == InvalidSnapshot)
- {
- /*
- * Default read_only behavior is to use the entry-time
- * ActiveSnapshot; if read-write, grab a full new snap.
- */
- if (read_only)
- ActiveSnapshot = CopySnapshot(saveActiveSnapshot);
- else
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
- }
- else
- {
- /*
- * We interpret read_only with a specified snapshot to be
- * exactly that snapshot, but read-write means use the
- * snap with advancing of command ID.
- */
- ActiveSnapshot = CopySnapshot(snapshot);
- if (!read_only)
- ActiveSnapshot->curcid = GetCurrentCommandId(false);
- }
+ dest = CreateDestReceiver(canSetTag ? DestSPI : DestNone,
+ NULL);
- if (IsA(stmt, PlannedStmt) &&
- ((PlannedStmt *) stmt)->utilityStmt == NULL)
+ if (snapshot == InvalidSnapshot)
+ {
+ /*
+ * Default read_only behavior is to use the entry-time
+ * ActiveSnapshot, if any; if read-write, grab a full new snap.
+ */
+ if (read_only)
{
- QueryDesc *qdesc;
-
- qdesc = CreateQueryDesc((PlannedStmt *) stmt,
- ActiveSnapshot,
- crosscheck_snapshot,
- dest,
- paramLI, false);
- res = _SPI_pquery(qdesc, fire_triggers,
- canSetTag ? tcount : 0);
- FreeQueryDesc(qdesc);
+ if (have_active_snap)
+ {
+ PushActiveSnapshot(GetActiveSnapshot());
+ pushed_active_snap = true;
+ }
}
else
{
- ProcessUtility(stmt,
- plansource->query_string,
- paramLI,
- false, /* not top level */
- dest,
- NULL);
- /* Update "processed" if stmt returned tuples */
- if (_SPI_current->tuptable)
- _SPI_current->processed = _SPI_current->tuptable->alloced - _SPI_current->tuptable->free;
- res = SPI_OK_UTILITY;
+ PushActiveSnapshot(GetTransactionSnapshot());
+ pushed_active_snap = true;
}
- FreeSnapshot(ActiveSnapshot);
- ActiveSnapshot = NULL;
-
+ }
+ else
+ {
/*
- * The last canSetTag query sets the status values returned to
- * the caller. Be careful to free any tuptables not returned,
- * to avoid intratransaction memory leak.
+ * We interpret read_only with a specified snapshot to be
+ * exactly that snapshot, but read-write means use the
+ * snap with advancing of command ID.
*/
- if (canSetTag)
- {
- my_processed = _SPI_current->processed;
- my_lastoid = _SPI_current->lastoid;
- SPI_freetuptable(my_tuptable);
- my_tuptable = _SPI_current->tuptable;
- my_res = res;
- }
+ if (read_only)
+ PushActiveSnapshot(snapshot);
else
- {
- SPI_freetuptable(_SPI_current->tuptable);
- _SPI_current->tuptable = NULL;
- }
- /* we know that the receiver doesn't need a destroy call */
- if (res < 0)
- {
- my_res = res;
- goto fail;
- }
+ PushUpdatedSnapshot(snapshot);
+ pushed_active_snap = true;
}
- /* Done with this plan, so release refcount */
- if (cplan)
- ReleaseCachedPlan(cplan, true);
- cplan = NULL;
+ if (IsA(stmt, PlannedStmt) &&
+ ((PlannedStmt *) stmt)->utilityStmt == NULL)
+ {
+ QueryDesc *qdesc;
+ Snapshot snap;
+
+ if (ActiveSnapshotSet())
+ snap = GetActiveSnapshot();
+ else
+ snap = InvalidSnapshot;
+
+ qdesc = CreateQueryDesc((PlannedStmt *) stmt,
+ snap, crosscheck_snapshot,
+ dest,
+ paramLI, false);
+ res = _SPI_pquery(qdesc, fire_triggers,
+ canSetTag ? tcount : 0);
+ FreeQueryDesc(qdesc);
+ }
+ else
+ {
+ ProcessUtility(stmt,
+ plansource->query_string,
+ paramLI,
+ false, /* not top level */
+ dest,
+ NULL);
+ /* Update "processed" if stmt returned tuples */
+ if (_SPI_current->tuptable)
+ _SPI_current->processed = _SPI_current->tuptable->alloced -
+ _SPI_current->tuptable->free;
+ res = SPI_OK_UTILITY;
+ }
+
+ if (pushed_active_snap)
+ PopActiveSnapshot();
/*
- * If not read-only mode, advance the command counter after the
- * last command. This ensures that its effects are visible, in
- * case it was DDL that would affect the next CachedPlanSource.
+ * The last canSetTag query sets the status values returned to
+ * the caller. Be careful to free any tuptables not returned,
+ * to avoid intratransaction memory leak.
*/
- if (!read_only)
- CommandCounterIncrement();
+ if (canSetTag)
+ {
+ my_processed = _SPI_current->processed;
+ my_lastoid = _SPI_current->lastoid;
+ SPI_freetuptable(my_tuptable);
+ my_tuptable = _SPI_current->tuptable;
+ my_res = res;
+ }
+ else
+ {
+ SPI_freetuptable(_SPI_current->tuptable);
+ _SPI_current->tuptable = NULL;
+ }
+ /* we know that the receiver doesn't need a destroy call */
+ if (res < 0)
+ {
+ my_res = res;
+ goto fail;
+ }
}
-fail:
-
- /* We no longer need the cached plan refcount, if any */
+ /* Done with this plan, so release refcount */
if (cplan)
ReleaseCachedPlan(cplan, true);
+ cplan = NULL;
/*
- * Pop the error context stack
+ * If not read-only mode, advance the command counter after the
+ * last command. This ensures that its effects are visible, in
+ * case it was DDL that would affect the next CachedPlanSource.
*/
- error_context_stack = spierrcontext.previous;
- }
- PG_CATCH();
- {
- /* Restore global vars and propagate error */
- ActiveSnapshot = saveActiveSnapshot;
- PG_RE_THROW();
+ if (!read_only)
+ CommandCounterIncrement();
}
- PG_END_TRY();
- ActiveSnapshot = saveActiveSnapshot;
+fail:
+
+ /* We no longer need the cached plan refcount, if any */
+ if (cplan)
+ ReleaseCachedPlan(cplan, true);
+
+ /*
+ * Pop the error context stack
+ */
+ error_context_stack = spierrcontext.previous;
/* Save results for caller */
SPI_processed = my_processed;
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.43 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/ipc/procarray.c,v 1.44 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
*
* We also update the following backend-global variables:
* TransactionXmin: the oldest xmin of any snapshot in use in the
- * current transaction (this is the same as MyProc->xmin). This
- * is just the xmin computed for the first, serializable snapshot.
+ * current transaction (this is the same as MyProc->xmin).
* RecentXmin: the xmin computed for the most recent snapshot. XIDs
* older than this are known not running any more.
* RecentGlobalXmin: the global xmin (oldest TransactionXmin across all
* the same computation done by GetOldestXmin(true, true).
*/
Snapshot
-GetSnapshotData(Snapshot snapshot, bool serializable)
+GetSnapshotData(Snapshot snapshot)
{
ProcArrayStruct *arrayP = procArray;
TransactionId xmin;
Assert(snapshot != NULL);
- /* Serializable snapshot must be computed before any other... */
- Assert(serializable ?
- !TransactionIdIsValid(MyProc->xmin) :
- TransactionIdIsValid(MyProc->xmin));
-
/*
* Allocating space for maxProcs xids is usually overkill; numProcs would
* be sufficient. But it seems better to do the malloc while not holding
}
}
- if (serializable)
+ if (!TransactionIdIsValid(MyProc->xmin))
MyProc->xmin = TransactionXmin = xmin;
LWLockRelease(ProcArrayLock);
snapshot->curcid = GetCurrentCommandId(false);
+ /*
+ * This is a new snapshot, so set both refcounts are zero, and mark it
+ * as not copied in persistent memory.
+ */
+ snapshot->active_count = 0;
+ snapshot->regd_count = 0;
+ snapshot->copied = false;
+
return snapshot;
}
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/storage/large_object/inv_api.c,v 1.132 2008/04/12 23:14:21 tgl Exp $
+ * $PostgreSQL: pgsql/src/backend/storage/large_object/inv_api.c,v 1.133 2008/05/12 20:02:00 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
}
else if (flags & INV_READ)
{
- /* be sure to copy snap into mcxt */
- MemoryContext oldContext = MemoryContextSwitchTo(mcxt);
-
- retval->snapshot = CopySnapshot(ActiveSnapshot);
+ retval->snapshot = RegisterSnapshot(GetActiveSnapshot());
retval->flags = IFS_RDLOCK;
- MemoryContextSwitchTo(oldContext);
}
else
elog(ERROR, "invalid flags: %d", flags);
{
Assert(PointerIsValid(obj_desc));
if (obj_desc->snapshot != SnapshotNow)
- FreeSnapshot(obj_desc->snapshot);
+ UnregisterSnapshot(obj_desc->snapshot);
pfree(obj_desc);
}
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/tcop/fastpath.c,v 1.99 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/tcop/fastpath.c,v 1.100 2008/05/12 20:02:01 alvherre Exp $
*
* NOTES
* This cruft is the server side of PQfn.
* Now that we know we are in a valid transaction, set snapshot in case
* needed by function itself or one of the datatype I/O routines.
*/
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
/*
* Begin parsing the buffer contents.
SendFunctionResult(retval, fcinfo.isnull, fip->rettype, rformat);
+ /* We no longer need the snapshot */
+ PopActiveSnapshot();
+
/*
* Emit duration logging if appropriate.
*/
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.551 2008/05/12 00:00:50 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.552 2008/05/12 20:02:01 alvherre Exp $
*
* NOTES
* this is the "main" module of the postgres backend and
pg_plan_queries(List *querytrees, int cursorOptions, ParamListInfo boundParams,
bool needSnapshot)
{
- List * volatile stmt_list = NIL;
- Snapshot saveActiveSnapshot = ActiveSnapshot;
+ List *stmt_list = NIL;
+ ListCell *query_list;
+ bool snapshot_set = false;
- /* PG_TRY to ensure previous ActiveSnapshot is restored on error */
- PG_TRY();
+ foreach(query_list, querytrees)
{
- Snapshot mySnapshot = NULL;
- ListCell *query_list;
+ Query *query = (Query *) lfirst(query_list);
+ Node *stmt;
- foreach(query_list, querytrees)
+ if (query->commandType == CMD_UTILITY)
{
- Query *query = (Query *) lfirst(query_list);
- Node *stmt;
-
- if (query->commandType == CMD_UTILITY)
- {
- /* Utility commands have no plans. */
- stmt = query->utilityStmt;
- }
- else
+ /* Utility commands have no plans. */
+ stmt = query->utilityStmt;
+ }
+ else
+ {
+ if (needSnapshot && !snapshot_set)
{
- if (needSnapshot && mySnapshot == NULL)
- {
- mySnapshot = CopySnapshot(GetTransactionSnapshot());
- ActiveSnapshot = mySnapshot;
- }
- stmt = (Node *) pg_plan_query(query, cursorOptions,
- boundParams);
+ PushActiveSnapshot(GetTransactionSnapshot());
+ snapshot_set = true;
}
- stmt_list = lappend(stmt_list, stmt);
+ stmt = (Node *) pg_plan_query(query, cursorOptions,
+ boundParams);
}
- if (mySnapshot)
- FreeSnapshot(mySnapshot);
+ stmt_list = lappend(stmt_list, stmt);
}
- PG_CATCH();
- {
- ActiveSnapshot = saveActiveSnapshot;
- PG_RE_THROW();
- }
- PG_END_TRY();
- ActiveSnapshot = saveActiveSnapshot;
+
+ if (snapshot_set)
+ PopActiveSnapshot();
return stmt_list;
}
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/tcop/pquery.c,v 1.122 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/tcop/pquery.c,v 1.123 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
qd->operation = plannedstmt->commandType; /* operation */
qd->plannedstmt = plannedstmt; /* plan */
qd->utilitystmt = plannedstmt->utilityStmt; /* in case DECLARE CURSOR */
- qd->snapshot = snapshot; /* snapshot */
- qd->crosscheck_snapshot = crosscheck_snapshot; /* RI check snapshot */
+ qd->snapshot = RegisterSnapshot(snapshot); /* snapshot */
+ /* RI check snapshot */
+ qd->crosscheck_snapshot = RegisterSnapshot(crosscheck_snapshot);
qd->dest = dest; /* output dest */
qd->params = params; /* parameter values passed into query */
qd->doInstrument = doInstrument; /* instrumentation wanted? */
qd->operation = CMD_UTILITY; /* operation */
qd->plannedstmt = NULL;
qd->utilitystmt = utilitystmt; /* utility command */
- qd->snapshot = snapshot; /* snapshot */
+ qd->snapshot = RegisterSnapshot(snapshot); /* snapshot */
qd->crosscheck_snapshot = InvalidSnapshot; /* RI check snapshot */
qd->dest = dest; /* output dest */
qd->params = params; /* parameter values passed into query */
{
/* Can't be a live query */
Assert(qdesc->estate == NULL);
+
+ /* forget our snapshots */
+ UnregisterSnapshot(qdesc->snapshot);
+ UnregisterSnapshot(qdesc->crosscheck_snapshot);
+
/* Only the QueryDesc itself need be freed */
pfree(qdesc);
}
elog(DEBUG3, "ProcessQuery");
/*
- * Must always set snapshot for plannable queries. Note we assume that
- * caller will take care of restoring ActiveSnapshot on exit/error.
+ * Must always set a snapshot for plannable queries.
*/
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
/*
* Create the QueryDesc object
*/
queryDesc = CreateQueryDesc(plan,
- ActiveSnapshot, InvalidSnapshot,
+ GetActiveSnapshot(), InvalidSnapshot,
dest, params, false);
/*
/* Now take care of any queued AFTER triggers */
AfterTriggerEndQuery(queryDesc->estate);
+ PopActiveSnapshot();
+
/*
* Now, we close down all the scans and free allocated resources.
*/
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
-
- FreeSnapshot(ActiveSnapshot);
- ActiveSnapshot = NULL;
}
/*
PortalStart(Portal portal, ParamListInfo params, Snapshot snapshot)
{
Portal saveActivePortal;
- Snapshot saveActiveSnapshot;
ResourceOwner saveResourceOwner;
MemoryContext savePortalContext;
MemoryContext oldContext;
* Set up global portal context pointers.
*/
saveActivePortal = ActivePortal;
- saveActiveSnapshot = ActiveSnapshot;
saveResourceOwner = CurrentResourceOwner;
savePortalContext = PortalContext;
PG_TRY();
{
ActivePortal = portal;
- ActiveSnapshot = NULL; /* will be set later */
CurrentResourceOwner = portal->resowner;
PortalContext = PortalGetHeapMemory(portal);
{
case PORTAL_ONE_SELECT:
- /*
- * Must set snapshot before starting executor. Be sure to
- * copy it into the portal's context.
- */
+ /* Must set snapshot before starting executor. */
if (snapshot)
- ActiveSnapshot = CopySnapshot(snapshot);
+ PushActiveSnapshot(snapshot);
else
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ PushActiveSnapshot(GetTransactionSnapshot());
/*
* Create QueryDesc in portal's context; for the moment, set
* the destination to DestNone.
*/
queryDesc = CreateQueryDesc((PlannedStmt *) linitial(portal->stmts),
- ActiveSnapshot,
+ GetActiveSnapshot(),
InvalidSnapshot,
None_Receiver,
params,
portal->atEnd = false; /* allow fetches */
portal->portalPos = 0;
portal->posOverflow = false;
+
+ PopActiveSnapshot();
break;
case PORTAL_ONE_RETURNING:
/* Restore global vars and propagate error */
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
CurrentResourceOwner = saveResourceOwner;
PortalContext = savePortalContext;
MemoryContextSwitchTo(oldContext);
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
CurrentResourceOwner = saveResourceOwner;
PortalContext = savePortalContext;
ResourceOwner saveTopTransactionResourceOwner;
MemoryContext saveTopTransactionContext;
Portal saveActivePortal;
- Snapshot saveActiveSnapshot;
ResourceOwner saveResourceOwner;
MemoryContext savePortalContext;
MemoryContext saveMemoryContext;
saveTopTransactionResourceOwner = TopTransactionResourceOwner;
saveTopTransactionContext = TopTransactionContext;
saveActivePortal = ActivePortal;
- saveActiveSnapshot = ActiveSnapshot;
saveResourceOwner = CurrentResourceOwner;
savePortalContext = PortalContext;
saveMemoryContext = CurrentMemoryContext;
PG_TRY();
{
ActivePortal = portal;
- ActiveSnapshot = NULL; /* will be set later */
CurrentResourceOwner = portal->resowner;
PortalContext = PortalGetHeapMemory(portal);
else
MemoryContextSwitchTo(saveMemoryContext);
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
if (saveResourceOwner == saveTopTransactionResourceOwner)
CurrentResourceOwner = TopTransactionResourceOwner;
else
else
MemoryContextSwitchTo(saveMemoryContext);
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
if (saveResourceOwner == saveTopTransactionResourceOwner)
CurrentResourceOwner = TopTransactionResourceOwner;
else
nprocessed = RunFromStore(portal, direction, count, dest);
else
{
- ActiveSnapshot = queryDesc->snapshot;
+ PushActiveSnapshot(queryDesc->snapshot);
ExecutorRun(queryDesc, direction, count);
nprocessed = queryDesc->estate->es_processed;
+ PopActiveSnapshot();
}
if (!ScanDirectionIsNoMovement(direction))
nprocessed = RunFromStore(portal, direction, count, dest);
else
{
- ActiveSnapshot = queryDesc->snapshot;
+ PushActiveSnapshot(queryDesc->snapshot);
ExecutorRun(queryDesc, direction, count);
nprocessed = queryDesc->estate->es_processed;
+ PopActiveSnapshot();
}
if (!ScanDirectionIsNoMovement(direction))
PortalRunUtility(Portal portal, Node *utilityStmt, bool isTopLevel,
DestReceiver *dest, char *completionTag)
{
+ bool active_snapshot_set;
+
elog(DEBUG3, "ProcessUtility");
/*
* hacks. Beware of listing anything that can modify the database --- if,
* say, it has to update an index with expressions that invoke
* user-defined functions, then it had better have a snapshot.
- *
- * Note we assume that caller will take care of restoring ActiveSnapshot
- * on exit/error.
*/
if (!(IsA(utilityStmt, TransactionStmt) ||
IsA(utilityStmt, LockStmt) ||
IsA(utilityStmt, NotifyStmt) ||
IsA(utilityStmt, UnlistenStmt) ||
IsA(utilityStmt, CheckPointStmt)))
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
+ {
+ PushActiveSnapshot(GetTransactionSnapshot());
+ active_snapshot_set = true;
+ }
else
- ActiveSnapshot = NULL;
+ active_snapshot_set = false;
ProcessUtility(utilityStmt,
portal->sourceText,
/* Some utility statements may change context on us */
MemoryContextSwitchTo(PortalGetHeapMemory(portal));
- if (ActiveSnapshot)
- FreeSnapshot(ActiveSnapshot);
- ActiveSnapshot = NULL;
+ /*
+ * Some utility commands may pop the ActiveSnapshot stack from under us,
+ * so we only pop the stack if we actually see a snapshot set. Note that
+ * the set of utility commands that do this must be the same set
+ * disallowed to run inside a transaction; otherwise, we could be popping
+ * a snapshot that belongs to some other operation.
+ */
+ if (active_snapshot_set && ActiveSnapshotSet())
+ PopActiveSnapshot();
}
/*
{
long result;
Portal saveActivePortal;
- Snapshot saveActiveSnapshot;
ResourceOwner saveResourceOwner;
MemoryContext savePortalContext;
MemoryContext oldContext;
* Set up global portal context pointers.
*/
saveActivePortal = ActivePortal;
- saveActiveSnapshot = ActiveSnapshot;
saveResourceOwner = CurrentResourceOwner;
savePortalContext = PortalContext;
PG_TRY();
{
ActivePortal = portal;
- ActiveSnapshot = NULL; /* will be set later */
CurrentResourceOwner = portal->resowner;
PortalContext = PortalGetHeapMemory(portal);
/* Restore global vars and propagate error */
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
CurrentResourceOwner = saveResourceOwner;
PortalContext = savePortalContext;
portal->status = PORTAL_READY;
ActivePortal = saveActivePortal;
- ActiveSnapshot = saveActiveSnapshot;
CurrentResourceOwner = saveResourceOwner;
PortalContext = savePortalContext;
*
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
*
- * $PostgreSQL: pgsql/src/backend/utils/adt/ri_triggers.c,v 1.107 2008/03/26 21:10:39 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/adt/ri_triggers.c,v 1.108 2008/05/12 20:02:02 alvherre Exp $
*
* ----------
*/
/*
* Run the plan. For safety we force a current snapshot to be used. (In
* serializable mode, this arguably violates serializability, but we
- * really haven't got much choice.) We need at most one tuple returned,
- * so pass limit = 1.
+ * really haven't got much choice.) We don't need to register the
+ * snapshot, because SPI_execute_snapshot will see to it. We need at most
+ * one tuple returned, so pass limit = 1.
*/
spi_result = SPI_execute_snapshot(qplan,
NULL, NULL,
- CopySnapshot(GetLatestSnapshot()),
+ GetLatestSnapshot(),
InvalidSnapshot,
true, false, 1);
* caller passes detectNewRows == false then it's okay to do the query
* with the transaction snapshot; otherwise we use a current snapshot, and
* tell the executor to error out if it finds any rows under the current
- * snapshot that wouldn't be visible per the transaction snapshot.
+ * snapshot that wouldn't be visible per the transaction snapshot. Note
+ * that SPI_execute_snapshot will register the snapshots, so we don't need
+ * to bother here.
*/
if (IsXactIsoLevelSerializable && detectNewRows)
{
CommandCounterIncrement(); /* be sure all my own work is visible */
- test_snapshot = CopySnapshot(GetLatestSnapshot());
- crosscheck_snapshot = CopySnapshot(GetTransactionSnapshot());
+ test_snapshot = GetLatestSnapshot();
+ crosscheck_snapshot = GetTransactionSnapshot();
}
else
{
* Author: Jan Wieck, Afilias USA INC.
* 64-bit txids: Marko Kreen, Skype Technologies
*
- * $PostgreSQL: pgsql/src/backend/utils/adt/txid.c,v 1.6 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/adt/txid.c,v 1.7 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
TxidEpoch state;
Snapshot cur;
- cur = ActiveSnapshot;
+ cur = GetActiveSnapshot();
if (cur == NULL)
- elog(ERROR, "txid_current_snapshot: ActiveSnapshot == NULL");
+ elog(ERROR, "no active snapshot set");
load_xid_epoch(&state);
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/utils/cache/plancache.c,v 1.17 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/cache/plancache.c,v 1.18 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
static void StoreCachedPlan(CachedPlanSource *plansource, List *stmt_list,
MemoryContext plan_context);
-static List *do_planning(List *querytrees, int cursorOptions);
static void AcquireExecutorLocks(List *stmt_list, bool acquire);
static void AcquirePlannerLocks(List *stmt_list, bool acquire);
static void LockRelid(Oid relid, LOCKMODE lockmode, void *arg);
if (plansource->fully_planned)
{
- /* Generate plans for queries */
- slist = do_planning(slist, plansource->cursor_options);
+ /*
+ * Generate plans for queries.
+ *
+ * If a snapshot is already set (the normal case), we can just use
+ * that for planning. But if it isn't, we have to tell
+ * pg_plan_queries to make a snap if it needs one.
+ */
+ slist = pg_plan_queries(slist, plansource->cursor_options,
+ NULL, !ActiveSnapshotSet());
}
/*
return plan;
}
-/*
- * Invoke the planner on some rewritten queries. This is broken out of
- * RevalidateCachedPlan just to avoid plastering "volatile" all over that
- * function's variables.
- */
-static List *
-do_planning(List *querytrees, int cursorOptions)
-{
- List *stmt_list;
-
- /*
- * If a snapshot is already set (the normal case), we can just use that
- * for planning. But if it isn't, we have to tell pg_plan_queries to make
- * a snap if it needs one. In that case we should arrange to reset
- * ActiveSnapshot afterward, to ensure that RevalidateCachedPlan has no
- * caller-visible effects on the snapshot. Having to replan is an unusual
- * case, and it seems a really bad idea for RevalidateCachedPlan to affect
- * the snapshot only in unusual cases. (Besides, the snap might have been
- * created in a short-lived context.)
- */
- if (ActiveSnapshot != NULL)
- stmt_list = pg_plan_queries(querytrees, cursorOptions, NULL, false);
- else
- {
- PG_TRY();
- {
- stmt_list = pg_plan_queries(querytrees, cursorOptions, NULL, true);
- }
- PG_CATCH();
- {
- /* Restore global vars and propagate error */
- ActiveSnapshot = NULL;
- PG_RE_THROW();
- }
- PG_END_TRY();
-
- ActiveSnapshot = NULL;
- }
-
- return stmt_list;
-}
-
-
/*
* ReleaseCachedPlan: release active use of a cached plan.
*
* snapmgr.c
* PostgreSQL snapshot manager
*
+ * We keep track of snapshots in two ways: the "registered snapshots" list,
+ * and the "active snapshot" stack. All snapshots in any of them is supposed
+ * to be in persistent memory. When a snapshot is no longer in any of these
+ * lists (tracked by separate refcounts of each snapshot), its memory can be
+ * freed.
+ *
+ * These arrangements let us reset MyProc->xmin when there are no snapshots
+ * referenced by this transaction. (One possible improvement would be to be
+ * able to advance Xmin when the snapshot with the earliest Xmin is no longer
+ * referenced. That's a bit harder though, it requires more locking, and
+ * anyway it should be rather uncommon to keep snapshots referenced for too
+ * long.)
+ *
+ * Note: parts of this code could probably be replaced by appropriate use
+ * of resowner.c.
+ *
+ *
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/utils/time/snapmgr.c,v 1.1 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/backend/utils/time/snapmgr.c,v 1.2 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
#include "access/xact.h"
#include "access/transam.h"
+#include "storage/proc.h"
#include "storage/procarray.h"
+#include "utils/memutils.h"
#include "utils/snapmgr.h"
#include "utils/tqual.h"
+#include "utils/memutils.h"
/*
+ * CurrentSnapshot points to the only snapshot taken in a serializable
+ * transaction, and to the latest one taken in a read-committed transaction.
+ * SecondarySnapshot is a snapshot that's always up-to-date as of the current
+ * instant, even on a serializable transaction. It should only be used for
+ * special-purpose code (say, RI checking.)
+ *
* These SnapshotData structs are static to simplify memory allocation
* (see the hack in GetSnapshotData to avoid repeated malloc/free).
*/
-static SnapshotData SerializableSnapshotData = {HeapTupleSatisfiesMVCC};
-static SnapshotData LatestSnapshotData = {HeapTupleSatisfiesMVCC};
-
-/* Externally visible pointers to valid snapshots: */
-Snapshot SerializableSnapshot = NULL;
-Snapshot LatestSnapshot = NULL;
+static SnapshotData CurrentSnapshotData = {HeapTupleSatisfiesMVCC};
+static SnapshotData SecondarySnapshotData = {HeapTupleSatisfiesMVCC};
-/*
- * This pointer is not maintained by this module, but it's convenient
- * to declare it here anyway. Callers typically assign a copy of
- * GetTransactionSnapshot's result to ActiveSnapshot.
- */
-Snapshot ActiveSnapshot = NULL;
+/* Pointers to valid snapshots */
+static Snapshot CurrentSnapshot = NULL;
+static Snapshot SecondarySnapshot = NULL;
/*
* These are updated by GetSnapshotData. We initialize them this way
TransactionId RecentXmin = FirstNormalTransactionId;
TransactionId RecentGlobalXmin = FirstNormalTransactionId;
+/*
+ * Elements of the list of registered snapshots.
+ *
+ * Note that we keep refcounts both here and in SnapshotData. This is because
+ * the same snapshot may be registered more than once in a subtransaction, and
+ * if a subxact aborts we want to be able to substract the correct amount of
+ * counts from SnapshotData. (Another approach would be keeping one
+ * RegdSnapshotElt each time a snapshot is registered, but that seems
+ * unnecessary wastage.)
+ *
+ * NB: the code assumes that elements in this list are in non-increasing
+ * order of s_level; also, the list must be NULL-terminated.
+ */
+typedef struct RegdSnapshotElt
+{
+ Snapshot s_snap;
+ uint32 s_count;
+ int s_level;
+ struct RegdSnapshotElt *s_next;
+} RegdSnapshotElt;
+
+/*
+ * Elements of the active snapshot stack.
+ *
+ * It's not necessary to keep a refcount like we do for the registered list;
+ * each element here accounts for exactly one active_count on SnapshotData.
+ * We cannot condense them like we do for RegdSnapshotElt because it would mess
+ * up the order of entries in the stack.
+ *
+ * NB: the code assumes that elements in this list are in non-increasing
+ * order of as_level; also, the list must be NULL-terminated.
+ */
+typedef struct ActiveSnapshotElt
+{
+ Snapshot as_snap;
+ int as_level;
+ struct ActiveSnapshotElt *as_next;
+} ActiveSnapshotElt;
+
+/* Head of the list of registered snapshots */
+static RegdSnapshotElt *RegisteredSnapshotList = NULL;
+
+/* Top of the stack of active snapshots */
+static ActiveSnapshotElt *ActiveSnapshot = NULL;
+
+/* first GetTransactionSnapshot call in a transaction? */
+bool FirstSnapshotSet = false;
+
+/*
+ * Remembers whether this transaction registered a serializable snapshot at
+ * start. We cannot trust FirstSnapshotSet in combination with
+ * IsXactIsoLevelSerializable, because GUC may be reset before us.
+ */
+static bool registered_serializable = false;
+
+
+static Snapshot CopySnapshot(Snapshot snapshot);
+static void FreeSnapshot(Snapshot snapshot);
+static void SnapshotResetXmin(void);
+
/*
* GetTransactionSnapshot
* Get the appropriate snapshot for a new query in a transaction.
*
- * The SerializableSnapshot is the first one taken in a transaction.
- * In serializable mode we just use that one throughout the transaction.
- * In read-committed mode, we take a new snapshot each time we are called.
*
- * Note that the return value points at static storage that will be modified
- * by future calls and by CommandCounterIncrement(). Callers should copy
- * the result with CopySnapshot() if it is to be used very long.
+ * Note that the return value may point at static storage that will be modified
+ * by future calls and by CommandCounterIncrement(). Callers should call
+ * RegisterSnapshot or PushActiveSnapshot on the returned snap if it is to be
+ * used very long.
*/
Snapshot
GetTransactionSnapshot(void)
{
/* First call in transaction? */
- if (SerializableSnapshot == NULL)
+ if (!FirstSnapshotSet)
{
- SerializableSnapshot = GetSnapshotData(&SerializableSnapshotData, true);
- return SerializableSnapshot;
+ CurrentSnapshot = GetSnapshotData(&CurrentSnapshotData);
+ FirstSnapshotSet = true;
+
+ /*
+ * In serializable mode, the first snapshot must live until end of xact
+ * regardless of what the caller does with it, so we must register it
+ * internally here and unregister it at end of xact.
+ */
+ if (IsXactIsoLevelSerializable)
+ {
+ CurrentSnapshot = RegisterSnapshot(CurrentSnapshot);
+ registered_serializable = true;
+ }
+
+ return CurrentSnapshot;
}
if (IsXactIsoLevelSerializable)
- return SerializableSnapshot;
+ return CurrentSnapshot;
- LatestSnapshot = GetSnapshotData(&LatestSnapshotData, false);
+ CurrentSnapshot = GetSnapshotData(&CurrentSnapshotData);
- return LatestSnapshot;
+ return CurrentSnapshot;
}
/*
GetLatestSnapshot(void)
{
/* Should not be first call in transaction */
- if (SerializableSnapshot == NULL)
+ if (!FirstSnapshotSet)
elog(ERROR, "no snapshot has been set");
- LatestSnapshot = GetSnapshotData(&LatestSnapshotData, false);
+ SecondarySnapshot = GetSnapshotData(&SecondarySnapshotData);
- return LatestSnapshot;
+ return SecondarySnapshot;
+}
+
+/*
+ * SnapshotSetCommandId
+ * Propagate CommandCounterIncrement into the static snapshots, if set
+ */
+void
+SnapshotSetCommandId(CommandId curcid)
+{
+ if (!FirstSnapshotSet)
+ return;
+
+ if (CurrentSnapshot)
+ CurrentSnapshot->curcid = curcid;
+ if (SecondarySnapshot)
+ SecondarySnapshot->curcid = curcid;
}
/*
* CopySnapshot
* Copy the given snapshot.
*
- * The copy is palloc'd in the current memory context.
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
*/
-Snapshot
+static Snapshot
CopySnapshot(Snapshot snapshot)
{
Snapshot newsnap;
Size subxipoff;
Size size;
+ Assert(snapshot != InvalidSnapshot);
+
/* We allocate any XID arrays needed in the same palloc block. */
size = subxipoff = sizeof(SnapshotData) +
snapshot->xcnt * sizeof(TransactionId);
if (snapshot->subxcnt > 0)
size += snapshot->subxcnt * sizeof(TransactionId);
- newsnap = (Snapshot) palloc(size);
+ newsnap = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
memcpy(newsnap, snapshot, sizeof(SnapshotData));
+ newsnap->regd_count = 0;
+ newsnap->active_count = 0;
+ newsnap->copied = true;
+
/* setup XID array */
if (snapshot->xcnt > 0)
{
/*
* FreeSnapshot
- * Free a snapshot previously copied with CopySnapshot.
+ * Free the memory associated with a snapshot.
+ */
+static void
+FreeSnapshot(Snapshot snapshot)
+{
+ Assert(snapshot->regd_count == 0);
+ Assert(snapshot->active_count == 0);
+
+ pfree(snapshot);
+}
+
+/*
+ * PushActiveSnapshot
+ * Set the given snapshot as the current active snapshot
*
- * This is currently identical to pfree, but is provided for cleanliness.
+ * If this is the first use of this snapshot, create a new long-lived copy with
+ * active refcount=1. Otherwise, only increment the refcount.
+ */
+void
+PushActiveSnapshot(Snapshot snap)
+{
+ ActiveSnapshotElt *newactive;
+
+ Assert(snap != InvalidSnapshot);
+
+ newactive = MemoryContextAlloc(TopTransactionContext, sizeof(ActiveSnapshotElt));
+ /* Static snapshot? Create a persistent copy */
+ newactive->as_snap = snap->copied ? snap : CopySnapshot(snap);
+ newactive->as_next = ActiveSnapshot;
+ newactive->as_level = GetCurrentTransactionNestLevel();
+
+ newactive->as_snap->active_count++;
+
+ ActiveSnapshot = newactive;
+}
+
+/*
+ * PushUpdatedSnapshot
+ * As above, except we set the snapshot's CID to the current CID.
+ */
+void
+PushUpdatedSnapshot(Snapshot snapshot)
+{
+ Snapshot newsnap;
+
+ /*
+ * We cannot risk modifying a snapshot that's possibly already used
+ * elsewhere, so make a new copy to scribble on.
+ */
+ newsnap = CopySnapshot(snapshot);
+ newsnap->curcid = GetCurrentCommandId(false);
+
+ PushActiveSnapshot(newsnap);
+}
+
+/*
+ * PopActiveSnapshot
*
- * Do *not* apply this to the results of GetTransactionSnapshot or
- * GetLatestSnapshot, since those are just static structs.
+ * Remove the topmost snapshot from the active snapshot stack, decrementing the
+ * reference count, and free it if this was the last reference.
*/
void
-FreeSnapshot(Snapshot snapshot)
+PopActiveSnapshot(void)
{
- pfree(snapshot);
+ ActiveSnapshotElt *newstack;
+
+ newstack = ActiveSnapshot->as_next;
+
+ Assert(ActiveSnapshot->as_snap->active_count > 0);
+
+ ActiveSnapshot->as_snap->active_count--;
+
+ if (ActiveSnapshot->as_snap->active_count == 0 &&
+ ActiveSnapshot->as_snap->regd_count == 0)
+ FreeSnapshot(ActiveSnapshot->as_snap);
+
+ pfree(ActiveSnapshot);
+ ActiveSnapshot = newstack;
+
+ SnapshotResetXmin();
}
/*
- * FreeXactSnapshot
- * Free snapshot(s) at end of transaction.
+ * GetActiveSnapshot
+ * Return the topmost snapshot in the Active stack.
+ */
+Snapshot
+GetActiveSnapshot(void)
+{
+ Assert(ActiveSnapshot != NULL);
+
+ return ActiveSnapshot->as_snap;
+}
+
+/*
+ * ActiveSnapshotSet
+ * Return whether there is at least one snapsho in the Active stack
+ */
+bool
+ActiveSnapshotSet(void)
+{
+ return ActiveSnapshot != NULL;
+}
+
+/*
+ * RegisterSnapshot
+ * Register a snapshot as being in use
+ *
+ * If InvalidSnapshot is passed, it is not registered.
+ */
+Snapshot
+RegisterSnapshot(Snapshot snapshot)
+{
+ RegdSnapshotElt *elt;
+ RegdSnapshotElt *newhead;
+ int level;
+
+ if (snapshot == InvalidSnapshot)
+ return InvalidSnapshot;
+
+ level = GetCurrentTransactionNestLevel();
+
+ /*
+ * If there's already an item in the list for the same snapshot and the
+ * same subxact nest level, increment its refcounts. Otherwise create a
+ * new one.
+ */
+ for (elt = RegisteredSnapshotList; elt != NULL; elt = elt->s_next)
+ {
+ if (elt->s_level < level)
+ break;
+
+ if (elt->s_snap == snapshot && elt->s_level == level)
+ {
+ elt->s_snap->regd_count++;
+ elt->s_count++;
+
+ return elt->s_snap;
+ }
+ }
+
+ /*
+ * Create the new list element. If it's not been copied into persistent
+ * memory already, we must do so; otherwise we can just increment the
+ * reference count.
+ */
+ newhead = MemoryContextAlloc(TopTransactionContext, sizeof(RegdSnapshotElt));
+ newhead->s_next = RegisteredSnapshotList;
+ /* Static snapshot? Create a persistent copy */
+ newhead->s_snap = snapshot->copied ? snapshot : CopySnapshot(snapshot);
+ newhead->s_level = level;
+ newhead->s_count = 1;
+
+ newhead->s_snap->regd_count++;
+
+ RegisteredSnapshotList = newhead;
+
+ return RegisteredSnapshotList->s_snap;
+}
+
+/*
+ * UnregisterSnapshot
+ * Signals that a snapshot is no longer necessary
+ *
+ * If both reference counts fall to zero, the snapshot memory is released.
+ * If only the registered list refcount falls to zero, just the list element is
+ * freed.
+ */
+void
+UnregisterSnapshot(Snapshot snapshot)
+{
+ RegdSnapshotElt *prev = NULL;
+ RegdSnapshotElt *elt;
+ bool found = false;
+
+ if (snapshot == InvalidSnapshot)
+ return;
+
+ for (elt = RegisteredSnapshotList; elt != NULL; elt = elt->s_next)
+ {
+ if (elt->s_snap == snapshot)
+ {
+ Assert(elt->s_snap->regd_count > 0);
+ Assert(elt->s_count > 0);
+
+ elt->s_snap->regd_count--;
+ elt->s_count--;
+ found = true;
+
+ if (elt->s_count == 0)
+ {
+ /* delink it from the registered snapshot list */
+ if (prev)
+ prev->s_next = elt->s_next;
+ else
+ RegisteredSnapshotList = elt->s_next;
+
+ /* free the snapshot itself if it's no longer relevant */
+ if (elt->s_snap->regd_count == 0 && elt->s_snap->active_count == 0)
+ FreeSnapshot(elt->s_snap);
+
+ /* and free the list element */
+ pfree(elt);
+ }
+
+ break;
+ }
+
+ prev = elt;
+ }
+
+ if (!found)
+ elog(WARNING, "unregistering failed for snapshot %p", snapshot);
+
+ SnapshotResetXmin();
+}
+
+/*
+ * SnapshotResetXmin
+ *
+ * If there are no more snapshots, we can reset our PGPROC->xmin to InvalidXid.
+ * Note we can do this without locking because we assume that storing an Xid
+ * is atomic.
+ */
+static void
+SnapshotResetXmin(void)
+{
+ if (RegisteredSnapshotList == NULL && ActiveSnapshot == NULL)
+ MyProc->xmin = InvalidTransactionId;
+}
+
+/*
+ * AtSubCommit_Snapshot
*/
void
-FreeXactSnapshot(void)
+AtSubCommit_Snapshot(int level)
{
+ ActiveSnapshotElt *active;
+ RegdSnapshotElt *regd;
+
/*
- * We do not free the xip arrays for the static snapshot structs; they
- * will be reused soon. So this is now just a state change to prevent
- * outside callers from accessing the snapshots.
+ * Relabel the active snapshots set in this subtransaction as though they
+ * are owned by the parent subxact.
*/
- SerializableSnapshot = NULL;
- LatestSnapshot = NULL;
- ActiveSnapshot = NULL; /* just for cleanliness */
+ for (active = ActiveSnapshot; active != NULL; active = active->as_next)
+ {
+ if (active->as_level < level)
+ break;
+ active->as_level = level - 1;
+ }
+
+ /*
+ * Reassign all registered snapshots to the parent subxact.
+ *
+ * Note: this code is somewhat bogus in that we could end up with multiple
+ * entries for the same snapshot and the same subxact level (my parent's
+ * level). Cleaning that up is more trouble than it's currently worth,
+ * however.
+ */
+ for (regd = RegisteredSnapshotList; regd != NULL; regd = regd->s_next)
+ {
+ if (regd->s_level == level)
+ regd->s_level--;
+ }
+}
+
+/*
+ * AtSubAbort_Snapshot
+ * Clean up snapshots after a subtransaction abort
+ */
+void
+AtSubAbort_Snapshot(int level)
+{
+ RegdSnapshotElt *prev;
+ RegdSnapshotElt *regd;
+
+ /* Forget the active snapshots set by this subtransaction */
+ while (ActiveSnapshot && ActiveSnapshot->as_level >= level)
+ {
+ ActiveSnapshotElt *next;
+
+ next = ActiveSnapshot->as_next;
+
+ /*
+ * Decrement the snapshot's active count. If it's still registered or
+ * marked as active by an outer subtransaction, we can't free it yet.
+ */
+ Assert(ActiveSnapshot->as_snap->active_count >= 1);
+ ActiveSnapshot->as_snap->active_count -= 1;
+
+ if (ActiveSnapshot->as_snap->active_count == 0 &&
+ ActiveSnapshot->as_snap->regd_count == 0)
+ FreeSnapshot(ActiveSnapshot->as_snap);
+
+ /* and free the stack element */
+ pfree(ActiveSnapshot);
+
+ ActiveSnapshot = next;
+ }
+
+ /* Unregister all snapshots registered during this subtransaction */
+ prev = NULL;
+ for (regd = RegisteredSnapshotList; regd != NULL; )
+ {
+ if (regd->s_level >= level)
+ {
+ RegdSnapshotElt *tofree;
+
+ if (prev)
+ prev->s_next = regd->s_next;
+ else
+ RegisteredSnapshotList = regd->s_next;
+
+ tofree = regd;
+ regd = regd->s_next;
+
+ tofree->s_snap->regd_count -= tofree->s_count;
+
+ /* free the snapshot if possible */
+ if (tofree->s_snap->regd_count == 0 &&
+ tofree->s_snap->active_count == 0)
+ FreeSnapshot(tofree->s_snap);
+
+ /* and free the list element */
+ pfree(tofree);
+ }
+ else
+ {
+ prev = regd;
+ regd = regd->s_next;
+ }
+ }
+
+ SnapshotResetXmin();
+}
+
+/*
+ * AtEOXact_Snapshot
+ * Snapshot manager's cleanup function for end of transaction
+ */
+void
+AtEOXact_Snapshot(bool isCommit)
+{
+ /* On commit, complain about leftover snapshots */
+ if (isCommit)
+ {
+ ActiveSnapshotElt *active;
+ RegdSnapshotElt *regd;
+
+ /*
+ * On a serializable snapshot we must first unregister our private
+ * refcount to the serializable snapshot.
+ */
+ if (registered_serializable)
+ UnregisterSnapshot(CurrentSnapshot);
+
+ /* complain about unpopped active snapshots */
+ for (active = ActiveSnapshot; active != NULL; active = active->as_next)
+ {
+ ereport(WARNING,
+ (errmsg("snapshot %p still active", active)));
+ }
+
+ /* complain about any unregistered snapshot */
+ for (regd = RegisteredSnapshotList; regd != NULL; regd = regd->s_next)
+ {
+ ereport(WARNING,
+ (errmsg("snapshot %p not destroyed at commit (%d regd refs, %d active refs)",
+ regd->s_snap, regd->s_snap->regd_count,
+ regd->s_snap->active_count)));
+ }
+ }
+
+ /*
+ * And reset our state. We don't need to free the memory explicitely --
+ * it'll go away with TopTransactionContext.
+ */
+ ActiveSnapshot = NULL;
+ RegisteredSnapshotList = NULL;
+
+ CurrentSnapshot = NULL;
+ SecondarySnapshot = NULL;
+
+ FirstSnapshotSet = false;
+ registered_serializable = false;
}
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/storage/procarray.h,v 1.21 2008/03/26 16:20:48 alvherre Exp $
+ * $PostgreSQL: pgsql/src/include/storage/procarray.h,v 1.22 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
extern void ProcArrayEndTransaction(PGPROC *proc, TransactionId latestXid);
extern void ProcArrayClearTransaction(PGPROC *proc);
-extern Snapshot GetSnapshotData(Snapshot snapshot, bool serializable);
+extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/snapmgr.h,v 1.1 2008/03/26 18:48:59 alvherre Exp $
+ * $PostgreSQL: pgsql/src/include/utils/snapmgr.h,v 1.2 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
#include "utils/snapshot.h"
-extern PGDLLIMPORT Snapshot SerializableSnapshot;
-extern PGDLLIMPORT Snapshot LatestSnapshot;
-extern PGDLLIMPORT Snapshot ActiveSnapshot;
+extern bool FirstSnapshotSet;
extern TransactionId TransactionXmin;
extern TransactionId RecentXmin;
extern Snapshot GetTransactionSnapshot(void);
extern Snapshot GetLatestSnapshot(void);
-extern Snapshot CopySnapshot(Snapshot snapshot);
-extern void FreeSnapshot(Snapshot snapshot);
-extern void FreeXactSnapshot(void);
+extern void SnapshotSetCommandId(CommandId curcid);
+
+extern void PushActiveSnapshot(Snapshot snapshot);
+extern void PushUpdatedSnapshot(Snapshot snapshot);
+extern void PopActiveSnapshot(void);
+extern Snapshot GetActiveSnapshot(void);
+extern bool ActiveSnapshotSet(void);
+
+extern Snapshot RegisterSnapshot(Snapshot snapshot);
+extern void UnregisterSnapshot(Snapshot snapshot);
+
+extern void AtSubCommit_Snapshot(int level);
+extern void AtSubAbort_Snapshot(int level);
+extern void AtEOXact_Snapshot(bool isCommit);
#endif /* SNAPMGR_H */
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $PostgreSQL: pgsql/src/include/utils/snapshot.h,v 1.2 2008/03/26 21:10:39 alvherre Exp $
+ * $PostgreSQL: pgsql/src/include/utils/snapshot.h,v 1.3 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
* out any that are >= xmax
*/
CommandId curcid; /* in my xact, CID < curcid are visible */
+ uint32 active_count; /* refcount on ActiveSnapshot stack */
+ uint32 regd_count; /* refcount on RegisteredSnapshotList */
+ bool copied; /* false if it's a static snapshot */
} SnapshotData;
/*
*
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.212 2008/05/12 00:00:54 alvherre Exp $
+ * $PostgreSQL: pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.213 2008/05/12 20:02:02 alvherre Exp $
*
*-------------------------------------------------------------------------
*/
CachedPlan *cplan;
ParamListInfo paramLI;
int i;
- Snapshot saveActiveSnapshot;
+ MemoryContext oldcontext;
/*
* Forget it if expression wasn't simple before.
* updates made so far by our own function.
*/
SPI_push();
- saveActiveSnapshot = ActiveSnapshot;
- PG_TRY();
+ oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
+ if (!estate->readonly_func)
{
- MemoryContext oldcontext;
+ CommandCounterIncrement();
+ PushActiveSnapshot(GetTransactionSnapshot());
+ }
- oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
- if (!estate->readonly_func)
- {
- CommandCounterIncrement();
- ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
- }
+ /*
+ * Finally we can call the executor to evaluate the expression
+ */
+ *result = ExecEvalExpr(expr->expr_simple_state,
+ econtext,
+ isNull,
+ NULL);
+ MemoryContextSwitchTo(oldcontext);
- /*
- * Finally we can call the executor to evaluate the expression
- */
- *result = ExecEvalExpr(expr->expr_simple_state,
- econtext,
- isNull,
- NULL);
- MemoryContextSwitchTo(oldcontext);
- }
- PG_CATCH();
- {
- /* Restore global vars and propagate error */
- ActiveSnapshot = saveActiveSnapshot;
- PG_RE_THROW();
- }
- PG_END_TRY();
+ if (!estate->readonly_func)
+ PopActiveSnapshot();
- ActiveSnapshot = saveActiveSnapshot;
SPI_pop();
/*