Commit 5bab36e9 authored by Tom Lane's avatar Tom Lane

Revise executor APIs so that all per-query state structure is built in

a per-query memory context created by CreateExecutorState --- and destroyed
by FreeExecutorState.  This provides a final solution to the longstanding
problem of memory leaked by various ExecEndNode calls.
parent 90b3a0b6
......@@ -8,7 +8,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/bootstrap/bootstrap.c,v 1.146 2002/12/13 19:45:45 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/bootstrap/bootstrap.c,v 1.147 2002/12/15 16:17:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -1143,8 +1143,7 @@ index_register(Oid heap,
/* predicate will likely be null, but may as well copy it */
newind->il_info->ii_Predicate = (List *)
copyObject(indexInfo->ii_Predicate);
newind->il_info->ii_PredicateState = (List *)
ExecInitExpr((Expr *) newind->il_info->ii_Predicate, NULL);
newind->il_info->ii_PredicateState = NIL;
newind->il_next = ILHead;
ILHead = newind;
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/catalog/index.c,v 1.207 2002/12/13 19:45:47 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/catalog/index.c,v 1.208 2002/12/15 16:17:38 tgl Exp $
*
*
* INTERFACE ROUTINES
......@@ -41,7 +41,6 @@
#include "executor/executor.h"
#include "miscadmin.h"
#include "optimizer/clauses.h"
#include "optimizer/planmain.h"
#include "optimizer/prep.h"
#include "parser/parse_func.h"
#include "storage/sinval.h"
......@@ -912,7 +911,6 @@ BuildIndexInfo(Form_pg_index indexStruct)
/*
* If partial index, convert predicate into expression nodetree
* and prepare an execution state nodetree for it
*/
if (VARSIZE(&indexStruct->indpred) > VARHDRSZ)
{
......@@ -921,9 +919,7 @@ BuildIndexInfo(Form_pg_index indexStruct)
predString = DatumGetCString(DirectFunctionCall1(textout,
PointerGetDatum(&indexStruct->indpred)));
ii->ii_Predicate = stringToNode(predString);
fix_opfuncids((Node *) ii->ii_Predicate);
ii->ii_PredicateState = (List *)
ExecInitExpr((Expr *) ii->ii_Predicate, NULL);
ii->ii_PredicateState = NIL;
pfree(predString);
}
else
......@@ -1489,9 +1485,10 @@ IndexBuildHeapScan(Relation heapRelation,
Datum attdata[INDEX_MAX_KEYS];
char nulls[INDEX_MAX_KEYS];
double reltuples;
List *predicate = indexInfo->ii_PredicateState;
List *predicate;
TupleTable tupleTable;
TupleTableSlot *slot;
EState *estate;
ExprContext *econtext;
Snapshot snapshot;
TransactionId OldestXmin;
......@@ -1503,28 +1500,42 @@ IndexBuildHeapScan(Relation heapRelation,
heapDescriptor = RelationGetDescr(heapRelation);
/*
* Need an EState for evaluation of functional-index functions
* and partial-index predicates.
*/
estate = CreateExecutorState();
econtext = GetPerTupleExprContext(estate);
/*
* If this is a predicate (partial) index, we will need to evaluate
* the predicate using ExecQual, which requires the current tuple to
* be in a slot of a TupleTable. In addition, ExecQual must have an
* ExprContext referring to that slot. Here, we initialize dummy
* TupleTable and ExprContext objects for this purpose. --Nels, Feb 92
*
* We construct the ExprContext anyway since we need a per-tuple
* temporary memory context for function evaluation -- tgl July 00
* be in a slot of a TupleTable.
*/
if (predicate != NIL)
if (indexInfo->ii_Predicate != NIL)
{
tupleTable = ExecCreateTupleTable(1);
slot = ExecAllocTableSlot(tupleTable);
ExecSetSlotDescriptor(slot, heapDescriptor, false);
/* Arrange for econtext's scan tuple to be the tuple under test */
econtext->ecxt_scantuple = slot;
/*
* Set up execution state for predicate. Note: we mustn't attempt to
* cache this in the indexInfo, since we're building it in a transient
* EState.
*/
predicate = (List *)
ExecPrepareExpr((Expr *) indexInfo->ii_Predicate,
estate);
}
else
{
tupleTable = NULL;
slot = NULL;
predicate = NIL;
}
econtext = MakeExprContext(slot, TransactionCommandContext);
/*
* Ok, begin our scan of the base relation. We use SnapshotAny
......@@ -1687,9 +1698,10 @@ IndexBuildHeapScan(Relation heapRelation,
heap_endscan(scan);
if (predicate != NIL)
if (tupleTable)
ExecDropTupleTable(tupleTable, true);
FreeExprContext(econtext);
FreeExecutorState(estate);
return reltuples;
}
......
......@@ -9,7 +9,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/copy.c,v 1.186 2002/12/13 19:45:48 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/copy.c,v 1.187 2002/12/15 16:17:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -35,7 +35,6 @@
#include "mb/pg_wchar.h"
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "optimizer/planmain.h"
#include "parser/parse_coerce.h"
#include "parser/parse_relation.h"
#include "rewrite/rewriteHandler.h"
......@@ -803,6 +802,8 @@ CopyFrom(Relation rel, List *attnumlist, bool binary, bool oids,
slot = ExecAllocTableSlot(tupleTable);
ExecSetSlotDescriptor(slot, tupDesc, false);
econtext = GetPerTupleExprContext(estate);
/*
* Pick up the required catalog information for each attribute in the
* relation, including the input function, the element type (to pass
......@@ -841,8 +842,8 @@ CopyFrom(Relation rel, List *attnumlist, bool binary, bool oids,
if (defexpr != NULL)
{
fix_opfuncids(defexpr);
defexprs[num_defaults] = ExecInitExpr((Expr *) defexpr, NULL);
defexprs[num_defaults] = ExecPrepareExpr((Expr *) defexpr,
estate);
defmap[num_defaults] = i;
num_defaults++;
}
......@@ -873,8 +874,8 @@ CopyFrom(Relation rel, List *attnumlist, bool binary, bool oids,
/* check whether any constraints actually found */
if (node != (Node *) prm)
{
fix_opfuncids(node);
constraintexprs[i] = ExecInitExpr((Expr *) node, NULL);
constraintexprs[i] = ExecPrepareExpr((Expr *) node,
estate);
hasConstraints = true;
}
}
......@@ -934,8 +935,6 @@ CopyFrom(Relation rel, List *attnumlist, bool binary, bool oids,
copy_lineno = 0;
fe_eof = false;
econtext = GetPerTupleExprContext(estate);
/* Make room for a PARAM_EXEC value for domain constraint checks */
if (hasConstraints)
econtext->ecxt_param_exec_vals = (ParamExecData *)
......@@ -953,9 +952,8 @@ CopyFrom(Relation rel, List *attnumlist, bool binary, bool oids,
/* Reset the per-tuple exprcontext */
ResetPerTupleExprContext(estate);
/* Switch to and reset per-tuple memory context, too */
/* Switch into its memory context */
MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
MemoryContextReset(CurrentMemoryContext);
/* Initialize all values for row to NULL */
MemSet(values, 0, num_phys_attrs * sizeof(Datum));
......@@ -1268,6 +1266,8 @@ CopyFrom(Relation rel, List *attnumlist, bool binary, bool oids,
ExecDropTupleTable(tupleTable, true);
ExecCloseIndices(resultRelInfo);
FreeExecutorState(estate);
}
......
......@@ -5,7 +5,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994-5, Regents of the University of California
*
* $Header: /cvsroot/pgsql/src/backend/commands/explain.c,v 1.98 2002/12/14 00:17:50 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/explain.c,v 1.99 2002/12/15 16:17:38 tgl Exp $
*
*/
......@@ -206,6 +206,8 @@ ExplainOneQuery(Query *query, ExplainStmt *stmt, TupOutputState *tstate)
gettimeofday(&starttime, NULL);
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
CommandCounterIncrement();
totaltime += elapsed_time(&starttime);
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/indexcmds.c,v 1.94 2002/12/13 19:45:50 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/indexcmds.c,v 1.95 2002/12/15 16:17:39 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -27,7 +27,6 @@
#include "executor/executor.h"
#include "miscadmin.h"
#include "optimizer/clauses.h"
#include "optimizer/planmain.h"
#include "optimizer/prep.h"
#include "parser/parsetree.h"
#include "parser/parse_coerce.h"
......@@ -163,7 +162,6 @@ DefineIndex(RangeVar *heapRelation,
if (predicate)
{
cnfPred = canonicalize_qual((Expr *) copyObject(predicate), true);
fix_opfuncids((Node *) cnfPred);
CheckPredicate(cnfPred, rangetable, relationId);
}
......@@ -173,8 +171,7 @@ DefineIndex(RangeVar *heapRelation,
*/
indexInfo = makeNode(IndexInfo);
indexInfo->ii_Predicate = cnfPred;
indexInfo->ii_PredicateState = (List *)
ExecInitExpr((Expr *) cnfPred, NULL);
indexInfo->ii_PredicateState = NIL;
indexInfo->ii_FuncOid = InvalidOid;
indexInfo->ii_Unique = unique;
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/portalcmds.c,v 1.5 2002/12/05 15:50:30 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/portalcmds.c,v 1.6 2002/12/15 16:17:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -31,28 +31,22 @@
void
PortalCleanup(Portal portal)
{
MemoryContext oldcontext;
/*
* sanity checks
*/
AssertArg(PortalIsValid(portal));
AssertArg(portal->cleanup == PortalCleanup);
/*
* set proper portal-executor context before calling ExecMain.
*/
oldcontext = MemoryContextSwitchTo(PortalGetHeapMemory(portal));
/*
* tell the executor to shutdown the query
*/
ExecutorEnd(PortalGetQueryDesc(portal));
/*
* switch back to previous context
* This should be unnecessary since the querydesc should be in the
* portal's memory context, but do it anyway for symmetry.
*/
MemoryContextSwitchTo(oldcontext);
FreeQueryDesc(PortalGetQueryDesc(portal));
}
......
......@@ -6,7 +6,7 @@
* Copyright (c) 2002, PostgreSQL Global Development Group
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/prepare.c,v 1.10 2002/12/13 19:45:51 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/prepare.c,v 1.11 2002/12/15 16:17:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -15,7 +15,6 @@
#include "commands/prepare.h"
#include "executor/executor.h"
#include "utils/guc.h"
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "rewrite/rewriteHandler.h"
#include "tcop/pquery.h"
......@@ -50,7 +49,6 @@ static void InitQueryHashTable(void);
static void StoreQuery(const char *stmt_name, List *query_list,
List *plan_list, List *argtype_list);
static QueryHashEntry *FetchQuery(const char *plan_name);
static void RunQuery(QueryDesc *qdesc);
/*
......@@ -96,33 +94,37 @@ ExecuteQuery(ExecuteStmt *stmt, CommandDest outputDest)
*query_list,
*plan_list;
ParamListInfo paramLI = NULL;
EState *estate;
/* Look it up in the hash table */
entry = FetchQuery(stmt->name);
/* Make working copies the executor can safely scribble on */
query_list = (List *) copyObject(entry->query_list);
plan_list = (List *) copyObject(entry->plan_list);
query_list = entry->query_list;
plan_list = entry->plan_list;
Assert(length(query_list) == length(plan_list));
/*
* Need an EState to evaluate parameters; must not delete it till end
* of query, in case parameters are pass-by-reference.
*/
estate = CreateExecutorState();
/* Evaluate parameters, if any */
if (entry->argtype_list != NIL)
{
int nargs = length(entry->argtype_list);
int i = 0;
List *exprstates;
ExprContext *econtext = MakeExprContext(NULL, CurrentMemoryContext);
/* Parser should have caught this error, but check */
if (nargs != length(stmt->params))
elog(ERROR, "ExecuteQuery: wrong number of arguments");
fix_opfuncids((Node *) stmt->params);
exprstates = (List *) ExecInitExpr((Expr *) stmt->params, NULL);
exprstates = (List *) ExecPrepareExpr((Expr *) stmt->params, estate);
paramLI = (ParamListInfo) palloc0((nargs + 1) * sizeof(ParamListInfoData));
paramLI = (ParamListInfo)
palloc0((nargs + 1) * sizeof(ParamListInfoData));
foreach(l, exprstates)
{
......@@ -130,7 +132,7 @@ ExecuteQuery(ExecuteStmt *stmt, CommandDest outputDest)
bool isNull;
paramLI[i].value = ExecEvalExprSwitchContext(n,
econtext,
GetPerTupleExprContext(estate),
&isNull,
NULL);
paramLI[i].kind = PARAM_NUM;
......@@ -173,7 +175,13 @@ ExecuteQuery(ExecuteStmt *stmt, CommandDest outputDest)
qdesc->dest = None;
}
RunQuery(qdesc);
ExecutorStart(qdesc);
ExecutorRun(qdesc, ForwardScanDirection, 0L);
ExecutorEnd(qdesc);
FreeQueryDesc(qdesc);
if (log_executor_stats)
ShowUsage("EXECUTOR STATISTICS");
......@@ -188,7 +196,9 @@ ExecuteQuery(ExecuteStmt *stmt, CommandDest outputDest)
CommandCounterIncrement();
}
/* No need to pfree memory, MemoryContext will be reset */
FreeExecutorState(estate);
/* No need to pfree other memory, MemoryContext will be reset */
}
/*
......@@ -333,17 +343,6 @@ FetchQueryParams(const char *plan_name)
return entry->argtype_list;
}
/*
* Actually execute a prepared query.
*/
static void
RunQuery(QueryDesc *qdesc)
{
ExecutorStart(qdesc);
ExecutorRun(qdesc, ForwardScanDirection, 0L);
ExecutorEnd(qdesc);
}
/*
* Implements the 'DEALLOCATE' utility statement: deletes the
* specified plan from storage.
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/tablecmds.c,v 1.60 2002/12/13 19:45:51 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/tablecmds.c,v 1.61 2002/12/15 16:17:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -37,7 +37,6 @@
#include "nodes/makefuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/plancat.h"
#include "optimizer/planmain.h"
#include "optimizer/prep.h"
#include "parser/gramparse.h"
#include "parser/parse_coerce.h"
......@@ -2713,6 +2712,7 @@ AlterTableAddCheckConstraint(Relation rel, Constraint *constr)
ParseState *pstate;
bool successful = true;
HeapScanDesc scan;
EState *estate;
ExprContext *econtext;
TupleTableSlot *slot;
HeapTuple tuple;
......@@ -2723,9 +2723,7 @@ AlterTableAddCheckConstraint(Relation rel, Constraint *constr)
/*
* We need to make a parse state and range
* table to allow us to transformExpr and
* fix_opfuncids to get a version of the
* expression we can pass to ExecQual
* table to allow us to do transformExpr()
*/
pstate = make_parsestate(NULL);
rte = addRangeTableEntryForRelation(pstate,
......@@ -2765,19 +2763,22 @@ AlterTableAddCheckConstraint(Relation rel, Constraint *constr)
*/
expr = eval_const_expressions(expr);
/* And fix the opfuncids */
fix_opfuncids(expr);
/* Needs to be in implicit-ANDs form for ExecQual */
qual = make_ands_implicit((Expr *) expr);
qual = makeList1(expr);
/* Need an EState to run ExecQual */
estate = CreateExecutorState();
econtext = GetPerTupleExprContext(estate);
/* build execution state for qual */
qualstate = (List *) ExecInitExpr((Expr *) qual, NULL);
qualstate = (List *) ExecPrepareExpr((Expr *) qual, estate);
/* Make tuple slot to hold tuples */
slot = MakeTupleTableSlot();
ExecSetSlotDescriptor(slot, RelationGetDescr(rel), false);
/* Make an expression context for ExecQual */
econtext = MakeExprContext(slot, CurrentMemoryContext);
/* Arrange for econtext's scan tuple to be the tuple under test */
econtext->ecxt_scantuple = slot;
/*
* Scan through the rows now, checking the expression at each row.
......@@ -2797,8 +2798,8 @@ AlterTableAddCheckConstraint(Relation rel, Constraint *constr)
heap_endscan(scan);
FreeExprContext(econtext);
pfree(slot);
FreeExecutorState(estate);
if (!successful)
elog(ERROR, "AlterTableAddConstraint: rejected due to CHECK constraint %s",
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/typecmds.c,v 1.24 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/typecmds.c,v 1.25 2002/12/15 16:17:43 tgl Exp $
*
* DESCRIPTION
* The "DefineFoo" routines take the parse tree and pick out the
......@@ -47,7 +47,6 @@
#include "miscadmin.h"
#include "nodes/nodes.h"
#include "optimizer/clauses.h"
#include "optimizer/planmain.h"
#include "optimizer/var.h"
#include "parser/parse_coerce.h"
#include "parser/parse_expr.h"
......@@ -1242,6 +1241,7 @@ AlterDomainAddConstraint(List *names, Node *newConstraint)
List *rels;
List *rt;
Form_pg_type typTup;
EState *estate;
ExprContext *econtext;
char *ccbin;
Expr *expr;
......@@ -1338,11 +1338,13 @@ AlterDomainAddConstraint(List *names, Node *newConstraint)
* the constraint is being added to.
*/
expr = (Expr *) stringToNode(ccbin);
fix_opfuncids((Node *) expr);
exprstate = ExecInitExpr(expr, NULL);
/* Make an expression context for ExecEvalExpr */
econtext = MakeExprContext(NULL, CurrentMemoryContext);
/* Need an EState to run ExecEvalExpr */
estate = CreateExecutorState();
econtext = GetPerTupleExprContext(estate);
/* build execution state for expr */
exprstate = ExecPrepareExpr(expr, estate);
rels = get_rels_with_domain(domainoid);
......@@ -1377,7 +1379,9 @@ AlterDomainAddConstraint(List *names, Node *newConstraint)
econtext->domainValue_datum = d;
econtext->domainValue_isNull = isNull;
conResult = ExecEvalExpr(exprstate, econtext, &isNull, NULL);
conResult = ExecEvalExprSwitchContext(exprstate,
econtext,
&isNull, NULL);
if (!isNull && !DatumGetBool(conResult))
elog(ERROR, "AlterDomainAddConstraint: Domain %s constraint %s failed",
......@@ -1393,7 +1397,7 @@ AlterDomainAddConstraint(List *names, Node *newConstraint)
heap_close(testrel, NoLock);
}
FreeExprContext(econtext);
FreeExecutorState(estate);
/* Clean up */
heap_close(rel, NoLock);
......
......@@ -13,7 +13,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/commands/vacuum.c,v 1.244 2002/10/31 19:25:29 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/commands/vacuum.c,v 1.245 2002/12/15 16:17:44 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -1437,6 +1437,8 @@ repair_frag(VRelStats *vacrelstats, Relation onerel,
* We need a ResultRelInfo and an EState so we can use the regular
* executor's index-entry-making machinery.
*/
estate = CreateExecutorState();
resultRelInfo = makeNode(ResultRelInfo);
resultRelInfo->ri_RangeTableIndex = 1; /* dummy */
resultRelInfo->ri_RelationDesc = onerel;
......@@ -1444,7 +1446,6 @@ repair_frag(VRelStats *vacrelstats, Relation onerel,
ExecOpenIndices(resultRelInfo);
estate = CreateExecutorState();
estate->es_result_relations = resultRelInfo;
estate->es_num_result_relations = 1;
estate->es_result_relation_info = resultRelInfo;
......@@ -2484,6 +2485,8 @@ repair_frag(VRelStats *vacrelstats, Relation onerel,
ExecDropTupleTable(tupleTable, true);
ExecCloseIndices(resultRelInfo);
FreeExecutorState(estate);
}
/*
......
$Header: /cvsroot/pgsql/src/backend/executor/README,v 1.2 2002/12/05 15:50:30 tgl Exp $
$Header: /cvsroot/pgsql/src/backend/executor/README,v 1.3 2002/12/15 16:17:45 tgl Exp $
The Postgres Executor
---------------------
......@@ -60,6 +60,83 @@ ExprState nodes. (Actually, there are also List nodes, which are used as
"glue" in all four kinds of tree.)
Memory Management
-----------------
A "per query" memory context is created during CreateExecutorState();
all storage allocated during an executor invocation is allocated in that
context or a child context. This allows easy reclamation of storage
during executor shutdown --- rather than messing with retail pfree's and
probable storage leaks, we just destroy the memory context.
In particular, the plan state trees and expression state trees described
in the previous section are allocated in the per-query memory context.
To avoid intra-query memory leaks, most processing while a query runs
is done in "per tuple" memory contexts, which are so-called because they
are typically reset to empty once per tuple. Per-tuple contexts are usually
associated with ExprContexts, and commonly each PlanState node has its own
ExprContext to evaluate its qual and targetlist expressions in.
Query Processing Control Flow
-----------------------------
This is a sketch of control flow for full query processing:
CreateQueryDesc
ExecutorStart
CreateExecutorState
creates per-query context
switch to per-query context to run ExecInitNode
ExecInitNode --- recursively scans plan tree
CreateExprContext
creates per-tuple context
ExecInitExpr
ExecutorRun
ExecProcNode --- recursively called in per-query context
ExecEvalExpr --- called in per-tuple context
ResetExprContext --- to free memory
ExecutorEnd
ExecEndNode --- recursively releases resources
FreeExecutorState
frees per-query context and child contexts
FreeQueryDesc
Per above comments, it's not really critical for ExecEndNode to free any
memory; it'll all go away in FreeExecutorState anyway. However, we do need to
be careful to close relations, drop buffer pins, etc, so we do need to scan
the plan state tree to find these sorts of resources.
The executor can also be used to evaluate simple expressions without any Plan
tree ("simple" meaning "no aggregates and no sub-selects", though such might
be hidden inside function calls). This case has a flow of control like
CreateExecutorState
creates per-query context
CreateExprContext -- or use GetPerTupleExprContext(estate)
creates per-tuple context
ExecPrepareExpr
switch to per-query context to run ExecInitExpr
ExecInitExpr
Repeatedly do:
ExecEvalExprSwitchContext
ExecEvalExpr --- called in per-tuple context
ResetExprContext --- to free memory
FreeExecutorState
frees per-query context, as well as ExprContext
(a separate FreeExprContext call is not necessary)
EvalPlanQual (READ COMMITTED update checking)
---------------------------------------------
......
......@@ -26,7 +26,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/execMain.c,v 1.192 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/execMain.c,v 1.193 2002/12/15 16:17:45 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -40,7 +40,6 @@
#include "executor/execdebug.h"
#include "executor/execdefs.h"
#include "miscadmin.h"
#include "optimizer/planmain.h"
#include "optimizer/var.h"
#include "parser/parsetree.h"
#include "utils/acl.h"
......@@ -53,7 +52,6 @@ static void initResultRelInfo(ResultRelInfo *resultRelInfo,
Index resultRelationIndex,
List *rangeTable,
CmdType operation);
static void EndPlan(PlanState *planstate, EState *estate);
static TupleTableSlot *ExecutePlan(EState *estate, PlanState *planstate,
CmdType operation,
long numberTuples,
......@@ -86,27 +84,31 @@ static void ExecCheckRTEPerms(RangeTblEntry *rte, CmdType operation);
* 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.
*
* XXX this will change soon:
* NB: the CurrentMemoryContext when this is called must be the context
* to be used as the per-query context for the query plan. ExecutorRun()
* and ExecutorEnd() must be called in this same memory context.
* NB: the CurrentMemoryContext when this is called will become the parent
* of the per-query context used for this Executor invocation.
* ----------------------------------------------------------------
*/
void
ExecutorStart(QueryDesc *queryDesc)
{
EState *estate;
MemoryContext oldcontext;
/* sanity checks: queryDesc must not be started already */
Assert(queryDesc != NULL);
Assert(queryDesc->estate == NULL);
/*
* Build EState, fill with parameters from queryDesc
* Build EState, switch into per-query memory context for startup.
*/
estate = CreateExecutorState();
queryDesc->estate = estate;
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
/*
* Fill in parameters, if any, from queryDesc
*/
estate->es_param_list_info = queryDesc->params;
if (queryDesc->plantree->nParamExec > 0)
......@@ -128,6 +130,8 @@ ExecutorStart(QueryDesc *queryDesc)
* Initialize the plan state tree
*/
InitPlan(queryDesc);
MemoryContextSwitchTo(oldcontext);
}
/* ----------------------------------------------------------------
......@@ -152,23 +156,30 @@ TupleTableSlot *
ExecutorRun(QueryDesc *queryDesc,
ScanDirection direction, long count)
{
CmdType operation;
EState *estate;
CmdType operation;
CommandDest dest;
DestReceiver *destfunc;
TupleTableSlot *result;
MemoryContext oldcontext;
/* sanity checks */
Assert(queryDesc != NULL);
estate = queryDesc->estate;
Assert(estate != NULL);
/*
* sanity checks
* Switch into per-query memory context
*/
Assert(queryDesc != NULL);
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
/*
* extract information from the query descriptor and the query
* feature.
*/
operation = queryDesc->operation;
estate = queryDesc->estate;
dest = queryDesc->dest;
/*
......@@ -199,6 +210,8 @@ ExecutorRun(QueryDesc *queryDesc,
*/
(*destfunc->cleanup) (destfunc);
MemoryContextSwitchTo(oldcontext);
return result;
}
......@@ -213,72 +226,37 @@ void
ExecutorEnd(QueryDesc *queryDesc)
{
EState *estate;
MemoryContext oldcontext;
/* sanity checks */
Assert(queryDesc != NULL);
estate = queryDesc->estate;
EndPlan(queryDesc->planstate, estate);
if (estate->es_snapshot != NULL)
{
if (estate->es_snapshot->xcnt > 0)
pfree(estate->es_snapshot->xip);
pfree(estate->es_snapshot);
estate->es_snapshot = NULL;
}
if (estate->es_param_exec_vals != NULL)
{
pfree(estate->es_param_exec_vals);
estate->es_param_exec_vals = NULL;
}
}
/*
* CreateExecutorState
*/
EState *
CreateExecutorState(void)
{
EState *state;
Assert(estate != NULL);
/*
* create a new executor state
* Switch into per-query memory context to run ExecEndPlan
*/
state = makeNode(EState);
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
ExecEndPlan(queryDesc->planstate, estate);
/*
* initialize the Executor State structure
* Must switch out of context before destroying it
*/
state->es_direction = ForwardScanDirection;
state->es_range_table = NIL;
state->es_result_relations = NULL;
state->es_num_result_relations = 0;
state->es_result_relation_info = NULL;
state->es_junkFilter = NULL;
state->es_into_relation_descriptor = NULL;
state->es_param_list_info = NULL;
state->es_param_exec_vals = NULL;
state->es_tupleTable = NULL;
state->es_query_cxt = CurrentMemoryContext;
state->es_instrument = false;
state->es_per_tuple_exprcontext = NULL;
MemoryContextSwitchTo(oldcontext);
/*
* return the executor state structure
* Release EState and per-query memory context. This should release
* everything the executor has allocated.
*/
return state;
FreeExecutorState(estate);
/* Reset queryDesc fields that no longer point to anything */
queryDesc->tupDesc = NULL;
queryDesc->estate = NULL;
queryDesc->planstate = NULL;
}
......@@ -794,13 +772,13 @@ initResultRelInfo(ResultRelInfo *resultRelInfo,
}
/* ----------------------------------------------------------------
* EndPlan
* ExecEndPlan
*
* Cleans up the query plan -- closes files and frees up storage
* ----------------------------------------------------------------
*/
static void
EndPlan(PlanState *planstate, EState *estate)
void
ExecEndPlan(PlanState *planstate, EState *estate)
{
ResultRelInfo *resultRelInfo;
int i;
......@@ -1542,9 +1520,8 @@ ExecRelCheck(ResultRelInfo *resultRelInfo,
for (i = 0; i < ncheck; i++)
{
qual = (List *) stringToNode(check[i].ccbin);
fix_opfuncids((Node *) qual);
resultRelInfo->ri_ConstraintExprs[i] = (List *)
ExecInitExpr((Expr *) qual, NULL);
ExecPrepareExpr((Expr *) qual, estate);
}
MemoryContextSwitchTo(oldContext);
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/execQual.c,v 1.120 2002/12/14 00:17:50 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/execQual.c,v 1.121 2002/12/15 16:17:45 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -40,6 +40,7 @@
#include "executor/functions.h"
#include "executor/nodeSubplan.h"
#include "miscadmin.h"
#include "optimizer/planmain.h"
#include "parser/parse_expr.h"
#include "utils/acl.h"
#include "utils/array.h"
......@@ -1896,9 +1897,11 @@ ExecEvalExprSwitchContext(ExprState *expression,
* cleanup work can register a shutdown callback in the ExprContext.
*
* 'node' is the root of the expression tree to examine
* 'parent' is the PlanState node that owns the expression,
* or NULL if we are preparing an expression that is not associated
* with a plan. (If so, it can't have aggs or subplans.)
* 'parent' is the PlanState node that owns the expression.
*
* 'parent' may be NULL if we are preparing an expression that is not
* associated with a plan tree. (If so, it can't have aggs or subplans.)
* This case should usually come through ExecPrepareExpr, not directly here.
*/
ExprState *
ExecInitExpr(Expr *node, PlanState *parent)
......@@ -2017,6 +2020,7 @@ ExecInitExpr(Expr *node, PlanState *parent)
* parent->subPlan. The subplans will be initialized later.
*/
parent->subPlan = lcons(sstate, parent->subPlan);
sstate->sub_estate = NULL;
sstate->planstate = NULL;
sstate->oper = (List *)
......@@ -2149,6 +2153,7 @@ ExecInitExprInitPlan(SubPlan *node, PlanState *parent)
elog(ERROR, "ExecInitExpr: SubPlan not expected here");
/* The subplan's state will be initialized later */
sstate->sub_estate = NULL;
sstate->planstate = NULL;
sstate->oper = (List *) ExecInitExpr((Expr *) node->oper, parent);
......@@ -2159,6 +2164,33 @@ ExecInitExprInitPlan(SubPlan *node, PlanState *parent)
return sstate;
}
/*
* ExecPrepareExpr --- initialize for expression execution outside a normal
* Plan tree context.
*
* This differs from ExecInitExpr in that we don't assume the caller is
* already running in the EState's per-query context. Also, we apply
* fix_opfuncids() to the passed expression tree to be sure it is ready
* to run. (In ordinary Plan trees the planner will have fixed opfuncids,
* but callers outside the executor will not have done this.)
*/
ExprState *
ExecPrepareExpr(Expr *node, EState *estate)
{
ExprState *result;
MemoryContext oldcontext;
fix_opfuncids((Node *) node);
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
result = ExecInitExpr(node, NULL);
MemoryContextSwitchTo(oldcontext);
return result;
}
/* ----------------------------------------------------------------
* ExecQual / ExecTargetList / ExecProject
......
This diff is collapsed.
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/functions.c,v 1.61 2002/12/05 15:50:32 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/functions.c,v 1.62 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -284,7 +284,8 @@ postquel_end(execution_state *es)
if (es->qd->operation != CMD_UTILITY)
ExecutorEnd(es->qd);
pfree(es->qd);
FreeQueryDesc(es->qd);
es->qd = NULL;
es->status = F_EXEC_DONE;
......
......@@ -45,7 +45,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeAgg.c,v 1.100 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeAgg.c,v 1.101 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -1392,8 +1392,6 @@ ExecEndAgg(AggState *node)
tuplesort_end(peraggstate->sortstate);
}
ExecFreeProjectionInfo(&node->ss.ps);
/*
* Free both the expr contexts.
*/
......@@ -1401,18 +1399,13 @@ ExecEndAgg(AggState *node)
node->ss.ps.ps_ExprContext = node->tmpcontext;
ExecFreeExprContext(&node->ss.ps);
/* clean up tuple table */
ExecClearTuple(node->ss.ss_ScanTupleSlot);
MemoryContextDelete(node->aggcontext);
outerPlan = outerPlanState(node);
ExecEndNode(outerPlan);
/* clean up tuple table */
ExecClearTuple(node->ss.ss_ScanTupleSlot);
if (node->grp_firstTuple != NULL)
{
heap_freetuple(node->grp_firstTuple);
node->grp_firstTuple = NULL;
}
}
void
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeFunctionscan.c,v 1.15 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeFunctionscan.c,v 1.16 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -273,9 +273,8 @@ void
ExecEndFunctionScan(FunctionScanState *node)
{
/*
* Free the projection info and the scan attribute info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->ss.ps);
ExecFreeExprContext(&node->ss.ps);
/*
......
......@@ -15,7 +15,7 @@
* locate group boundaries.
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeGroup.c,v 1.52 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeGroup.c,v 1.53 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -218,19 +218,13 @@ ExecEndGroup(GroupState *node)
{
PlanState *outerPlan;
ExecFreeProjectionInfo(&node->ss.ps);
ExecFreeExprContext(&node->ss.ps);
outerPlan = outerPlanState(node);
ExecEndNode(outerPlan);
/* clean up tuple table */
ExecClearTuple(node->ss.ss_ScanTupleSlot);
if (node->grp_firstTuple != NULL)
{
heap_freetuple(node->grp_firstTuple);
node->grp_firstTuple = NULL;
}
outerPlan = outerPlanState(node);
ExecEndNode(outerPlan);
}
void
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeHash.c,v 1.70 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeHash.c,v 1.71 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -179,10 +179,8 @@ ExecEndHash(HashState *node)
PlanState *outerPlan;
/*
* free projection info. no need to free result type info because
* that came from the outer plan...
* free exprcontext
*/
ExecFreeProjectionInfo(&node->ps);
ExecFreeExprContext(&node->ps);
/*
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeHashjoin.c,v 1.44 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeHashjoin.c,v 1.45 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -456,17 +456,10 @@ ExecEndHashJoin(HashJoinState *node)
}
/*
* Free the projection info and the scan attribute info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->js.ps);
ExecFreeExprContext(&node->js.ps);
/*
* clean up subtrees
*/
ExecEndNode(outerPlanState(node));
ExecEndNode(innerPlanState(node));
/*
* clean out the tuple table
*/
......@@ -474,6 +467,11 @@ ExecEndHashJoin(HashJoinState *node)
ExecClearTuple(node->hj_OuterTupleSlot);
ExecClearTuple(node->hj_HashTupleSlot);
/*
* clean up subtrees
*/
ExecEndNode(outerPlanState(node));
ExecEndNode(innerPlanState(node));
}
/* ----------------------------------------------------------------
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeIndexscan.c,v 1.74 2002/12/13 19:45:52 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeIndexscan.c,v 1.75 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -399,44 +399,38 @@ ExecIndexReScan(IndexScanState *node, ExprContext *exprCtxt)
/* ----------------------------------------------------------------
* ExecEndIndexScan
*
* old comments
* Releases any storage allocated through C routines.
* Returns nothing.
* ----------------------------------------------------------------
*/
void
ExecEndIndexScan(IndexScanState *node)
{
ExprState ***runtimeKeyInfo;
ScanKey *scanKeys;
int *numScanKeys;
int numIndices;
Relation relation;
RelationPtr indexRelationDescs;
IndexScanDescPtr indexScanDescs;
Relation relation;
int i;
runtimeKeyInfo = node->iss_RuntimeKeyInfo;
/*
* extract information from the node
*/
numIndices = node->iss_NumIndices;
scanKeys = node->iss_ScanKeys;
numScanKeys = node->iss_NumScanKeys;
indexRelationDescs = node->iss_RelationDescs;
indexScanDescs = node->iss_ScanDescs;
relation = node->ss.ss_currentRelation;
/*
* Free the projection info and the scan attribute info
* Free the exprcontext(s)
*/
ExecFreeProjectionInfo(&node->ss.ps);
ExecFreeExprContext(&node->ss.ps);
if (node->iss_RuntimeContext)
FreeExprContext(node->iss_RuntimeContext);
/*
* clear out tuple table slots
*/
ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
ExecClearTuple(node->ss.ss_ScanTupleSlot);
/*
* close the index relations
*/
......@@ -458,36 +452,6 @@ ExecEndIndexScan(IndexScanState *node)
* locking, however.)
*/
heap_close(relation, NoLock);
/*
* free the scan keys used in scanning the indices
*/
for (i = 0; i < numIndices; i++)
{
if (scanKeys[i] != NULL)
pfree(scanKeys[i]);
}
pfree(scanKeys);
pfree(numScanKeys);
if (runtimeKeyInfo)
{
for (i = 0; i < numIndices; i++)
{
if (runtimeKeyInfo[i] != NULL)
pfree(runtimeKeyInfo[i]);
}
pfree(runtimeKeyInfo);
}
/*
* clear out tuple table slots
*/
ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
ExecClearTuple(node->ss.ss_ScanTupleSlot);
pfree(node->iss_RelationDescs);
pfree(node->iss_ScanDescs);
pfree(node);
}
/* ----------------------------------------------------------------
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeLimit.c,v 1.13 2002/12/13 19:45:54 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeLimit.c,v 1.14 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -349,10 +349,10 @@ ExecEndLimit(LimitState *node)
{
ExecFreeExprContext(&node->ps);
ExecEndNode(outerPlanState(node));
/* clean up tuple table */
ExecClearTuple(node->ps.ps_ResultTupleSlot);
ExecEndNode(outerPlanState(node));
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeMaterial.c,v 1.39 2002/12/05 15:50:33 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeMaterial.c,v 1.40 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -196,17 +196,17 @@ ExecEndMaterial(MaterialState *node)
*/
ExecClearTuple(node->ss.ss_ScanTupleSlot);
/*
* shut down the subplan
*/
ExecEndNode(outerPlanState(node));
/*
* Release tuplestore resources
*/
if (node->tuplestorestate != NULL)
tuplestore_end((Tuplestorestate *) node->tuplestorestate);
node->tuplestorestate = NULL;
/*
* shut down the subplan
*/
ExecEndNode(outerPlanState(node));
}
/* ----------------------------------------------------------------
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeMergejoin.c,v 1.54 2002/12/13 19:45:54 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeMergejoin.c,v 1.55 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -1551,23 +1551,22 @@ ExecEndMergeJoin(MergeJoinState *node)
"ending node processing");
/*
* Free the projection info and the scan attribute info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->js.ps);
ExecFreeExprContext(&node->js.ps);
/*
* shut down the subplans
*/
ExecEndNode(innerPlanState(node));
ExecEndNode(outerPlanState(node));
/*
* clean out the tuple table
*/
ExecClearTuple(node->js.ps.ps_ResultTupleSlot);
ExecClearTuple(node->mj_MarkedTupleSlot);
/*
* shut down the subplans
*/
ExecEndNode(innerPlanState(node));
ExecEndNode(outerPlanState(node));
MJ1_printf("ExecEndMergeJoin: %s\n",
"node processing ended");
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeNestloop.c,v 1.28 2002/12/13 19:45:55 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeNestloop.c,v 1.29 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -364,21 +364,20 @@ ExecEndNestLoop(NestLoopState *node)
"ending node processing");
/*
* Free the projection info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->js.ps);
ExecFreeExprContext(&node->js.ps);
/*
* close down subplans
* clean out the tuple table
*/
ExecEndNode(outerPlanState(node));
ExecEndNode(innerPlanState(node));
ExecClearTuple(node->js.ps.ps_ResultTupleSlot);
/*
* clean out the tuple table
* close down subplans
*/
ExecClearTuple(node->js.ps.ps_ResultTupleSlot);
ExecEndNode(outerPlanState(node));
ExecEndNode(innerPlanState(node));
NL1_printf("ExecEndNestLoop: %s\n",
"node processing ended");
......
......@@ -34,7 +34,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeResult.c,v 1.23 2002/12/13 19:45:55 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeResult.c,v 1.24 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -248,9 +248,8 @@ void
ExecEndResult(ResultState *node)
{
/*
* Free the projection info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->ps);
ExecFreeExprContext(&node->ps);
/*
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSeqscan.c,v 1.40 2002/12/13 19:45:55 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSeqscan.c,v 1.41 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -264,22 +264,21 @@ ExecEndSeqScan(SeqScanState *node)
scanDesc = node->ss_currentScanDesc;
/*
* Free the projection info and the scan attribute info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->ps);
ExecFreeExprContext(&node->ps);
/*
* close heap scan
*/
heap_endscan(scanDesc);
/*
* clean out the tuple table
*/
ExecClearTuple(node->ps.ps_ResultTupleSlot);
ExecClearTuple(node->ss_ScanTupleSlot);
/*
* close heap scan
*/
heap_endscan(scanDesc);
/*
* close the heap relation.
*
......
......@@ -21,7 +21,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSetOp.c,v 1.7 2002/12/05 15:50:33 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSetOp.c,v 1.8 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -301,9 +301,9 @@ ExecEndSetOp(SetOpState *node)
ExecClearTuple(node->ps.ps_ResultTupleSlot);
node->ps.ps_OuterTupleSlot = NULL;
ExecEndNode(outerPlanState(node));
MemoryContextDelete(node->tempContext);
ExecEndNode(outerPlanState(node));
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSort.c,v 1.41 2002/12/05 15:50:33 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSort.c,v 1.42 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -284,11 +284,6 @@ ExecEndSort(SortState *node)
*/
ExecClearTuple(node->ss.ss_ScanTupleSlot);
/*
* shut down the subplan
*/
ExecEndNode(outerPlanState(node));
/*
* Release tuplesort resources
*/
......@@ -296,6 +291,11 @@ ExecEndSort(SortState *node)
tuplesort_end((Tuplesortstate *) node->tuplesortstate);
node->tuplesortstate = NULL;
/*
* shut down the subplan
*/
ExecEndNode(outerPlanState(node));
SO1_printf("ExecEndSort: %s\n",
"sort node shutdown");
}
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubplan.c,v 1.38 2002/12/14 00:17:50 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubplan.c,v 1.39 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -47,9 +47,10 @@ ExecSubPlan(SubPlanState *node,
/*
* We are probably in a short-lived expression-evaluation context.
* Switch to longer-lived per-query context.
* Switch to the child plan's per-query context for manipulating its
* chgParam, calling ExecProcNode on it, etc.
*/
oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory);
oldcontext = MemoryContextSwitchTo(node->sub_estate->es_query_cxt);
if (subplan->setParam != NIL)
elog(ERROR, "ExecSubPlan: can't set parent params from subquery");
......@@ -132,10 +133,13 @@ ExecSubPlan(SubPlanState *node,
* ExecProcNode() call. node->curTuple keeps track of the
* copied tuple for eventual freeing.
*/
MemoryContextSwitchTo(econtext->ecxt_per_query_memory);
tup = heap_copytuple(tup);
if (node->curTuple)
heap_freetuple(node->curTuple);
node->curTuple = tup;
MemoryContextSwitchTo(node->sub_estate->es_query_cxt);
result = heap_getattr(tup, col, tdesc, isNull);
/* keep scanning subplan to make sure there's only one tuple */
continue;
......@@ -295,6 +299,7 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
{
SubPlan *subplan = (SubPlan *) node->xprstate.expr;
EState *sp_estate;
MemoryContext oldcontext;
/*
* Do access checking on the rangetable entries in the subquery.
......@@ -303,15 +308,23 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
ExecCheckRTPerms(subplan->rtable, CMD_SELECT);
/*
* initialize state
* initialize my state
*/
node->needShutdown = false;
node->curTuple = NULL;
/*
* create an EState for the subplan
*
* The subquery needs its own EState because it has its own rangetable.
* It shares our Param ID space, however. XXX if rangetable access were
* done differently, the subquery could share our EState, which would
* eliminate some thrashing about in this module...
*/
sp_estate = CreateExecutorState();
node->sub_estate = sp_estate;
oldcontext = MemoryContextSwitchTo(sp_estate->es_query_cxt);
sp_estate->es_range_table = subplan->rtable;
sp_estate->es_param_list_info = estate->es_param_list_info;
......@@ -322,12 +335,14 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
sp_estate->es_instrument = estate->es_instrument;
/*
* Start up the subplan
* Start up the subplan (this is a very cut-down form of InitPlan())
*/
node->planstate = ExecInitNode(subplan->plan, sp_estate);
node->needShutdown = true; /* now we need to shutdown the subplan */
MemoryContextSwitchTo(oldcontext);
/*
* If this plan is un-correlated or undirect correlated one and want
* to set params for parent plan then prepare parameters.
......@@ -376,10 +391,9 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
bool found = false;
/*
* We are probably in a short-lived expression-evaluation context.
* Switch to longer-lived per-query context.
* Must switch to child query's per-query memory context.
*/
oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory);
oldcontext = MemoryContextSwitchTo(node->sub_estate->es_query_cxt);
if (subLinkType == ANY_SUBLINK ||
subLinkType == ALL_SUBLINK)
......@@ -415,15 +429,18 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
found = true;
/*
* We need to copy the subplan's tuple in case any of the params
* are pass-by-ref type --- the pointers stored in the param
* structs will point at this copied tuple! node->curTuple keeps
* track of the copied tuple for eventual freeing.
* We need to copy the subplan's tuple into our own context,
* in case any of the params are pass-by-ref type --- the pointers
* stored in the param structs will point at this copied tuple!
* node->curTuple keeps track of the copied tuple for eventual
* freeing.
*/
MemoryContextSwitchTo(econtext->ecxt_per_query_memory);
tup = heap_copytuple(tup);
if (node->curTuple)
heap_freetuple(node->curTuple);
node->curTuple = tup;
MemoryContextSwitchTo(node->sub_estate->es_query_cxt);
foreach(lst, subplan->setParam)
{
......@@ -460,7 +477,10 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
if (planstate->plan->extParam == NULL) /* un-correlated ... */
{
ExecEndNode(planstate);
ExecEndPlan(planstate, node->sub_estate);
/* mustn't free context while still in it... */
MemoryContextSwitchTo(oldcontext);
FreeExecutorState(node->sub_estate);
node->needShutdown = false;
}
......@@ -476,7 +496,12 @@ ExecEndSubPlan(SubPlanState *node)
{
if (node->needShutdown)
{
ExecEndNode(node->planstate);
MemoryContext oldcontext;
oldcontext = MemoryContextSwitchTo(node->sub_estate->es_query_cxt);
ExecEndPlan(node->planstate, node->sub_estate);
MemoryContextSwitchTo(oldcontext);
FreeExecutorState(node->sub_estate);
node->needShutdown = false;
}
if (node->curTuple)
......
......@@ -12,7 +12,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubqueryscan.c,v 1.15 2002/12/13 19:45:55 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubqueryscan.c,v 1.16 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -53,6 +53,7 @@ SubqueryNext(SubqueryScanState *node)
EState *estate;
ScanDirection direction;
TupleTableSlot *slot;
MemoryContext oldcontext;
/*
* get information from the estate and scan state
......@@ -66,12 +67,17 @@ SubqueryNext(SubqueryScanState *node)
*/
/*
* get the next tuple from the sub-query
* Get the next tuple from the sub-query. We have to be careful to
* run it in its appropriate memory context.
*/
node->sss_SubEState->es_direction = direction;
oldcontext = MemoryContextSwitchTo(node->sss_SubEState->es_query_cxt);
slot = ExecProcNode(node->subplan);
MemoryContextSwitchTo(oldcontext);
node->ss.ss_ScanTupleSlot = slot;
return slot;
......@@ -106,6 +112,7 @@ ExecInitSubqueryScan(SubqueryScan *node, EState *estate)
SubqueryScanState *subquerystate;
RangeTblEntry *rte;
EState *sp_estate;
MemoryContext oldcontext;
/*
* SubqueryScan should not have any "normal" children.
......@@ -152,9 +159,17 @@ ExecInitSubqueryScan(SubqueryScan *node, EState *estate)
rte = rt_fetch(node->scan.scanrelid, estate->es_range_table);
Assert(rte->rtekind == RTE_SUBQUERY);
/*
* The subquery needs its own EState because it has its own rangetable.
* It shares our Param ID space, however. XXX if rangetable access were
* done differently, the subquery could share our EState, which would
* eliminate some thrashing about in this module...
*/
sp_estate = CreateExecutorState();
subquerystate->sss_SubEState = sp_estate;
oldcontext = MemoryContextSwitchTo(sp_estate->es_query_cxt);
sp_estate->es_range_table = rte->subquery->rtable;
sp_estate->es_param_list_info = estate->es_param_list_info;
sp_estate->es_param_exec_vals = estate->es_param_exec_vals;
......@@ -163,8 +178,13 @@ ExecInitSubqueryScan(SubqueryScan *node, EState *estate)
sp_estate->es_snapshot = estate->es_snapshot;
sp_estate->es_instrument = estate->es_instrument;
/*
* Start up the subplan (this is a very cut-down form of InitPlan())
*/
subquerystate->subplan = ExecInitNode(node->subplan, sp_estate);
MemoryContextSwitchTo(oldcontext);
subquerystate->ss.ss_ScanTupleSlot = NULL;
subquerystate->ss.ps.ps_TupFromTlist = false;
......@@ -197,10 +217,11 @@ ExecCountSlotsSubqueryScan(SubqueryScan *node)
void
ExecEndSubqueryScan(SubqueryScanState *node)
{
MemoryContext oldcontext;
/*
* Free the projection info and the scan attribute info
* Free the exprcontext
*/
ExecFreeProjectionInfo(&node->ss.ps);
ExecFreeExprContext(&node->ss.ps);
/*
......@@ -211,15 +232,13 @@ ExecEndSubqueryScan(SubqueryScanState *node)
/*
* close down subquery
*/
ExecEndNode(node->subplan);
oldcontext = MemoryContextSwitchTo(node->sss_SubEState->es_query_cxt);
/*
* clean up subquery's tuple table
*/
node->ss.ss_ScanTupleSlot = NULL;
ExecDropTupleTable(node->sss_SubEState->es_tupleTable, true);
ExecEndPlan(node->subplan, node->sss_SubEState);
/* XXX we seem to be leaking the sub-EState... */
MemoryContextSwitchTo(oldcontext);
FreeExecutorState(node->sss_SubEState);
}
/* ----------------------------------------------------------------
......@@ -232,12 +251,17 @@ void
ExecSubqueryReScan(SubqueryScanState *node, ExprContext *exprCtxt)
{
EState *estate;
MemoryContext oldcontext;
estate = node->ss.ps.state;
oldcontext = MemoryContextSwitchTo(node->sss_SubEState->es_query_cxt);
/*
* ExecReScan doesn't know about my subplan, so I have to do
* changed-parameter signaling myself.
* changed-parameter signaling myself. This is just as well,
* because the subplan has its own memory context in which its
* chgParam lists live.
*/
if (node->ss.ps.chgParam != NULL)
SetChangedParamList(node->subplan, node->ss.ps.chgParam);
......@@ -249,5 +273,7 @@ ExecSubqueryReScan(SubqueryScanState *node, ExprContext *exprCtxt)
if (node->subplan->chgParam == NULL)
ExecReScan(node->subplan, NULL);
MemoryContextSwitchTo(oldcontext);
node->ss.ss_ScanTupleSlot = NULL;
}
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeTidscan.c,v 1.29 2002/12/13 19:45:56 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeTidscan.c,v 1.30 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -278,19 +278,8 @@ void
ExecEndTidScan(TidScanState *node)
{
/*
* extract information from the node
* Free the exprcontext
*/
if (node && node->tss_TidList)
pfree(node->tss_TidList);
/*
* Free the projection info and the scan attribute info
*
* Note: we don't ExecFreeResultType(scanstate) because the rule manager
* depends on the tupType returned by ExecMain(). So for now, this is
* freed at end-transaction time. -cim 6/2/91
*/
ExecFreeProjectionInfo(&node->ss.ps);
ExecFreeExprContext(&node->ss.ps);
/*
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeUnique.c,v 1.35 2002/12/05 15:50:34 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeUnique.c,v 1.36 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -194,15 +194,10 @@ ExecEndUnique(UniqueState *node)
{
/* clean up tuple table */
ExecClearTuple(node->ps.ps_ResultTupleSlot);
if (node->priorTuple != NULL)
{
heap_freetuple(node->priorTuple);
node->priorTuple = NULL;
}
ExecEndNode(outerPlanState(node));
MemoryContextDelete(node->tempContext);
ExecEndNode(outerPlanState(node));
}
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/spi.c,v 1.79 2002/12/05 15:50:34 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/spi.c,v 1.80 2002/12/15 16:17:46 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -1287,23 +1287,23 @@ _SPI_pquery(QueryDesc *queryDesc, bool runit, int tcount)
elog(FATAL, "SPI_select: # of processed tuples check failed");
}
ExecutorEnd(queryDesc);
#ifdef SPI_EXECUTOR_STATS
if (ShowExecutorStats)
ShowUsage("SPI EXECUTOR STATS");
#endif
if (dest == SPI)
{
SPI_processed = _SPI_current->processed;
SPI_lastoid = save_lastoid;
SPI_tuptable = _SPI_current->tuptable;
}
queryDesc->dest = dest;
return res;
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
#ifdef SPI_EXECUTOR_STATS
if (ShowExecutorStats)
ShowUsage("SPI EXECUTOR STATS");
#endif
return res;
}
/*
......
......@@ -9,7 +9,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/path/indxpath.c,v 1.128 2002/12/13 19:45:56 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/path/indxpath.c,v 1.129 2002/12/15 16:17:49 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -1132,7 +1132,8 @@ pred_test_simple_clause(Expr *predicate, Node *clause)
HeapTuple tuple;
ScanKeyData entry[1];
Form_pg_amop aform;
ExprContext *econtext;
EState *estate;
MemoryContext oldcontext;
/* First try the equal() test */
if (equal((Node *) predicate, clause))
......@@ -1267,20 +1268,33 @@ pred_test_simple_clause(Expr *predicate, Node *clause)
ReleaseSysCache(tuple);
/*
* 5. Evaluate the test
* 5. Evaluate the test. For this we need an EState.
*/
estate = CreateExecutorState();
/* We can use the estate's working context to avoid memory leaks. */
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
/* Build expression tree */
test_expr = make_opclause(test_op,
BOOLOID,
false,
(Expr *) clause_const,
(Expr *) pred_const);
set_opfuncid((OpExpr *) test_expr);
test_exprstate = ExecInitExpr(test_expr, NULL);
econtext = MakeExprContext(NULL, CurrentMemoryContext);
test_result = ExecEvalExprSwitchContext(test_exprstate, econtext,
/* Prepare it for execution */
test_exprstate = ExecPrepareExpr(test_expr, estate);
/* And execute it. */
test_result = ExecEvalExprSwitchContext(test_exprstate,
GetPerTupleExprContext(estate),
&isNull, NULL);
FreeExprContext(econtext);
/* Get back to outer memory context */
MemoryContextSwitchTo(oldcontext);
/* Release all the junk we just created */
FreeExecutorState(estate);
if (isNull)
{
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/clauses.c,v 1.119 2002/12/14 00:17:59 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/clauses.c,v 1.120 2002/12/15 16:17:50 tgl Exp $
*
* HISTORY
* AUTHOR DATE MAJOR EVENT
......@@ -1684,7 +1684,8 @@ evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
bool has_null_input = false;
FuncExpr *newexpr;
ExprState *newexprstate;
ExprContext *econtext;
EState *estate;
MemoryContext oldcontext;
Datum const_val;
bool const_is_null;
List *arg;
......@@ -1729,7 +1730,14 @@ evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
*
* We use the executor's routine ExecEvalExpr() to avoid duplication of
* code and ensure we get the same result as the executor would get.
*
* To use the executor, we need an EState.
*/
estate = CreateExecutorState();
/* We can use the estate's working context to avoid memory leaks. */
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
/*
* Build a new FuncExpr node containing the already-simplified arguments.
*/
newexpr = makeNode(FuncExpr);
......@@ -1739,27 +1747,35 @@ evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
newexpr->funcformat = COERCE_EXPLICIT_CALL; /* doesn't matter */
newexpr->args = args;
/* Get info needed about result datatype */
get_typlenbyval(result_typeid, &resultTypLen, &resultTypByVal);
/*
* Prepare it for execution.
*/
newexprstate = ExecPrepareExpr((Expr *) newexpr, estate);
/*
* It is OK to use a dummy econtext because none of the
* And evaluate it.
*
* It is OK to use a default econtext because none of the
* ExecEvalExpr() code used in this situation will use econtext. That
* might seem fortuitous, but it's not so unreasonable --- a constant
* expression does not depend on context, by definition, n'est ce pas?
*/
econtext = MakeExprContext(NULL, CurrentMemoryContext);
const_val = ExecEvalExprSwitchContext(newexprstate,
GetPerTupleExprContext(estate),
&const_is_null, NULL);
newexprstate = ExecInitExpr((Expr *) newexpr, NULL);
/* Get info needed about result datatype */
get_typlenbyval(result_typeid, &resultTypLen, &resultTypByVal);
const_val = ExecEvalExprSwitchContext(newexprstate, econtext,
&const_is_null, NULL);
/* Get back to outer memory context */
MemoryContextSwitchTo(oldcontext);
/* Must copy result out of sub-context used by expression eval */
if (!const_is_null)
const_val = datumCopy(const_val, resultTypByVal, resultTypLen);
FreeExprContext(econtext);
/* Release all the junk we just created */
FreeExecutorState(estate);
/*
* Make the constant result node.
......
......@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/tcop/pquery.c,v 1.57 2002/12/05 15:50:35 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/tcop/pquery.c,v 1.58 2002/12/15 16:17:52 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -52,6 +52,18 @@ CreateQueryDesc(Query *parsetree,
return qd;
}
/*
* FreeQueryDesc
*/
void
FreeQueryDesc(QueryDesc *qdesc)
{
/* Can't be a live query */
Assert(qdesc->estate == NULL);
/* Only the QueryDesc itself need be freed */
pfree(qdesc);
}
/* ----------------
* PreparePortal
* ----------------
......@@ -152,9 +164,8 @@ ProcessQuery(Query *parsetree,
* QueryDesc */
/*
* We stay in portal's memory context for now, so that query desc,
* exec state, and plan startup info are also allocated in the portal
* context.
* We stay in portal's memory context for now, so that query desc
* is also allocated in the portal context.
*/
}
......@@ -231,4 +242,6 @@ ProcessQuery(Query *parsetree,
* Now, we close down all the scans and free allocated resources.
*/
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
}
......@@ -8,7 +8,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: execdesc.h,v 1.21 2002/12/05 15:50:36 tgl Exp $
* $Id: execdesc.h,v 1.22 2002/12/15 16:17:54 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -50,4 +50,6 @@ extern QueryDesc *CreateQueryDesc(Query *parsetree, Plan *plantree,
ParamListInfo params,
bool doInstrument);
extern void FreeQueryDesc(QueryDesc *qdesc);
#endif /* EXECDESC_H */
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: executor.h,v 1.83 2002/12/14 00:17:59 tgl Exp $
* $Id: executor.h,v 1.84 2002/12/15 16:17:54 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -54,8 +54,8 @@ extern void ExecutorStart(QueryDesc *queryDesc);
extern TupleTableSlot *ExecutorRun(QueryDesc *queryDesc,
ScanDirection direction, long count);
extern void ExecutorEnd(QueryDesc *queryDesc);
extern EState *CreateExecutorState(void);
extern void ExecCheckRTPerms(List *rangeTable, CmdType operation);
extern void ExecEndPlan(PlanState *planstate, EState *estate);
extern void ExecConstraints(const char *caller, ResultRelInfo *resultRelInfo,
TupleTableSlot *slot, EState *estate);
extern TupleTableSlot *EvalPlanQual(EState *estate, Index rti,
......@@ -93,6 +93,7 @@ extern Datum ExecEvalExprSwitchContext(ExprState *expression, ExprContext *econt
bool *isNull, ExprDoneCond *isDone);
extern ExprState *ExecInitExpr(Expr *node, PlanState *parent);
extern SubPlanState *ExecInitExprInitPlan(SubPlan *node, PlanState *parent);
extern ExprState *ExecPrepareExpr(Expr *node, EState *estate);
extern bool ExecQual(List *qual, ExprContext *econtext, bool resultForNull);
extern int ExecTargetListLength(List *targetlist);
extern int ExecCleanTargetListLength(List *targetlist);
......@@ -157,23 +158,9 @@ extern void end_tup_output(TupOutputState *tstate);
/*
* prototypes from functions in execUtils.c
*/
extern void ResetTupleCount(void);
extern void ExecAssignExprContext(EState *estate, PlanState *planstate);
extern void ExecAssignResultType(PlanState *planstate,
TupleDesc tupDesc, bool shouldFree);
extern void ExecAssignResultTypeFromOuterPlan(PlanState *planstate);
extern void ExecAssignResultTypeFromTL(PlanState *planstate);
extern TupleDesc ExecGetResultType(PlanState *planstate);
extern void ExecAssignProjectionInfo(PlanState *planstate);
extern void ExecFreeProjectionInfo(PlanState *planstate);
extern void ExecFreeExprContext(PlanState *planstate);
extern TupleDesc ExecGetScanType(ScanState *scanstate);
extern void ExecAssignScanType(ScanState *scanstate,
TupleDesc tupDesc, bool shouldFree);
extern void ExecAssignScanTypeFromOuterPlan(ScanState *scanstate);
extern ExprContext *MakeExprContext(TupleTableSlot *slot,
MemoryContext queryContext);
extern EState *CreateExecutorState(void);
extern void FreeExecutorState(EState *estate);
extern ExprContext *CreateExprContext(EState *estate);
extern void FreeExprContext(ExprContext *econtext);
#define ResetExprContext(econtext) \
......@@ -197,6 +184,19 @@ extern ExprContext *MakePerTupleExprContext(EState *estate);
ResetExprContext((estate)->es_per_tuple_exprcontext); \
} while (0)
extern void ExecAssignExprContext(EState *estate, PlanState *planstate);
extern void ExecAssignResultType(PlanState *planstate,
TupleDesc tupDesc, bool shouldFree);
extern void ExecAssignResultTypeFromOuterPlan(PlanState *planstate);
extern void ExecAssignResultTypeFromTL(PlanState *planstate);
extern TupleDesc ExecGetResultType(PlanState *planstate);
extern void ExecAssignProjectionInfo(PlanState *planstate);
extern void ExecFreeExprContext(PlanState *planstate);
extern TupleDesc ExecGetScanType(ScanState *scanstate);
extern void ExecAssignScanType(ScanState *scanstate,
TupleDesc tupDesc, bool shouldFree);
extern void ExecAssignScanTypeFromOuterPlan(ScanState *scanstate);
extern void ExecOpenIndices(ResultRelInfo *resultRelInfo);
extern void ExecCloseIndices(ResultRelInfo *resultRelInfo);
extern void ExecInsertIndexTuples(TupleTableSlot *slot, ItemPointer tupleid,
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: execnodes.h,v 1.85 2002/12/14 00:17:59 tgl Exp $
* $Id: execnodes.h,v 1.86 2002/12/15 16:17:56 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -81,14 +81,14 @@ typedef struct ExprContext_CB
* context.
*
* There are two memory contexts associated with an ExprContext:
* * ecxt_per_query_memory is a relatively long-lived context (such as
* TransactionCommandContext); typically it's the same context the
* ExprContext node itself is allocated in. This context can be
* used for purposes such as storing function call cache info.
* * ecxt_per_query_memory is a query-lifespan context, typically the same
* context the ExprContext node itself is allocated in. This context
* can be used for purposes such as storing function call cache info.
* * ecxt_per_tuple_memory is a short-term context for expression results.
* As the name suggests, it will typically be reset once per tuple,
* before we begin to evaluate expressions for that tuple. Each
* ExprContext normally has its very own per-tuple memory context.
*
* CurrentMemoryContext should be set to ecxt_per_tuple_memory before
* calling ExecEvalExpr() --- see ExecEvalExprSwitchContext().
* ----------------
......@@ -118,6 +118,9 @@ typedef struct ExprContext
Datum domainValue_datum;
bool domainValue_isNull;
/* Link to containing EState */
struct EState *ecxt_estate;
/* Functions to call back when ExprContext is shut down */
ExprContext_CB *ecxt_callbacks;
} ExprContext;
......@@ -277,45 +280,43 @@ typedef struct ResultRelInfo
/* ----------------
* EState information
*
* direction direction of the scan
*
* snapshot time qual to use
*
* range_table array of scan relation information
*
* result_relation information for insert/update/delete queries
*
* into_relation_descriptor relation being retrieved "into"
*
* param_list_info information about Param values
*
* tupleTable this is a pointer to an array
* of pointers to tuples used by
* the executor at any given moment.
* Master working state for an Executor invocation
* ----------------
*/
typedef struct EState
{
NodeTag type;
ScanDirection es_direction;
Snapshot es_snapshot;
List *es_range_table;
/* Basic state for all query types: */
ScanDirection es_direction; /* current scan direction */
Snapshot es_snapshot; /* time qual to use */
List *es_range_table; /* List of RangeTableEntrys */
/* Info about target table for insert/update/delete queries: */
ResultRelInfo *es_result_relations; /* array of ResultRelInfos */
int es_num_result_relations; /* length of array */
ResultRelInfo *es_result_relation_info; /* currently active array
* elt */
JunkFilter *es_junkFilter; /* currently active junk filter */
Relation es_into_relation_descriptor;
Relation es_into_relation_descriptor; /* for SELECT INTO */
/* Parameter info: */
ParamListInfo es_param_list_info; /* values of external params */
ParamExecData *es_param_exec_vals; /* values of internal params */
TupleTable es_tupleTable;
/* Other working state: */
MemoryContext es_query_cxt; /* per-query context in which EState lives */
TupleTable es_tupleTable; /* Array of TupleTableSlots */
uint32 es_processed; /* # of tuples processed */
Oid es_lastoid; /* last oid processed (by INSERT) */
List *es_rowMark; /* not good place, but there is no other */
MemoryContext es_query_cxt; /* per-query context in which EState lives */
bool es_instrument; /* true requests runtime instrumentation */
List *es_exprcontexts; /* List of ExprContexts within EState */
/*
* this ExprContext is for per-output-tuple operations, such as
* constraint checks and index-value computations. It will be reset
......@@ -457,6 +458,7 @@ typedef struct BoolExprState
typedef struct SubPlanState
{
ExprState xprstate;
EState *sub_estate; /* subselect plan has its own EState */
struct PlanState *planstate; /* subselect plan's state tree */
bool needShutdown; /* TRUE = need to shutdown subplan */
HeapTuple curTuple; /* copy of most recent tuple from subplan */
......
......@@ -3,7 +3,7 @@
* procedural language
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.74 2002/12/13 19:46:01 tgl Exp $
* $Header: /cvsroot/pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.75 2002/12/15 16:17:58 tgl Exp $
*
* This software is copyrighted by Jan Wieck - Hamburg.
*
......@@ -3227,7 +3227,6 @@ exec_eval_simple_expr(PLpgSQL_execstate * estate,
bool *isNull,
Oid *rettype)
{
_SPI_plan *spi_plan = (_SPI_plan *) expr->plan;
Datum retval;
PLpgSQL_var *var;
PLpgSQL_rec *rec;
......@@ -3242,14 +3241,11 @@ exec_eval_simple_expr(PLpgSQL_execstate * estate,
ParamListInfo paramLI;
/*
* Create a simple expression context to hold the arguments.
*
* NOTE: we pass the SPI plan's context as the query-lifetime context for
* function cache nodes and suchlike allocations. This is appropriate
* because that's where the expression tree itself is, and the
* function cache nodes must live as long as it does.
* Create an expression context to hold the arguments and the result
* of this expression evaluation. This must be a child of the EState
* we created in the SPI plan's context.
*/
econtext = MakeExprContext(NULL, spi_plan->plancxt);
econtext = CreateExprContext(expr->plan_simple_estate);
/*
* Param list can live in econtext's temporary memory context.
......@@ -3691,13 +3687,20 @@ exec_simple_check_plan(PLpgSQL_expr * expr)
return;
/*
* Yes - this is a simple expression. Prepare to execute it, and
* stash away the result type. Put the expression state tree in the
* plan context so it will have appropriate lifespan.
* Yes - this is a simple expression. Prepare to execute it.
* We need an EState and an expression state tree, which we'll put
* into the plan context so they will have appropriate lifespan.
*/
oldcontext = MemoryContextSwitchTo(spi_plan->plancxt);
expr->plan_simple_expr = ExecInitExpr(tle->expr, NULL);
expr->plan_simple_estate = CreateExecutorState();
expr->plan_simple_expr = ExecPrepareExpr(tle->expr,
expr->plan_simple_estate);
MemoryContextSwitchTo(oldcontext);
/* Also stash away the expression result type */
expr->plan_simple_type = exprType((Node *) tle->expr);
}
......
......@@ -3,7 +3,7 @@
* procedural language
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/pl/plpgsql/src/plpgsql.h,v 1.30 2002/12/13 19:46:01 tgl Exp $
* $Header: /cvsroot/pgsql/src/pl/plpgsql/src/plpgsql.h,v 1.31 2002/12/15 16:17:59 tgl Exp $
*
* This software is copyrighted by Jan Wieck - Hamburg.
*
......@@ -166,6 +166,7 @@ typedef struct
char *query;
void *plan;
ExprState *plan_simple_expr;
EState *plan_simple_estate;
Oid plan_simple_type;
Oid *plan_argtypes;
int nparams;
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment