/* Saved hook values in case of unload */
static ExecutorStart_hook_type prev_ExecutorStart = NULL;
static ExecutorRun_hook_type prev_ExecutorRun = NULL;
+static ExecutorFinish_hook_type prev_ExecutorFinish = NULL;
static ExecutorEnd_hook_type prev_ExecutorEnd = NULL;
#define auto_explain_enabled() \
static void explain_ExecutorRun(QueryDesc *queryDesc,
ScanDirection direction,
long count);
+static void explain_ExecutorFinish(QueryDesc *queryDesc);
static void explain_ExecutorEnd(QueryDesc *queryDesc);
ExecutorStart_hook = explain_ExecutorStart;
prev_ExecutorRun = ExecutorRun_hook;
ExecutorRun_hook = explain_ExecutorRun;
+ prev_ExecutorFinish = ExecutorFinish_hook;
+ ExecutorFinish_hook = explain_ExecutorFinish;
prev_ExecutorEnd = ExecutorEnd_hook;
ExecutorEnd_hook = explain_ExecutorEnd;
}
/* Uninstall hooks. */
ExecutorStart_hook = prev_ExecutorStart;
ExecutorRun_hook = prev_ExecutorRun;
+ ExecutorFinish_hook = prev_ExecutorFinish;
ExecutorEnd_hook = prev_ExecutorEnd;
}
PG_END_TRY();
}
+/*
+ * ExecutorFinish hook: all we need do is track nesting depth
+ */
+static void
+explain_ExecutorFinish(QueryDesc *queryDesc)
+{
+ nesting_level++;
+ PG_TRY();
+ {
+ if (prev_ExecutorFinish)
+ prev_ExecutorFinish(queryDesc);
+ else
+ standard_ExecutorFinish(queryDesc);
+ nesting_level--;
+ }
+ PG_CATCH();
+ {
+ nesting_level--;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+}
+
/*
* ExecutorEnd hook: log results if needed
*/
static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
static ExecutorStart_hook_type prev_ExecutorStart = NULL;
static ExecutorRun_hook_type prev_ExecutorRun = NULL;
+static ExecutorFinish_hook_type prev_ExecutorFinish = NULL;
static ExecutorEnd_hook_type prev_ExecutorEnd = NULL;
static ProcessUtility_hook_type prev_ProcessUtility = NULL;
static void pgss_ExecutorRun(QueryDesc *queryDesc,
ScanDirection direction,
long count);
+static void pgss_ExecutorFinish(QueryDesc *queryDesc);
static void pgss_ExecutorEnd(QueryDesc *queryDesc);
static void pgss_ProcessUtility(Node *parsetree,
const char *queryString, ParamListInfo params, bool isTopLevel,
ExecutorStart_hook = pgss_ExecutorStart;
prev_ExecutorRun = ExecutorRun_hook;
ExecutorRun_hook = pgss_ExecutorRun;
+ prev_ExecutorFinish = ExecutorFinish_hook;
+ ExecutorFinish_hook = pgss_ExecutorFinish;
prev_ExecutorEnd = ExecutorEnd_hook;
ExecutorEnd_hook = pgss_ExecutorEnd;
prev_ProcessUtility = ProcessUtility_hook;
shmem_startup_hook = prev_shmem_startup_hook;
ExecutorStart_hook = prev_ExecutorStart;
ExecutorRun_hook = prev_ExecutorRun;
+ ExecutorFinish_hook = prev_ExecutorFinish;
ExecutorEnd_hook = prev_ExecutorEnd;
ProcessUtility_hook = prev_ProcessUtility;
}
PG_END_TRY();
}
+/*
+ * ExecutorFinish hook: all we need do is track nesting depth
+ */
+static void
+pgss_ExecutorFinish(QueryDesc *queryDesc)
+{
+ nested_level++;
+ PG_TRY();
+ {
+ if (prev_ExecutorFinish)
+ prev_ExecutorFinish(queryDesc);
+ else
+ standard_ExecutorFinish(queryDesc);
+ nested_level--;
+ }
+ PG_CATCH();
+ {
+ nested_level--;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+}
+
/*
* ExecutorEnd hook: store results if needed
*/
Assert(s->parent == NULL);
/*
- * Do pre-commit processing (most of this stuff requires database access,
- * and in fact could still cause an error...)
- *
- * It is possible for CommitHoldablePortals to invoke functions that queue
- * deferred triggers, and it's also possible that triggers create holdable
- * cursors. So we have to loop until there's nothing left to do.
+ * Do pre-commit processing that involves calling user-defined code, such
+ * as triggers. Since closing cursors could queue trigger actions,
+ * triggers could open cursors, etc, we have to keep looping until there's
+ * nothing left to do.
*/
for (;;)
{
AfterTriggerFireDeferred();
/*
- * Convert any open holdable cursors into static portals. If there
- * weren't any, we are done ... otherwise loop back to check if they
- * queued deferred triggers. Lather, rinse, repeat.
+ * Close open portals (converting holdable ones into static portals).
+ * If there weren't any, we are done ... otherwise loop back to check
+ * if they queued deferred triggers. Lather, rinse, repeat.
*/
- if (!CommitHoldablePortals())
+ if (!PreCommit_Portals(false))
break;
}
- /* Now we can shut down the deferred-trigger manager */
- AfterTriggerEndXact(true);
+ /*
+ * The remaining actions cannot call any user-defined code, so it's
+ * safe to start shutting down within-transaction services. But note
+ * that most of this stuff could still throw an error, which would
+ * switch us into the transaction-abort path.
+ */
- /* Close any open regular cursors */
- AtCommit_Portals();
+ /* Shut down the deferred-trigger manager */
+ AfterTriggerEndXact(true);
/*
* Let ON COMMIT management do its thing (must happen after closing
Assert(s->parent == NULL);
/*
- * Do pre-commit processing (most of this stuff requires database access,
- * and in fact could still cause an error...)
- *
- * It is possible for PrepareHoldablePortals to invoke functions that
- * queue deferred triggers, and it's also possible that triggers create
- * holdable cursors. So we have to loop until there's nothing left to do.
+ * Do pre-commit processing that involves calling user-defined code, such
+ * as triggers. Since closing cursors could queue trigger actions,
+ * triggers could open cursors, etc, we have to keep looping until there's
+ * nothing left to do.
*/
for (;;)
{
AfterTriggerFireDeferred();
/*
- * Convert any open holdable cursors into static portals. If there
- * weren't any, we are done ... otherwise loop back to check if they
- * queued deferred triggers. Lather, rinse, repeat.
+ * Close open portals (converting holdable ones into static portals).
+ * If there weren't any, we are done ... otherwise loop back to check
+ * if they queued deferred triggers. Lather, rinse, repeat.
*/
- if (!PrepareHoldablePortals())
+ if (!PreCommit_Portals(true))
break;
}
- /* Now we can shut down the deferred-trigger manager */
- AfterTriggerEndXact(true);
+ /*
+ * The remaining actions cannot call any user-defined code, so it's
+ * safe to start shutting down within-transaction services. But note
+ * that most of this stuff could still throw an error, which would
+ * switch us into the transaction-abort path.
+ */
- /* Close any open regular cursors */
- AtCommit_Portals();
+ /* Shut down the deferred-trigger manager */
+ AfterTriggerEndXact(true);
/*
* Let ON COMMIT management do its thing (must happen after closing
if (cstate->queryDesc != NULL)
{
/* Close down the query and free resources. */
+ ExecutorFinish(cstate->queryDesc);
ExecutorEnd(cstate->queryDesc);
FreeQueryDesc(cstate->queryDesc);
PopActiveSnapshot();
*/
PreventTransactionChain(isTopLevel, "DISCARD ALL");
+ /* Closing portals might run user-defined code, so do that first. */
+ PortalHashTableDeleteAll();
SetPGVariable("session_authorization", NIL, false);
ResetAllOptions();
DropAllPreparedStatements();
- PortalHashTableDeleteAll();
Async_UnlistenAll();
LockReleaseAll(USER_LOCKMETHOD, true);
ResetPlanCache();
if (es->buffers)
instrument_option |= INSTRUMENT_BUFFERS;
+ INSTR_TIME_SET_CURRENT(starttime);
+
/*
* Use a snapshot with an updated command ID to ensure this query sees
* results of any previously executed queries.
GetActiveSnapshot(), InvalidSnapshot,
None_Receiver, params, instrument_option);
- INSTR_TIME_SET_CURRENT(starttime);
-
- /* If analyzing, we need to cope with queued triggers */
- if (es->analyze)
- AfterTriggerBeginQuery();
-
/* Select execution options */
if (es->analyze)
eflags = 0; /* default run-to-completion flags */
/* run the plan */
ExecutorRun(queryDesc, ForwardScanDirection, 0L);
- /* We can't clean up 'till we're done printing the stats... */
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* We can't run ExecutorEnd 'till we're done printing the stats... */
totaltime += elapsed_time(&starttime);
}
/* Create textual dump of plan tree */
ExplainPrintPlan(es, queryDesc);
- /*
- * If we ran the command, run any AFTER triggers it queued. (Note this
- * will not include DEFERRED triggers; since those don't run until end of
- * transaction, we can't measure them.) Include into total runtime.
- */
- if (es->analyze)
- {
- INSTR_TIME_SET_CURRENT(starttime);
- AfterTriggerEndQuery(queryDesc->estate);
- totaltime += elapsed_time(&starttime);
- }
-
/* Print info about runtime of triggers */
if (es->analyze)
{
GetActiveSnapshot(), NULL,
dest, NULL, 0);
- AfterTriggerBeginQuery();
ExecutorStart(qdesc, 0);
ExecutorRun(qdesc, ForwardScanDirection, 0);
- AfterTriggerEndQuery(qdesc->estate);
+ ExecutorFinish(qdesc);
ExecutorEnd(qdesc);
FreeQueryDesc(qdesc);
queryDesc = PortalGetQueryDesc(portal);
if (queryDesc)
{
+ /*
+ * Reset the queryDesc before anything else. This prevents us
+ * from trying to shut down the executor twice, in case of an
+ * error below. The transaction abort mechanisms will take care
+ * of resource cleanup in such a case.
+ */
portal->queryDesc = NULL;
+
if (portal->status != PORTAL_FAILED)
{
ResourceOwner saveResourceOwner;
PG_TRY();
{
CurrentResourceOwner = portal->resowner;
- /* we do not need AfterTriggerEndQuery() here */
+ ExecutorFinish(queryDesc);
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
}
* Now shut down the inner executor.
*/
portal->queryDesc = NULL; /* prevent double shutdown */
- /* we do not need AfterTriggerEndQuery() here */
+ ExecutorFinish(queryDesc);
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
* we invoke all AFTER IMMEDIATE trigger events queued by the query, and
* transfer deferred trigger events to the global deferred-trigger list.
*
- * Note that this should be called just BEFORE closing down the executor
+ * Note that this must be called BEFORE closing down the executor
* with ExecutorEnd, because we make use of the EState's info about
- * target relations.
+ * target relations. Normally it is called from ExecutorFinish.
* ----------
*/
void
CreateExprContext
creates per-tuple context
ExecInitExpr
+ AfterTriggerBeginQuery
ExecutorRun
ExecProcNode --- recursively called in per-query context
ExecEvalExpr --- called in per-tuple context
ResetExprContext --- to free memory
+ ExecutorFinish
+ ExecPostprocessPlan --- run any unfinished ModifyTable nodes
+ AfterTriggerEndQuery
+
ExecutorEnd
ExecEndNode --- recursively releases resources
FreeExecutorState
* INTERFACE ROUTINES
* ExecutorStart()
* ExecutorRun()
+ * ExecutorFinish()
* ExecutorEnd()
*
- * The old ExecutorMain() has been replaced by ExecutorStart(),
- * ExecutorRun() and ExecutorEnd()
- *
- * These three procedures are the external interfaces to the executor.
+ * These four procedures are the external interface to the executor.
* In each case, the query descriptor is required as an argument.
*
- * ExecutorStart() must be called at the beginning of execution of any
- * query plan and ExecutorEnd() should always be called at the end of
- * execution of a plan.
+ * ExecutorStart must be called at the beginning of execution of any
+ * query plan and ExecutorEnd must always be called at the end of
+ * execution of a plan (unless it is aborted due to error).
*
* ExecutorRun accepts direction and count arguments that specify whether
* the plan is to be executed forwards, backwards, and for how many tuples.
+ * In some cases ExecutorRun may be called multiple times to process all
+ * the tuples for a plan. It is also acceptable to stop short of executing
+ * the whole plan (but only if it is a SELECT).
+ *
+ * ExecutorFinish must be called after the final ExecutorRun call and
+ * before ExecutorEnd. This can be omitted only in case of EXPLAIN,
+ * which should also omit ExecutorRun.
*
* Portions Copyright (c) 1996-2011, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
#include "utils/tqual.h"
-/* Hooks for plugins to get control in ExecutorStart/Run/End() */
+/* Hooks for plugins to get control in ExecutorStart/Run/Finish/End */
ExecutorStart_hook_type ExecutorStart_hook = NULL;
ExecutorRun_hook_type ExecutorRun_hook = NULL;
+ExecutorFinish_hook_type ExecutorFinish_hook = NULL;
ExecutorEnd_hook_type ExecutorEnd_hook = NULL;
/* Hook for plugin to get control in ExecCheckRTPerms() */
* This routine must be called at the beginning of any execution of any
* query plan
*
- * Takes a QueryDesc previously created by CreateQueryDesc (it's not real
- * clear why we bother to separate the two functions, but...). The tupDesc
+ * Takes a QueryDesc previously created by CreateQueryDesc (which is separate
+ * only because some places use QueryDescs for utility commands). The tupDesc
* field of the QueryDesc is filled in to describe the tuples that will be
* returned, and the internal fields (estate and planstate) are set up.
*
queryDesc->plannedstmt->rowMarks != NIL ||
queryDesc->plannedstmt->hasModifyingCTE)
estate->es_output_cid = GetCurrentCommandId(true);
+
+ /*
+ * A SELECT without modifying CTEs can't possibly queue triggers,
+ * so force skip-triggers mode. This is just a marginal efficiency
+ * hack, since AfterTriggerBeginQuery/AfterTriggerEndQuery aren't
+ * all that expensive, but we might as well do it.
+ */
+ if (!queryDesc->plannedstmt->hasModifyingCTE)
+ eflags |= EXEC_FLAG_SKIP_TRIGGERS;
break;
case CMD_INSERT:
*/
estate->es_snapshot = RegisterSnapshot(queryDesc->snapshot);
estate->es_crosscheck_snapshot = RegisterSnapshot(queryDesc->crosscheck_snapshot);
+ estate->es_top_eflags = eflags;
estate->es_instrument = queryDesc->instrument_options;
/*
*/
InitPlan(queryDesc, eflags);
+ /*
+ * Set up an AFTER-trigger statement context, unless told not to, or
+ * unless it's EXPLAIN-only mode (when ExecutorFinish won't be called).
+ */
+ if (!(eflags & (EXEC_FLAG_SKIP_TRIGGERS | EXEC_FLAG_EXPLAIN_ONLY)))
+ AfterTriggerBeginQuery();
+
MemoryContextSwitchTo(oldcontext);
}
estate = queryDesc->estate;
Assert(estate != NULL);
+ Assert(!(estate->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY));
/*
* Switch into per-query memory context
*/
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
- /* Allow instrumentation of ExecutorRun overall runtime */
+ /* Allow instrumentation of Executor overall runtime */
if (queryDesc->totaltime)
InstrStartNode(queryDesc->totaltime);
MemoryContextSwitchTo(oldcontext);
}
+/* ----------------------------------------------------------------
+ * ExecutorFinish
+ *
+ * This routine must be called after the last ExecutorRun call.
+ * It performs cleanup such as firing AFTER triggers. It is
+ * separate from ExecutorEnd because EXPLAIN ANALYZE needs to
+ * include these actions in the total runtime.
+ *
+ * We provide a function hook variable that lets loadable plugins
+ * get control when ExecutorFinish is called. Such a plugin would
+ * normally call standard_ExecutorFinish().
+ *
+ * ----------------------------------------------------------------
+ */
+void
+ExecutorFinish(QueryDesc *queryDesc)
+{
+ if (ExecutorFinish_hook)
+ (*ExecutorFinish_hook) (queryDesc);
+ else
+ standard_ExecutorFinish(queryDesc);
+}
+
+void
+standard_ExecutorFinish(QueryDesc *queryDesc)
+{
+ EState *estate;
+ MemoryContext oldcontext;
+
+ /* sanity checks */
+ Assert(queryDesc != NULL);
+
+ estate = queryDesc->estate;
+
+ Assert(estate != NULL);
+ Assert(!(estate->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY));
+
+ /* This should be run once and only once per Executor instance */
+ Assert(!estate->es_finished);
+
+ /* Switch into per-query memory context */
+ oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
+
+ /* Allow instrumentation of Executor overall runtime */
+ if (queryDesc->totaltime)
+ InstrStartNode(queryDesc->totaltime);
+
+ /* Run ModifyTable nodes to completion */
+ ExecPostprocessPlan(estate);
+
+ /* Execute queued AFTER triggers, unless told not to */
+ if (!(estate->es_top_eflags & EXEC_FLAG_SKIP_TRIGGERS))
+ AfterTriggerEndQuery(estate);
+
+ if (queryDesc->totaltime)
+ InstrStopNode(queryDesc->totaltime, 0);
+
+ MemoryContextSwitchTo(oldcontext);
+
+ estate->es_finished = true;
+}
+
/* ----------------------------------------------------------------
* ExecutorEnd
*
*
* We provide a function hook variable that lets loadable plugins
* get control when ExecutorEnd is called. Such a plugin would
- * normally call standard_ExecutorEnd(). Because such hooks expect
- * to execute after all plan execution is done, we run
- * ExecPostprocessPlan before invoking the hook.
+ * normally call standard_ExecutorEnd().
*
* ----------------------------------------------------------------
*/
void
ExecutorEnd(QueryDesc *queryDesc)
{
- /* Let plan nodes do any final processing required */
- ExecPostprocessPlan(queryDesc->estate);
-
- /* Now close down */
if (ExecutorEnd_hook)
(*ExecutorEnd_hook) (queryDesc);
else
Assert(estate != NULL);
+ /*
+ * Check that ExecutorFinish was called, unless in EXPLAIN-only mode.
+ * This Assert is needed because ExecutorFinish is new as of 9.1, and
+ * callers might forget to call it.
+ */
+ Assert(estate->es_finished ||
+ (estate->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY));
+
/*
* Switch into per-query memory context to run ExecEndPlan
*/
static void
ExecPostprocessPlan(EState *estate)
{
- MemoryContext oldcontext;
ListCell *lc;
- /*
- * Switch into per-query memory context
- */
- oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
-
/*
* Make sure nodes run forward.
*/
break;
}
}
-
- MemoryContextSwitchTo(oldcontext);
}
/* ----------------------------------------------------------------
estate->es_result_relation_info = parentestate->es_result_relation_info;
/* es_trig_target_relations must NOT be copied */
estate->es_rowMarks = parentestate->es_rowMarks;
+ estate->es_top_eflags = parentestate->es_top_eflags;
estate->es_instrument = parentestate->es_instrument;
estate->es_select_into = parentestate->es_select_into;
estate->es_into_oids = parentestate->es_into_oids;
- estate->es_auxmodifytables = NIL;
+ /* es_auxmodifytables must NOT be copied */
/*
* The external param list is simply shared from parent. The internal
estate->es_processed = 0;
estate->es_lastoid = InvalidOid;
- estate->es_instrument = false;
+ estate->es_top_eflags = 0;
+ estate->es_instrument = 0;
estate->es_select_into = false;
estate->es_into_oids = false;
+ estate->es_finished = false;
estate->es_exprcontexts = NIL;
if (ps->commandType == CMD_SELECT &&
ps->utilityStmt == NULL &&
- ps->intoClause == NULL)
+ ps->intoClause == NULL &&
+ !ps->hasModifyingCTE)
fcache->lazyEval = lasttages->lazyEval = true;
}
}
if (es->qd->utilitystmt == NULL)
{
/*
- * Only set up to collect queued triggers if it's not a SELECT. This
- * isn't just an optimization, but is necessary in case a SELECT
- * returns multiple rows to caller --- we mustn't exit from the
- * function execution with a stacked AfterTrigger level still active.
+ * In lazyEval mode, do not let the executor set up an AfterTrigger
+ * context. This is necessary not just an optimization, because we
+ * mustn't exit from the function execution with a stacked
+ * AfterTrigger level still active. We are careful not to select
+ * lazyEval mode for any statement that could possibly queue triggers.
*/
- if (es->qd->operation != CMD_SELECT)
- AfterTriggerBeginQuery();
- ExecutorStart(es->qd, 0);
+ int eflags;
+
+ if (es->lazyEval)
+ eflags = EXEC_FLAG_SKIP_TRIGGERS;
+ else
+ eflags = 0; /* default run-to-completion flags */
+ ExecutorStart(es->qd, eflags);
}
es->status = F_EXEC_RUN;
/* Make our snapshot the active one for any called functions */
PushActiveSnapshot(es->qd->snapshot);
- if (es->qd->operation != CMD_SELECT)
- AfterTriggerEndQuery(es->qd->estate);
+ ExecutorFinish(es->qd);
ExecutorEnd(es->qd);
PopActiveSnapshot();
_SPI_pquery(QueryDesc *queryDesc, bool fire_triggers, long tcount)
{
int operation = queryDesc->operation;
+ int eflags;
int res;
switch (operation)
ResetUsage();
#endif
+ /* Select execution options */
if (fire_triggers)
- AfterTriggerBeginQuery();
+ eflags = 0; /* default run-to-completion flags */
+ else
+ eflags = EXEC_FLAG_SKIP_TRIGGERS;
- ExecutorStart(queryDesc, 0);
+ ExecutorStart(queryDesc, eflags);
ExecutorRun(queryDesc, ForwardScanDirection, tcount);
elog(ERROR, "consistency check on SPI tuple count failed");
}
- /* Take care of any queued AFTER triggers */
- if (fire_triggers)
- AfterTriggerEndQuery(queryDesc->estate);
-
+ ExecutorFinish(queryDesc);
ExecutorEnd(queryDesc);
/* FreeQueryDesc is done by the caller */
GetActiveSnapshot(), InvalidSnapshot,
dest, params, 0);
- /*
- * Set up to collect AFTER triggers
- */
- AfterTriggerBeginQuery();
-
/*
* Call ExecutorStart to prepare the plan for execution
*/
}
}
- /* Now take care of any queued AFTER triggers */
- AfterTriggerEndQuery(queryDesc->estate);
-
- PopActiveSnapshot();
-
/*
* Now, we close down all the scans and free allocated resources.
*/
+ ExecutorFinish(queryDesc);
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
+
+ PopActiveSnapshot();
}
/*
params,
0);
- /*
- * We do *not* call AfterTriggerBeginQuery() here. We assume
- * that a SELECT cannot queue any triggers. It would be messy
- * to support triggers since the execution of the portal may
- * be interleaved with other queries.
- */
-
/*
* If it's a scrollable cursor, executor needs to support
* REWIND and backwards scan.
errmsg("cannot drop active portal \"%s\"", portal->name)));
/*
- * Remove portal from hash table. Because we do this first, we will not
+ * Allow portalcmds.c to clean up the state it knows about, in particular
+ * shutting down the executor if still active. This step potentially runs
+ * user-defined code so failure has to be expected. It's the cleanup
+ * hook's responsibility to not try to do that more than once, in the case
+ * that failure occurs and then we come back to drop the portal again
+ * during transaction abort.
+ */
+ if (PointerIsValid(portal->cleanup))
+ {
+ (*portal->cleanup) (portal);
+ portal->cleanup = NULL;
+ }
+
+ /*
+ * Remove portal from hash table. Because we do this here, we will not
* come back to try to remove the portal again if there's any error in the
* subsequent steps. Better to leak a little memory than to get into an
* infinite error-recovery loop.
*/
PortalHashTableDelete(portal);
- /* let portalcmds.c clean up the state it knows about */
- if (PointerIsValid(portal->cleanup))
- (*portal->cleanup) (portal);
-
/* drop cached plan reference, if any */
PortalReleaseCachedPlan(portal);
{
Portal portal = hentry->portal;
- if (portal->status != PORTAL_ACTIVE)
- PortalDrop(portal, false);
+ /* Can't close the active portal (the one running the command) */
+ if (portal->status == PORTAL_ACTIVE)
+ continue;
+
+ PortalDrop(portal, false);
+
+ /* Restart the iteration in case that led to other drops */
+ hash_seq_term(&status);
+ hash_seq_init(&status, PortalHashTable);
}
}
/*
* Pre-commit processing for portals.
*
- * Any holdable cursors created in this transaction need to be converted to
+ * Holdable cursors created in this transaction need to be converted to
* materialized form, since we are going to close down the executor and
- * release locks. Other portals are not touched yet.
+ * release locks. Non-holdable portals created in this transaction are
+ * simply removed. Portals remaining from prior transactions should be
+ * left untouched.
*
- * Returns TRUE if any holdable cursors were processed, FALSE if not.
+ * Returns TRUE if any portals changed state (possibly causing user-defined
+ * code to be run), FALSE if not.
*/
bool
-CommitHoldablePortals(void)
+PreCommit_Portals(bool isPrepare)
{
bool result = false;
HASH_SEQ_STATUS status;
{
Portal portal = hentry->portal;
+ /*
+ * There should be no pinned portals anymore. Complain if someone
+ * leaked one.
+ */
+ if (portal->portalPinned)
+ elog(ERROR, "cannot commit while a portal is pinned");
+
+ /*
+ * Do not touch active portals --- this can only happen in the case of
+ * a multi-transaction utility command, such as VACUUM.
+ *
+ * Note however that any resource owner attached to such a portal is
+ * still going to go away, so don't leave a dangling pointer.
+ */
+ if (portal->status == PORTAL_ACTIVE)
+ {
+ portal->resowner = NULL;
+ continue;
+ }
+
/* Is it a holdable portal created in the current xact? */
if ((portal->cursorOptions & CURSOR_OPT_HOLD) &&
portal->createSubid != InvalidSubTransactionId &&
* Instead of dropping the portal, prepare it for access by later
* transactions.
*
+ * However, if this is PREPARE TRANSACTION rather than COMMIT,
+ * refuse PREPARE, because the semantics seem pretty unclear.
+ */
+ if (isPrepare)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot PREPARE a transaction that has created a cursor WITH HOLD")));
+
+ /*
* Note that PersistHoldablePortal() must release all resources
* used by the portal that are local to the creating transaction.
*/
*/
portal->createSubid = InvalidSubTransactionId;
+ /* Report we changed state */
result = true;
}
- }
-
- return result;
-}
-
-/*
- * Pre-prepare processing for portals.
- *
- * Currently we refuse PREPARE if the transaction created any holdable
- * cursors, since it's quite unclear what to do with one. However, this
- * has the same API as CommitHoldablePortals and is invoked in the same
- * way by xact.c, so that we can easily do something reasonable if anyone
- * comes up with something reasonable to do.
- *
- * Returns TRUE if any holdable cursors were processed, FALSE if not.
- */
-bool
-PrepareHoldablePortals(void)
-{
- bool result = false;
- HASH_SEQ_STATUS status;
- PortalHashEnt *hentry;
-
- hash_seq_init(&status, PortalHashTable);
-
- while ((hentry = (PortalHashEnt *) hash_seq_search(&status)) != NULL)
- {
- Portal portal = hentry->portal;
-
- /* Is it a holdable portal created in the current xact? */
- if ((portal->cursorOptions & CURSOR_OPT_HOLD) &&
- portal->createSubid != InvalidSubTransactionId &&
- portal->status == PORTAL_READY)
+ else if (portal->createSubid == InvalidSubTransactionId)
{
/*
- * We are exiting the transaction that created a holdable cursor.
- * Can't do PREPARE.
+ * Do nothing to cursors held over from a previous transaction
+ * (including ones we just froze in a previous cycle of this loop)
*/
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- errmsg("cannot PREPARE a transaction that has created a cursor WITH HOLD")));
- }
- }
-
- return result;
-}
-
-/*
- * Pre-commit processing for portals.
- *
- * Remove all non-holdable portals created in this transaction.
- * Portals remaining from prior transactions should be left untouched.
- */
-void
-AtCommit_Portals(void)
-{
- HASH_SEQ_STATUS status;
- PortalHashEnt *hentry;
-
- hash_seq_init(&status, PortalHashTable);
-
- while ((hentry = (PortalHashEnt *) hash_seq_search(&status)) != NULL)
- {
- Portal portal = hentry->portal;
-
- /*
- * Do not touch active portals --- this can only happen in the case of
- * a multi-transaction utility command, such as VACUUM.
- *
- * Note however that any resource owner attached to such a portal is
- * still going to go away, so don't leave a dangling pointer.
- */
- if (portal->status == PORTAL_ACTIVE)
- {
- portal->resowner = NULL;
continue;
}
+ else
+ {
+ /* Zap all non-holdable portals */
+ PortalDrop(portal, true);
- /*
- * There should be no pinned portals anymore. Complain if someone
- * leaked one.
- */
- if (portal->portalPinned)
- elog(ERROR, "cannot commit while a portal is pinned");
+ /* Report we changed state */
+ result = true;
+ }
/*
- * Do nothing to cursors held over from a previous transaction
- * (including holdable ones just frozen by CommitHoldablePortals).
+ * After either freezing or dropping a portal, we have to restart
+ * the iteration, because we could have invoked user-defined code
+ * that caused a drop of the next portal in the hash chain.
*/
- if (portal->createSubid == InvalidSubTransactionId)
- continue;
-
- /* Zap all non-holdable portals */
- PortalDrop(portal, true);
-
- /* Restart the iteration in case that led to other drops */
- /* XXX is this really necessary? */
hash_seq_term(&status);
hash_seq_init(&status, PortalHashTable);
}
+
+ return result;
}
/*
if (portal->portalPinned)
portal->portalPinned = false;
+ /* We had better not be calling any user-defined code here */
+ Assert(portal->cleanup == NULL);
+
/* Zap it. */
PortalDrop(portal, false);
}
if (portal->portalPinned)
portal->portalPinned = false;
+ /* We had better not be calling any user-defined code here */
+ Assert(portal->cleanup == NULL);
+
/* Zap it. */
PortalDrop(portal, false);
}
*
* MARK indicates that the plan node must support Mark/Restore calls.
* When this is not passed, no Mark/Restore will occur.
+ *
+ * SKIP_TRIGGERS tells ExecutorStart/ExecutorFinish to skip calling
+ * AfterTriggerBeginQuery/AfterTriggerEndQuery. This does not necessarily
+ * mean that the plan can't queue any AFTER triggers; just that the caller
+ * is responsible for there being a trigger context for them to be queued in.
*/
#define EXEC_FLAG_EXPLAIN_ONLY 0x0001 /* EXPLAIN, no ANALYZE */
#define EXEC_FLAG_REWIND 0x0002 /* need efficient rescan */
#define EXEC_FLAG_BACKWARD 0x0004 /* need backward scan */
#define EXEC_FLAG_MARK 0x0008 /* need mark/restore */
+#define EXEC_FLAG_SKIP_TRIGGERS 0x0010 /* skip AfterTrigger calls */
/*
long count);
extern PGDLLIMPORT ExecutorRun_hook_type ExecutorRun_hook;
+/* Hook for plugins to get control in ExecutorFinish() */
+typedef void (*ExecutorFinish_hook_type) (QueryDesc *queryDesc);
+extern PGDLLIMPORT ExecutorFinish_hook_type ExecutorFinish_hook;
+
/* Hook for plugins to get control in ExecutorEnd() */
typedef void (*ExecutorEnd_hook_type) (QueryDesc *queryDesc);
extern PGDLLIMPORT ExecutorEnd_hook_type ExecutorEnd_hook;
ScanDirection direction, long count);
extern void standard_ExecutorRun(QueryDesc *queryDesc,
ScanDirection direction, long count);
+extern void ExecutorFinish(QueryDesc *queryDesc);
+extern void standard_ExecutorFinish(QueryDesc *queryDesc);
extern void ExecutorEnd(QueryDesc *queryDesc);
extern void standard_ExecutorEnd(QueryDesc *queryDesc);
extern void ExecutorRewind(QueryDesc *queryDesc);
uint32 es_processed; /* # of tuples processed */
Oid es_lastoid; /* last oid processed (by INSERT) */
+ int es_top_eflags; /* eflags passed to ExecutorStart */
int es_instrument; /* OR of InstrumentOption flags */
bool es_select_into; /* true if doing SELECT INTO */
bool es_into_oids; /* true to generate OIDs in SELECT INTO */
+ bool es_finished; /* true when ExecutorFinish is done */
List *es_exprcontexts; /* List of ExprContexts within EState */
/* Prototypes for functions in utils/mmgr/portalmem.c */
extern void EnablePortalManager(void);
-extern bool CommitHoldablePortals(void);
-extern bool PrepareHoldablePortals(void);
-extern void AtCommit_Portals(void);
+extern bool PreCommit_Portals(bool isPrepare);
extern void AtAbort_Portals(void);
extern void AtCleanup_Portals(void);
extern void AtSubCommit_Portals(SubTransactionId mySubid,
(33)
RETURNING *
)
-SELECT * FROM t;
+SELECT * FROM t LIMIT 1;
NOTICE: y_trigger: a = 31
NOTICE: y_trigger: a = 32
NOTICE: y_trigger: a = 33
a
----
31
- 32
- 33
-(3 rows)
+(1 row)
SELECT * FROM y;
a
(33)
RETURNING *
)
-SELECT * FROM t;
+SELECT * FROM t LIMIT 1;
SELECT * FROM y;