diff options
Diffstat (limited to 'src/backend/executor')
63 files changed, 13677 insertions, 7935 deletions
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile index 6625d56b97..fef60fb4c9 100644 --- a/src/backend/executor/Makefile +++ b/src/backend/executor/Makefile @@ -12,19 +12,23 @@ subdir = src/backend/executor top_builddir = ../../.. include $(top_builddir)/src/Makefile.global -OBJS = execAmi.o execCurrent.o execGrouping.o execIndexing.o execJunk.o \ - execMain.o execParallel.o execProcnode.o execQual.o \ - execScan.o execTuples.o \ +OBJS = execAmi.o execCurrent.o execExpr.o execExprInterp.o \ + execGrouping.o execIndexing.o execJunk.o \ + execMain.o execParallel.o execProcnode.o \ + execReplication.o execScan.o execSRF.o execTuples.o \ execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \ nodeBitmapAnd.o nodeBitmapOr.o \ - nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeGather.o \ + nodeBitmapHeapscan.o nodeBitmapIndexscan.o \ + nodeCustom.o nodeFunctionscan.o nodeGather.o \ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \ - nodeLimit.o nodeLockRows.o \ + nodeLimit.o nodeLockRows.o nodeGatherMerge.o \ nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \ - nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \ + nodeNestloop.o nodeProjectSet.o nodeRecursiveunion.o nodeResult.o \ nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \ - nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \ + nodeValuesscan.o \ + nodeCtescan.o nodeNamedtuplestorescan.o nodeWorktablescan.o \ nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \ - nodeForeignscan.o nodeWindowAgg.o producerReceiver.o tstoreReceiver.o tqueue.o spi.o + nodeForeignscan.o nodeWindowAgg.o producerReceiver.o tstoreReceiver.o tqueue.o spi.o \ + nodeTableFuncscan.o include $(top_srcdir)/src/backend/common.mk diff --git a/src/backend/executor/README b/src/backend/executor/README index 8afa1e3e4a..a0045067fb 100644 --- a/src/backend/executor/README +++ b/src/backend/executor/README @@ -44,21 +44,171 @@ Plan Trees and State Trees -------------------------- The plan tree delivered by the planner contains a tree of Plan nodes (struct -types derived from struct Plan). Each Plan node may have expression trees -associated with it, to represent its target list, qualification conditions, -etc. During executor startup we build a parallel tree of identical structure -containing executor state nodes --- every plan and expression node type has -a corresponding executor state node type. Each node in the state tree has a -pointer to its corresponding node in the plan tree, plus executor state data -as needed to implement that node type. This arrangement allows the plan -tree to be completely read-only as far as the executor is concerned: all data -that is modified during execution is in the state tree. Read-only plan trees -make life much simpler for plan caching and reuse. +types derived from struct Plan). During executor startup we build a parallel +tree of identical structure containing executor state nodes --- every plan +node type has a corresponding executor state node type. Each node in the +state tree has a pointer to its corresponding node in the plan tree, plus +executor state data as needed to implement that node type. This arrangement +allows the plan tree to be completely read-only so far as the executor is +concerned: all data that is modified during execution is in the state tree. +Read-only plan trees make life much simpler for plan caching and reuse. + +Each Plan node may have expression trees associated with it, to represent +its target list, qualification conditions, etc. These trees are also +read-only to the executor, but the executor state for expression evaluation +does not mirror the Plan expression's tree shape, as explained below. +Rather, there's just one ExprState node per expression tree, although this +may have sub-nodes for some complex expression node types. Altogether there are four classes of nodes used in these trees: Plan nodes, -their corresponding PlanState nodes, Expr nodes, and their corresponding -ExprState nodes. (Actually, there are also List nodes, which are used as -"glue" in all four kinds of tree.) +their corresponding PlanState nodes, Expr nodes, and ExprState nodes. +(Actually, there are also List nodes, which are used as "glue" in all +three tree-based representations.) + + +Expression Trees and ExprState nodes +------------------------------------ + +Expression trees, in contrast to Plan trees, are not mirrored into a +corresponding tree of state nodes. Instead each separately executable +expression tree (e.g. a Plan's qual or targetlist) is represented by one +ExprState node. The ExprState node contains the information needed to +evaluate the expression in a compact, linear form. That compact form is +stored as a flat array in ExprState->steps[] (an array of ExprEvalStep, +not ExprEvalStep *). + +The reasons for choosing such a representation include: +- commonly the amount of work needed to evaluate one Expr-type node is + small enough that the overhead of having to perform a tree-walk + during evaluation is significant. +- the flat representation can be evaluated non-recursively within a single + function, reducing stack depth and function call overhead. +- such a representation is usable both for fast interpreted execution, + and for compiling into native code. + +The Plan-tree representation of an expression is compiled into an +ExprState node by ExecInitExpr(). As much complexity as possible should +be handled by ExecInitExpr() (and helpers), instead of execution time +where both interpreted and compiled versions would need to deal with the +complexity. Besides duplicating effort between execution approaches, +runtime initialization checks also have a small but noticeable cost every +time the expression is evaluated. Therefore, we allow ExecInitExpr() to +precompute information that we do not expect to vary across execution of a +single query, for example the set of CHECK constraint expressions to be +applied to a domain type. This could not be done at plan time without +greatly increasing the number of events that require plan invalidation. +(Previously, some information of this kind was rechecked on each +expression evaluation, but that seems like unnecessary overhead.) + + +Expression Initialization +------------------------- + +During ExecInitExpr() and similar routines, Expr trees are converted +into the flat representation. Each Expr node might be represented by +zero, one, or more ExprEvalSteps. + +Each ExprEvalStep's work is determined by its opcode (of enum ExprEvalOp) +and it stores the result of its work into the Datum variable and boolean +null flag variable pointed to by ExprEvalStep->resvalue/resnull. +Complex expressions are performed by chaining together several steps. +For example, "a + b" (one OpExpr, with two Var expressions) would be +represented as two steps to fetch the Var values, and one step for the +evaluation of the function underlying the + operator. The steps for the +Vars would have their resvalue/resnull pointing directly to the appropriate +arg[] and argnull[] array elements in the FunctionCallInfoData struct that +is used by the function evaluation step, thus avoiding extra work to copy +the result values around. + +The last entry in a completed ExprState->steps array is always an +EEOP_DONE step; this removes the need to test for end-of-array while +iterating. Also, if the expression contains any variable references (to +user columns of the ExprContext's INNER, OUTER, or SCAN tuples), the steps +array begins with EEOP_*_FETCHSOME steps that ensure that the relevant +tuples have been deconstructed to make the required columns directly +available (cf. slot_getsomeattrs()). This allows individual Var-fetching +steps to be little more than an array lookup. + +Most of ExecInitExpr()'s work is done by the recursive function +ExecInitExprRec() and its subroutines. ExecInitExprRec() maps one Expr +node into the steps required for execution, recursing as needed for +sub-expressions. + +Each ExecInitExprRec() call has to specify where that subexpression's +results are to be stored (via the resv/resnull parameters). This allows +the above scenario of evaluating a (sub-)expression directly into +fcinfo->arg/argnull, but also requires some care: target Datum/isnull +variables may not be shared with another ExecInitExprRec() unless the +results are only needed by steps executing before further usages of those +target Datum/isnull variables. Due to the non-recursiveness of the +ExprEvalStep representation that's usually easy to guarantee. + +ExecInitExprRec() pushes new operations into the ExprState->steps array +using ExprEvalPushStep(). To keep the steps as a consecutively laid out +array, ExprEvalPushStep() has to repalloc the entire array when there's +not enough space. Because of that it is *not* allowed to point directly +into any of the steps during expression initialization. Therefore, the +resv/resnull for a subexpression usually point to some storage that is +palloc'd separately from the steps array. For instance, the +FunctionCallInfoData for a function call step is separately allocated +rather than being part of the ExprEvalStep array. The overall result +of a complete expression is typically returned into the resvalue/resnull +fields of the ExprState node itself. + +Some steps, e.g. boolean expressions, allow skipping evaluation of +certain subexpressions. In the flat representation this amounts to +jumping to some later step rather than just continuing consecutively +with the next step. The target for such a jump is represented by +the integer index in the ExprState->steps array of the step to execute +next. (Compare the EEO_NEXT and EEO_JUMP macros in execExprInterp.c.) + +Typically, ExecInitExprRec() has to push a jumping step into the steps +array, then recursively generate steps for the subexpression that might +get skipped over, then go back and fix up the jump target index using +the now-known length of the subexpression's steps. This is handled by +adjust_jumps lists in execExpr.c. + +The last step in constructing an ExprState is to apply ExecReadyExpr(), +which readies it for execution using whichever execution method has been +selected. + + +Expression Evaluation +--------------------- + +To allow for different methods of expression evaluation, and for +better branch/jump target prediction, expressions are evaluated by +calling ExprState->evalfunc (via ExprEvalExpr() and friends). + +ExprReadyExpr() can choose the method of interpretation by setting +evalfunc to an appropriate function. The default execution function, +ExecInterpExpr, is implemented in execExprInterp.c; see its header +comment for details. Special-case evalfuncs are used for certain +especially-simple expressions. + +Note that a lot of the more complex expression evaluation steps, which are +less performance-critical than the simpler ones, are implemented as +separate functions outside the fast-path of expression execution, allowing +their implementation to be shared between interpreted and compiled +expression evaluation. This means that these helper functions are not +allowed to perform expression step dispatch themselves, as the method of +dispatch will vary based on the caller. The helpers therefore cannot call +for the execution of subexpressions; all subexpression results they need +must be computed by earlier steps. And dispatch to the following +expression step must be performed after returning from the helper. + + +Targetlist Evaluation +--------------------- + +ExecBuildProjectionInfo builds an ExprState that has the effect of +evaluating a targetlist into ExprState->resultslot. A generic targetlist +expression is executed by evaluating it as discussed above (storing the +result into the ExprState's resvalue/resnull fields) and then using an +EEOP_ASSIGN_TMP step to move the result into the appropriate tts_values[] +and tts_isnull[] array elements of the result slot. There are special +fast-path step types (EEOP_ASSIGN_*_VAR) to handle targetlist entries that +are simple Vars using only one step instead of two. Memory Management @@ -195,8 +345,7 @@ the entire row value in the join output row. We disallow set-returning functions in the targetlist of SELECT FOR UPDATE, so as to ensure that at most one tuple can be returned for any particular set of scan tuples. Otherwise we'd get duplicates due to the original -query returning the same set of scan tuples multiple times. (Note: there -is no explicit prohibition on SRFs in UPDATE, but the net effect will be -that only the first result row of an SRF counts, because all subsequent -rows will result in attempts to re-update an already updated target row. -This is historical behavior and seems not worth changing.) +query returning the same set of scan tuples multiple times. Likewise, +SRFs are disallowed in an UPDATE's targetlist. There, they would have the +effect of the same row being updated multiple times, which is not very +useful --- and updates after the first would have no effect anyway. diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c index 2cb83d75a7..b802ad6956 100644 --- a/src/backend/executor/execAmi.c +++ b/src/backend/executor/execAmi.c @@ -4,7 +4,7 @@ * miscellaneous executor access method routines * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * src/backend/executor/execAmi.c @@ -39,7 +39,9 @@ #include "executor/nodeMergeAppend.h" #include "executor/nodeMergejoin.h" #include "executor/nodeModifyTable.h" +#include "executor/nodeNamedtuplestorescan.h" #include "executor/nodeNestloop.h" +#include "executor/nodeProjectSet.h" #include "executor/nodeRecursiveunion.h" #include "executor/nodeResult.h" #include "executor/nodeSamplescan.h" @@ -48,6 +50,7 @@ #include "executor/nodeSort.h" #include "executor/nodeSubplan.h" #include "executor/nodeSubqueryscan.h" +#include "executor/nodeTableFuncscan.h" #include "executor/nodeTidscan.h" #include "executor/nodeUnique.h" #include "executor/nodeValuesscan.h" @@ -62,7 +65,6 @@ #endif -static bool TargetListSupportsBackwardScan(List *targetlist); static bool IndexSupportsBackwardScan(Oid indexid); @@ -123,7 +125,7 @@ ExecReScan(PlanState *node) UpdateChangedParamSet(node->righttree, node->chgParam); } - /* Shut down any SRFs in the plan node's targetlist */ + /* Call expression callbacks */ if (node->ps_ExprContext) ReScanExprContext(node->ps_ExprContext); @@ -134,6 +136,10 @@ ExecReScan(PlanState *node) ExecReScanResult((ResultState *) node); break; + case T_ProjectSetState: + ExecReScanProjectSet((ProjectSetState *) node); + break; + case T_ModifyTableState: ExecReScanModifyTable((ModifyTableState *) node); break; @@ -198,6 +204,10 @@ ExecReScan(PlanState *node) ExecReScanFunctionScan((FunctionScanState *) node); break; + case T_TableFuncScanState: + ExecReScanTableFuncScan((TableFuncScanState *) node); + break; + case T_ValuesScanState: ExecReScanValuesScan((ValuesScanState *) node); break; @@ -206,6 +216,10 @@ ExecReScan(PlanState *node) ExecReScanCteScan((CteScanState *) node); break; + case T_NamedTuplestoreScanState: + ExecReScanNamedTuplestoreScan((NamedTuplestoreScanState *) node); + break; + case T_WorkTableScanState: ExecReScanWorkTableScan((WorkTableScanState *) node); break; @@ -408,11 +422,13 @@ ExecSupportsMarkRestore(Path *pathnode) return true; case T_CustomScan: - Assert(IsA(pathnode, CustomPath)); - if (((CustomPath *) pathnode)->flags & CUSTOMPATH_SUPPORT_MARK_RESTORE) - return true; - return false; + { + CustomPath *customPath = castNode(CustomPath, pathnode); + if (customPath->flags & CUSTOMPATH_SUPPORT_MARK_RESTORE) + return true; + return false; + } case T_Result: /* @@ -464,8 +480,7 @@ ExecSupportsBackwardScan(Plan *node) { case T_Result: if (outerPlan(node) != NULL) - return ExecSupportsBackwardScan(outerPlan(node)) && - TargetListSupportsBackwardScan(node->targetlist); + return ExecSupportsBackwardScan(outerPlan(node)); else return false; @@ -482,13 +497,6 @@ ExecSupportsBackwardScan(Plan *node) return true; } - case T_SeqScan: - case T_TidScan: - case T_FunctionScan: - case T_ValuesScan: - case T_CteScan: - return TargetListSupportsBackwardScan(node->targetlist); - case T_SampleScan: /* Simplify life for tablesample methods by disallowing this */ return false; @@ -497,35 +505,34 @@ ExecSupportsBackwardScan(Plan *node) return false; case T_IndexScan: - return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) && - TargetListSupportsBackwardScan(node->targetlist); + return IndexSupportsBackwardScan(((IndexScan *) node)->indexid); case T_IndexOnlyScan: - return IndexSupportsBackwardScan(((IndexOnlyScan *) node)->indexid) && - TargetListSupportsBackwardScan(node->targetlist); + return IndexSupportsBackwardScan(((IndexOnlyScan *) node)->indexid); case T_SubqueryScan: - return ExecSupportsBackwardScan(((SubqueryScan *) node)->subplan) && - TargetListSupportsBackwardScan(node->targetlist); + return ExecSupportsBackwardScan(((SubqueryScan *) node)->subplan); case T_CustomScan: { uint32 flags = ((CustomScan *) node)->flags; - if ((flags & CUSTOMPATH_SUPPORT_BACKWARD_SCAN) && - TargetListSupportsBackwardScan(node->targetlist)) + if (flags & CUSTOMPATH_SUPPORT_BACKWARD_SCAN) return true; } return false; + case T_SeqScan: + case T_TidScan: + case T_FunctionScan: + case T_ValuesScan: + case T_CteScan: case T_Material: case T_Sort: - /* these don't evaluate tlist */ return true; case T_LockRows: case T_Limit: - /* these don't evaluate tlist */ return ExecSupportsBackwardScan(outerPlan(node)); default: @@ -534,18 +541,6 @@ ExecSupportsBackwardScan(Plan *node) } /* - * If the tlist contains set-returning functions, we can't support backward - * scan, because the TupFromTlist code is direction-ignorant. - */ -static bool -TargetListSupportsBackwardScan(List *targetlist) -{ - if (expression_returns_set((Node *) targetlist)) - return false; - return true; -} - -/* * An IndexScan or IndexOnlyScan node supports backward scan only if the * index's AM does. */ @@ -589,7 +584,9 @@ ExecMaterializesOutput(NodeTag plantype) { case T_Material: case T_FunctionScan: + case T_TableFuncScan: case T_CteScan: + case T_NamedTuplestoreScan: case T_WorkTableScan: case T_Sort: return true; diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c index 757ea8dddc..0224b9e4af 100644 --- a/src/backend/executor/execCurrent.c +++ b/src/backend/executor/execCurrent.c @@ -4,7 +4,7 @@ * executor support for WHERE CURRENT OF cursor * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * src/backend/executor/execCurrent.c diff --git a/src/backend/executor/execExpr.c b/src/backend/executor/execExpr.c new file mode 100644 index 0000000000..fe12326336 --- /dev/null +++ b/src/backend/executor/execExpr.c @@ -0,0 +1,2677 @@ +/*------------------------------------------------------------------------- + * + * execExpr.c + * Expression evaluation infrastructure. + * + * During executor startup, we compile each expression tree (which has + * previously been processed by the parser and planner) into an ExprState, + * using ExecInitExpr() et al. This converts the tree into a flat array + * of ExprEvalSteps, which may be thought of as instructions in a program. + * At runtime, we'll execute steps, starting with the first, until we reach + * an EEOP_DONE opcode. + * + * This file contains the "compilation" logic. It is independent of the + * specific execution technology we use (switch statement, computed goto, + * JIT compilation, etc). + * + * See src/backend/executor/README for some background, specifically the + * "Expression Trees and ExprState nodes", "Expression Initialization", + * and "Expression Evaluation" sections. + * + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * + * IDENTIFICATION + * src/backend/executor/execExpr.c + * + *------------------------------------------------------------------------- + */ +#include "postgres.h" + +#include "access/nbtree.h" +#include "catalog/objectaccess.h" +#include "catalog/pg_type.h" +#include "executor/execExpr.h" +#include "executor/nodeSubplan.h" +#include "funcapi.h" +#include "miscadmin.h" +#include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" +#include "optimizer/clauses.h" +#include "optimizer/planner.h" +#include "pgstat.h" +#include "utils/builtins.h" +#include "utils/lsyscache.h" +#include "utils/typcache.h" + + +typedef struct LastAttnumInfo +{ + AttrNumber last_inner; + AttrNumber last_outer; + AttrNumber last_scan; +} LastAttnumInfo; + +static void ExecReadyExpr(ExprState *state); +static void ExecInitExprRec(Expr *node, PlanState *parent, ExprState *state, + Datum *resv, bool *resnull); +static void ExprEvalPushStep(ExprState *es, const ExprEvalStep *s); +static void ExecInitFunc(ExprEvalStep *scratch, Expr *node, List *args, + Oid funcid, Oid inputcollid, PlanState *parent, + ExprState *state); +static void ExecInitExprSlots(ExprState *state, Node *node); +static bool get_last_attnums_walker(Node *node, LastAttnumInfo *info); +static void ExecInitWholeRowVar(ExprEvalStep *scratch, Var *variable, + PlanState *parent); +static void ExecInitArrayRef(ExprEvalStep *scratch, ArrayRef *aref, + PlanState *parent, ExprState *state, + Datum *resv, bool *resnull); +static bool isAssignmentIndirectionExpr(Expr *expr); +static void ExecInitCoerceToDomain(ExprEvalStep *scratch, CoerceToDomain *ctest, + PlanState *parent, ExprState *state, + Datum *resv, bool *resnull); + + +/* + * ExecInitExpr: prepare an expression tree for execution + * + * This function builds and returns an ExprState implementing the given + * Expr node tree. The return ExprState can then be handed to ExecEvalExpr + * for execution. Because the Expr tree itself is read-only as far as + * ExecInitExpr and ExecEvalExpr are concerned, several different executions + * of the same plan tree can occur concurrently. (But note that an ExprState + * does mutate at runtime, so it can't be re-used concurrently.) + * + * This must be called in a memory context that will last as long as repeated + * executions of the expression are needed. Typically the context will be + * the same as the per-query context of the associated ExprContext. + * + * Any Aggref, WindowFunc, or SubPlan nodes found in the tree are added to + * the lists of such nodes held by the parent PlanState (or more accurately, + * the AggrefExprState etc. nodes created for them are added). + * + * Note: there is no ExecEndExpr function; we assume that any resource + * cleanup needed will be handled by just releasing the memory context + * in which the state tree is built. Functions that require additional + * cleanup work can register a shutdown callback in the ExprContext. + * + * 'node' is the root of the expression tree to compile. + * '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.) + * Such cases should usually come through ExecPrepareExpr, not directly here. + * + * Also, if 'node' is NULL, we just return NULL. This is convenient for some + * callers that may or may not have an expression that needs to be compiled. + * Note that a NULL ExprState pointer *cannot* be handed to ExecEvalExpr, + * although ExecQual and ExecCheck will accept one (and treat it as "true"). + */ +ExprState * +ExecInitExpr(Expr *node, PlanState *parent) +{ + ExprState *state; + ExprEvalStep scratch; + + /* Special case: NULL expression produces a NULL ExprState pointer */ + if (node == NULL) + return NULL; + + /* Initialize ExprState with empty step list */ + state = makeNode(ExprState); + state->expr = node; + + /* Insert EEOP_*_FETCHSOME steps as needed */ + ExecInitExprSlots(state, (Node *) node); + + /* Compile the expression proper */ + ExecInitExprRec(node, parent, state, &state->resvalue, &state->resnull); + + /* Finally, append a DONE step */ + scratch.opcode = EEOP_DONE; + ExprEvalPushStep(state, &scratch); + + ExecReadyExpr(state); + + return state; +} + +/* + * ExecInitQual: prepare a qual for execution by ExecQual + * + * Prepares for the evaluation of a conjunctive boolean expression (qual list + * with implicit AND semantics) that returns true if none of the + * subexpressions are false. + * + * We must return true if the list is empty. Since that's a very common case, + * we optimize it a bit further by translating to a NULL ExprState pointer + * rather than setting up an ExprState that computes constant TRUE. (Some + * especially hot-spot callers of ExecQual detect this and avoid calling + * ExecQual at all.) + * + * If any of the subexpressions yield NULL, then the result of the conjunction + * is false. This makes ExecQual primarily useful for evaluating WHERE + * clauses, since SQL specifies that tuples with null WHERE results do not + * get selected. + */ +ExprState * +ExecInitQual(List *qual, PlanState *parent) +{ + ExprState *state; + ExprEvalStep scratch; + List *adjust_jumps = NIL; + ListCell *lc; + + /* short-circuit (here and in ExecQual) for empty restriction list */ + if (qual == NIL) + return NULL; + + Assert(IsA(qual, List)); + + state = makeNode(ExprState); + state->expr = (Expr *) qual; + /* mark expression as to be used with ExecQual() */ + state->flags = EEO_FLAG_IS_QUAL; + + /* Insert EEOP_*_FETCHSOME steps as needed */ + ExecInitExprSlots(state, (Node *) qual); + + /* + * ExecQual() needs to return false for an expression returning NULL. That + * allows us to short-circuit the evaluation the first time a NULL is + * encountered. As qual evaluation is a hot-path this warrants using a + * special opcode for qual evaluation that's simpler than BOOL_AND (which + * has more complex NULL handling). + */ + scratch.opcode = EEOP_QUAL; + + /* + * We can use ExprState's resvalue/resnull as target for each qual expr. + */ + scratch.resvalue = &state->resvalue; + scratch.resnull = &state->resnull; + + foreach(lc, qual) + { + Expr *node = (Expr *) lfirst(lc); + + /* first evaluate expression */ + ExecInitExprRec(node, parent, state, &state->resvalue, &state->resnull); + + /* then emit EEOP_QUAL to detect if it's false (or null) */ + scratch.d.qualexpr.jumpdone = -1; + ExprEvalPushStep(state, &scratch); + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + } + + /* adjust jump targets */ + foreach(lc, adjust_jumps) + { + ExprEvalStep *as = &state->steps[lfirst_int(lc)]; + + Assert(as->opcode == EEOP_QUAL); + Assert(as->d.qualexpr.jumpdone == -1); + as->d.qualexpr.jumpdone = state->steps_len; + } + + /* + * At the end, we don't need to do anything more. The last qual expr must + * have yielded TRUE, and since its result is stored in the desired output + * location, we're done. + */ + scratch.opcode = EEOP_DONE; + ExprEvalPushStep(state, &scratch); + + ExecReadyExpr(state); + + return state; +} + +/* + * ExecInitCheck: prepare a check constraint for execution by ExecCheck + * + * This is much like ExecInitQual/ExecQual, except that a null result from + * the conjunction is treated as TRUE. This behavior is appropriate for + * evaluating CHECK constraints, since SQL specifies that NULL constraint + * conditions are not failures. + * + * Note that like ExecInitQual, this expects input in implicit-AND format. + * Users of ExecCheck that have expressions in normal explicit-AND format + * can just apply ExecInitExpr to produce suitable input for ExecCheck. + */ +ExprState * +ExecInitCheck(List *qual, PlanState *parent) +{ + /* short-circuit (here and in ExecCheck) for empty restriction list */ + if (qual == NIL) + return NULL; + + Assert(IsA(qual, List)); + + /* + * Just convert the implicit-AND list to an explicit AND (if there's more + * than one entry), and compile normally. Unlike ExecQual, we can't + * short-circuit on NULL results, so the regular AND behavior is needed. + */ + return ExecInitExpr(make_ands_explicit(qual), parent); +} + +/* + * Call ExecInitExpr() on a list of expressions, return a list of ExprStates. + */ +List * +ExecInitExprList(List *nodes, PlanState *parent) +{ + List *result = NIL; + ListCell *lc; + + foreach(lc, nodes) + { + Expr *e = lfirst(lc); + + result = lappend(result, ExecInitExpr(e, parent)); + } + + return result; +} + +/* + * ExecBuildProjectionInfo + * + * Build a ProjectionInfo node for evaluating the given tlist in the given + * econtext, and storing the result into the tuple slot. (Caller must have + * ensured that tuple slot has a descriptor matching the tlist!) + * + * inputDesc can be NULL, but if it is not, we check to see whether simple + * Vars in the tlist match the descriptor. It is important to provide + * inputDesc for relation-scan plan nodes, as a cross check that the relation + * hasn't been changed since the plan was made. At higher levels of a plan, + * there is no need to recheck. + * + * This is implemented by internally building an ExprState that performs the + * whole projection in one go. + * + * Caution: before PG v10, the targetList was a list of ExprStates; now it + * should be the planner-created targetlist, since we do the compilation here. + */ +ProjectionInfo * +ExecBuildProjectionInfo(List *targetList, + ExprContext *econtext, + TupleTableSlot *slot, + PlanState *parent, + TupleDesc inputDesc) +{ + ProjectionInfo *projInfo = makeNode(ProjectionInfo); + ExprState *state; + ExprEvalStep scratch; + ListCell *lc; + + projInfo->pi_exprContext = econtext; + /* We embed ExprState into ProjectionInfo instead of doing extra palloc */ + projInfo->pi_state.tag.type = T_ExprState; + state = &projInfo->pi_state; + state->expr = (Expr *) targetList; + state->resultslot = slot; + + /* Insert EEOP_*_FETCHSOME steps as needed */ + ExecInitExprSlots(state, (Node *) targetList); + + /* Now compile each tlist column */ + foreach(lc, targetList) + { + TargetEntry *tle = lfirst_node(TargetEntry, lc); + Var *variable = NULL; + AttrNumber attnum = 0; + bool isSafeVar = false; + + /* + * If tlist expression is a safe non-system Var, use the fast-path + * ASSIGN_*_VAR opcodes. "Safe" means that we don't need to apply + * CheckVarSlotCompatibility() during plan startup. If a source slot + * was provided, we make the equivalent tests here; if a slot was not + * provided, we assume that no check is needed because we're dealing + * with a non-relation-scan-level expression. + */ + if (tle->expr != NULL && + IsA(tle->expr, Var) && + ((Var *) tle->expr)->varattno > 0) + { + /* Non-system Var, but how safe is it? */ + variable = (Var *) tle->expr; + attnum = variable->varattno; + + if (inputDesc == NULL) + isSafeVar = true; /* can't check, just assume OK */ + else if (attnum <= inputDesc->natts) + { + Form_pg_attribute attr = inputDesc->attrs[attnum - 1]; + + /* + * If user attribute is dropped or has a type mismatch, don't + * use ASSIGN_*_VAR. Instead let the normal expression + * machinery handle it (which'll possibly error out). + */ + if (!attr->attisdropped && variable->vartype == attr->atttypid) + { + isSafeVar = true; + } + } + } + + if (isSafeVar) + { + /* Fast-path: just generate an EEOP_ASSIGN_*_VAR step */ + switch (variable->varno) + { + case INNER_VAR: + /* get the tuple from the inner node */ + scratch.opcode = EEOP_ASSIGN_INNER_VAR; + break; + + case OUTER_VAR: + /* get the tuple from the outer node */ + scratch.opcode = EEOP_ASSIGN_OUTER_VAR; + break; + + /* INDEX_VAR is handled by default case */ + + default: + /* get the tuple from the relation being scanned */ + scratch.opcode = EEOP_ASSIGN_SCAN_VAR; + break; + } + + scratch.d.assign_var.attnum = attnum - 1; + scratch.d.assign_var.resultnum = tle->resno - 1; + ExprEvalPushStep(state, &scratch); + } + else + { + /* + * Otherwise, compile the column expression normally. + * + * We can't tell the expression to evaluate directly into the + * result slot, as the result slot (and the exprstate for that + * matter) can change between executions. We instead evaluate + * into the ExprState's resvalue/resnull and then move. + */ + ExecInitExprRec(tle->expr, parent, state, + &state->resvalue, &state->resnull); + + /* + * Column might be referenced multiple times in upper nodes, so + * force value to R/O - but only if it could be an expanded datum. + */ + if (get_typlen(exprType((Node *) tle->expr)) == -1) + scratch.opcode = EEOP_ASSIGN_TMP_MAKE_RO; + else + scratch.opcode = EEOP_ASSIGN_TMP; + scratch.d.assign_tmp.resultnum = tle->resno - 1; + ExprEvalPushStep(state, &scratch); + } + } + + scratch.opcode = EEOP_DONE; + ExprEvalPushStep(state, &scratch); + + ExecReadyExpr(state); + + return projInfo; +} + +/* + * 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 run the + * passed expression tree through expression_planner() to prepare it for + * execution. (In ordinary Plan trees the regular planning process will have + * made the appropriate transformations on expressions, but for standalone + * expressions this won't have happened.) + */ +ExprState * +ExecPrepareExpr(Expr *node, EState *estate) +{ + ExprState *result; + MemoryContext oldcontext; + + oldcontext = MemoryContextSwitchTo(estate->es_query_cxt); + + node = expression_planner(node); + + result = ExecInitExpr(node, NULL); + + MemoryContextSwitchTo(oldcontext); + + return result; +} + +/* + * ExecPrepareQual --- initialize for qual execution outside a normal + * Plan tree context. + * + * This differs from ExecInitQual in that we don't assume the caller is + * already running in the EState's per-query context. Also, we run the + * passed expression tree through expression_planner() to prepare it for + * execution. (In ordinary Plan trees the regular planning process will have + * made the appropriate transformations on expressions, but for standalone + * expressions this won't have happened.) + */ +ExprState * +ExecPrepareQual(List *qual, EState *estate) +{ + ExprState *result; + MemoryContext oldcontext; + + oldcontext = MemoryContextSwitchTo(estate->es_query_cxt); + + qual = (List *) expression_planner((Expr *) qual); + + result = ExecInitQual(qual, NULL); + + MemoryContextSwitchTo(oldcontext); + + return result; +} + +/* + * ExecPrepareCheck -- initialize check constraint for execution outside a + * normal Plan tree context. + * + * See ExecPrepareExpr() and ExecInitCheck() for details. + */ +ExprState * +ExecPrepareCheck(List *qual, EState *estate) +{ + ExprState *result; + MemoryContext oldcontext; + + oldcontext = MemoryContextSwitchTo(estate->es_query_cxt); + + qual = (List *) expression_planner((Expr *) qual); + + result = ExecInitCheck(qual, NULL); + + MemoryContextSwitchTo(oldcontext); + + return result; +} + +/* + * Call ExecPrepareExpr() on each member of a list of Exprs, and return + * a list of ExprStates. + * + * See ExecPrepareExpr() for details. + */ +List * +ExecPrepareExprList(List *nodes, EState *estate) +{ + List *result = NIL; + MemoryContext oldcontext; + ListCell *lc; + + /* Ensure that the list cell nodes are in the right context too */ + oldcontext = MemoryContextSwitchTo(estate->es_query_cxt); + + foreach(lc, nodes) + { + Expr *e = (Expr *) lfirst(lc); + + result = lappend(result, ExecPrepareExpr(e, estate)); + } + + MemoryContextSwitchTo(oldcontext); + + return result; +} + +/* + * ExecCheck - evaluate a check constraint + * + * For check constraints, a null result is taken as TRUE, ie the constraint + * passes. + * + * The check constraint may have been prepared with ExecInitCheck + * (possibly via ExecPrepareCheck) if the caller had it in implicit-AND + * format, but a regular boolean expression prepared with ExecInitExpr or + * ExecPrepareExpr works too. + */ +bool +ExecCheck(ExprState *state, ExprContext *econtext) +{ + Datum ret; + bool isnull; + + /* short-circuit (here and in ExecInitCheck) for empty restriction list */ + if (state == NULL) + return true; + + /* verify that expression was not compiled using ExecInitQual */ + Assert(!(state->flags & EEO_FLAG_IS_QUAL)); + + ret = ExecEvalExprSwitchContext(state, econtext, &isnull); + + if (isnull) + return true; + + return DatumGetBool(ret); +} + +/* + * Prepare a compiled expression for execution. This has to be called for + * every ExprState before it can be executed. + * + * NB: While this currently only calls ExecReadyInterpretedExpr(), + * this will likely get extended to further expression evaluation methods. + * Therefore this should be used instead of directly calling + * ExecReadyInterpretedExpr(). + */ +static void +ExecReadyExpr(ExprState *state) +{ + ExecReadyInterpretedExpr(state); +} + +/* + * Append the steps necessary for the evaluation of node to ExprState->steps, + * possibly recursing into sub-expressions of node. + * + * node - expression to evaluate + * parent - parent executor node (or NULL if a standalone expression) + * state - ExprState to whose ->steps to append the necessary operations + * resv / resnull - where to store the result of the node into + */ +static void +ExecInitExprRec(Expr *node, PlanState *parent, ExprState *state, + Datum *resv, bool *resnull) +{ + ExprEvalStep scratch; + + /* Guard against stack overflow due to overly complex expressions */ + check_stack_depth(); + + /* Step's output location is always what the caller gave us */ + Assert(resv != NULL && resnull != NULL); + scratch.resvalue = resv; + scratch.resnull = resnull; + + /* cases should be ordered as they are in enum NodeTag */ + switch (nodeTag(node)) + { + case T_Var: + { + Var *variable = (Var *) node; + + if (variable->varattno == InvalidAttrNumber) + { + /* whole-row Var */ + ExecInitWholeRowVar(&scratch, variable, parent); + } + else if (variable->varattno <= 0) + { + /* system column */ + scratch.d.var.attnum = variable->varattno; + scratch.d.var.vartype = variable->vartype; + switch (variable->varno) + { + case INNER_VAR: + scratch.opcode = EEOP_INNER_SYSVAR; + break; + case OUTER_VAR: + scratch.opcode = EEOP_OUTER_SYSVAR; + break; + + /* INDEX_VAR is handled by default case */ + + default: + scratch.opcode = EEOP_SCAN_SYSVAR; + break; + } + } + else + { + /* regular user column */ + scratch.d.var.attnum = variable->varattno - 1; + scratch.d.var.vartype = variable->vartype; + /* select EEOP_*_FIRST opcode to force one-time checks */ + switch (variable->varno) + { + case INNER_VAR: + scratch.opcode = EEOP_INNER_VAR_FIRST; + break; + case OUTER_VAR: + scratch.opcode = EEOP_OUTER_VAR_FIRST; + break; + + /* INDEX_VAR is handled by default case */ + + default: + scratch.opcode = EEOP_SCAN_VAR_FIRST; + break; + } + } + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_Const: + { + Const *con = (Const *) node; + + scratch.opcode = EEOP_CONST; + scratch.d.constval.value = con->constvalue; + scratch.d.constval.isnull = con->constisnull; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_Param: + { + Param *param = (Param *) node; + + switch (param->paramkind) + { + case PARAM_EXEC: + scratch.opcode = EEOP_PARAM_EXEC; + scratch.d.param.paramid = param->paramid; + scratch.d.param.paramtype = param->paramtype; + break; + case PARAM_EXTERN: + scratch.opcode = EEOP_PARAM_EXTERN; + scratch.d.param.paramid = param->paramid; + scratch.d.param.paramtype = param->paramtype; + break; + default: + elog(ERROR, "unrecognized paramkind: %d", + (int) param->paramkind); + break; + } + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_Aggref: + { + Aggref *aggref = (Aggref *) node; + AggrefExprState *astate = makeNode(AggrefExprState); + + scratch.opcode = EEOP_AGGREF; + scratch.d.aggref.astate = astate; + astate->aggref = aggref; + + if (parent && IsA(parent, AggState)) + { + AggState *aggstate = (AggState *) parent; + + aggstate->aggs = lcons(astate, aggstate->aggs); + aggstate->numaggs++; + } + else + { + /* planner messed up */ + elog(ERROR, "Aggref found in non-Agg plan node"); + } + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_GroupingFunc: + { + GroupingFunc *grp_node = (GroupingFunc *) node; + Agg *agg; + + if (!parent || !IsA(parent, AggState) || + !IsA(parent->plan, Agg)) + elog(ERROR, "GroupingFunc found in non-Agg plan node"); + + scratch.opcode = EEOP_GROUPING_FUNC; + scratch.d.grouping_func.parent = (AggState *) parent; + + agg = (Agg *) (parent->plan); + + if (agg->groupingSets) + scratch.d.grouping_func.clauses = grp_node->cols; + else + scratch.d.grouping_func.clauses = NIL; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_WindowFunc: + { + WindowFunc *wfunc = (WindowFunc *) node; + WindowFuncExprState *wfstate = makeNode(WindowFuncExprState); + + wfstate->wfunc = wfunc; + + if (parent && IsA(parent, WindowAggState)) + { + WindowAggState *winstate = (WindowAggState *) parent; + int nfuncs; + + winstate->funcs = lcons(wfstate, winstate->funcs); + nfuncs = ++winstate->numfuncs; + if (wfunc->winagg) + winstate->numaggs++; + + /* for now initialize agg using old style expressions */ + wfstate->args = ExecInitExprList(wfunc->args, parent); + wfstate->aggfilter = ExecInitExpr(wfunc->aggfilter, + parent); + + /* + * Complain if the windowfunc's arguments contain any + * windowfuncs; nested window functions are semantically + * nonsensical. (This should have been caught earlier, + * but we defend against it here anyway.) + */ + if (nfuncs != winstate->numfuncs) + ereport(ERROR, + (errcode(ERRCODE_WINDOWING_ERROR), + errmsg("window function calls cannot be nested"))); + } + else + { + /* planner messed up */ + elog(ERROR, "WindowFunc found in non-WindowAgg plan node"); + } + + scratch.opcode = EEOP_WINDOW_FUNC; + scratch.d.window_func.wfstate = wfstate; + ExprEvalPushStep(state, &scratch); + break; + } + + case T_ArrayRef: + { + ArrayRef *aref = (ArrayRef *) node; + + ExecInitArrayRef(&scratch, aref, parent, state, resv, resnull); + break; + } + + case T_FuncExpr: + { + FuncExpr *func = (FuncExpr *) node; + + ExecInitFunc(&scratch, node, + func->args, func->funcid, func->inputcollid, + parent, state); + ExprEvalPushStep(state, &scratch); + break; + } + + case T_OpExpr: + { + OpExpr *op = (OpExpr *) node; + + ExecInitFunc(&scratch, node, + op->args, op->opfuncid, op->inputcollid, + parent, state); + ExprEvalPushStep(state, &scratch); + break; + } + + case T_DistinctExpr: + { + DistinctExpr *op = (DistinctExpr *) node; + + ExecInitFunc(&scratch, node, + op->args, op->opfuncid, op->inputcollid, + parent, state); + + /* + * Change opcode of call instruction to EEOP_DISTINCT. + * + * XXX: historically we've not called the function usage + * pgstat infrastructure - that seems inconsistent given that + * we do so for normal function *and* operator evaluation. If + * we decided to do that here, we'd probably want separate + * opcodes for FUSAGE or not. + */ + scratch.opcode = EEOP_DISTINCT; + ExprEvalPushStep(state, &scratch); + break; + } + + case T_NullIfExpr: + { + NullIfExpr *op = (NullIfExpr *) node; + + ExecInitFunc(&scratch, node, + op->args, op->opfuncid, op->inputcollid, + parent, state); + + /* + * Change opcode of call instruction to EEOP_NULLIF. + * + * XXX: historically we've not called the function usage + * pgstat infrastructure - that seems inconsistent given that + * we do so for normal function *and* operator evaluation. If + * we decided to do that here, we'd probably want separate + * opcodes for FUSAGE or not. + */ + scratch.opcode = EEOP_NULLIF; + ExprEvalPushStep(state, &scratch); + break; + } + + case T_ScalarArrayOpExpr: + { + ScalarArrayOpExpr *opexpr = (ScalarArrayOpExpr *) node; + Expr *scalararg; + Expr *arrayarg; + FmgrInfo *finfo; + FunctionCallInfo fcinfo; + AclResult aclresult; + + Assert(list_length(opexpr->args) == 2); + scalararg = (Expr *) linitial(opexpr->args); + arrayarg = (Expr *) lsecond(opexpr->args); + + /* Check permission to call function */ + aclresult = pg_proc_aclcheck(opexpr->opfuncid, + GetUserId(), + ACL_EXECUTE); + if (aclresult != ACLCHECK_OK) + aclcheck_error(aclresult, ACL_KIND_PROC, + get_func_name(opexpr->opfuncid)); + InvokeFunctionExecuteHook(opexpr->opfuncid); + + /* Set up the primary fmgr lookup information */ + finfo = palloc0(sizeof(FmgrInfo)); + fcinfo = palloc0(sizeof(FunctionCallInfoData)); + fmgr_info(opexpr->opfuncid, finfo); + fmgr_info_set_expr((Node *) node, finfo); + InitFunctionCallInfoData(*fcinfo, finfo, 2, + opexpr->inputcollid, NULL, NULL); + + /* Evaluate scalar directly into left function argument */ + ExecInitExprRec(scalararg, parent, state, + &fcinfo->arg[0], &fcinfo->argnull[0]); + + /* + * Evaluate array argument into our return value. There's no + * danger in that, because the return value is guaranteed to + * be overwritten by EEOP_SCALARARRAYOP, and will not be + * passed to any other expression. + */ + ExecInitExprRec(arrayarg, parent, state, resv, resnull); + + /* And perform the operation */ + scratch.opcode = EEOP_SCALARARRAYOP; + scratch.d.scalararrayop.element_type = InvalidOid; + scratch.d.scalararrayop.useOr = opexpr->useOr; + scratch.d.scalararrayop.finfo = finfo; + scratch.d.scalararrayop.fcinfo_data = fcinfo; + scratch.d.scalararrayop.fn_addr = finfo->fn_addr; + ExprEvalPushStep(state, &scratch); + break; + } + + case T_BoolExpr: + { + BoolExpr *boolexpr = (BoolExpr *) node; + int nargs = list_length(boolexpr->args); + List *adjust_jumps = NIL; + int off; + ListCell *lc; + + /* allocate scratch memory used by all steps of AND/OR */ + if (boolexpr->boolop != NOT_EXPR) + scratch.d.boolexpr.anynull = (bool *) palloc(sizeof(bool)); + + /* + * For each argument evaluate the argument itself, then + * perform the bool operation's appropriate handling. + * + * We can evaluate each argument into our result area, since + * the short-circuiting logic means we only need to remember + * previous NULL values. + * + * AND/OR is split into separate STEP_FIRST (one) / STEP (zero + * or more) / STEP_LAST (one) steps, as each of those has to + * perform different work. The FIRST/LAST split is valid + * because AND/OR have at least two arguments. + */ + off = 0; + foreach(lc, boolexpr->args) + { + Expr *arg = (Expr *) lfirst(lc); + + /* Evaluate argument into our output variable */ + ExecInitExprRec(arg, parent, state, resv, resnull); + + /* Perform the appropriate step type */ + switch (boolexpr->boolop) + { + case AND_EXPR: + Assert(nargs >= 2); + + if (off == 0) + scratch.opcode = EEOP_BOOL_AND_STEP_FIRST; + else if (off + 1 == nargs) + scratch.opcode = EEOP_BOOL_AND_STEP_LAST; + else + scratch.opcode = EEOP_BOOL_AND_STEP; + break; + case OR_EXPR: + Assert(nargs >= 2); + + if (off == 0) + scratch.opcode = EEOP_BOOL_OR_STEP_FIRST; + else if (off + 1 == nargs) + scratch.opcode = EEOP_BOOL_OR_STEP_LAST; + else + scratch.opcode = EEOP_BOOL_OR_STEP; + break; + case NOT_EXPR: + Assert(nargs == 1); + + scratch.opcode = EEOP_BOOL_NOT_STEP; + break; + default: + elog(ERROR, "unrecognized boolop: %d", + (int) boolexpr->boolop); + break; + } + + scratch.d.boolexpr.jumpdone = -1; + ExprEvalPushStep(state, &scratch); + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + off++; + } + + /* adjust jump targets */ + foreach(lc, adjust_jumps) + { + ExprEvalStep *as = &state->steps[lfirst_int(lc)]; + + Assert(as->d.boolexpr.jumpdone == -1); + as->d.boolexpr.jumpdone = state->steps_len; + } + + break; + } + + case T_SubPlan: + { + SubPlan *subplan = (SubPlan *) node; + SubPlanState *sstate; + + if (!parent) + elog(ERROR, "SubPlan found with no parent plan"); + + sstate = ExecInitSubPlan(subplan, parent); + + /* add SubPlanState nodes to parent->subPlan */ + parent->subPlan = lappend(parent->subPlan, sstate); + + scratch.opcode = EEOP_SUBPLAN; + scratch.d.subplan.sstate = sstate; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_AlternativeSubPlan: + { + AlternativeSubPlan *asplan = (AlternativeSubPlan *) node; + AlternativeSubPlanState *asstate; + + if (!parent) + elog(ERROR, "AlternativeSubPlan found with no parent plan"); + + asstate = ExecInitAlternativeSubPlan(asplan, parent); + + scratch.opcode = EEOP_ALTERNATIVE_SUBPLAN; + scratch.d.alternative_subplan.asstate = asstate; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_FieldSelect: + { + FieldSelect *fselect = (FieldSelect *) node; + + /* evaluate row/record argument into result area */ + ExecInitExprRec(fselect->arg, parent, state, resv, resnull); + + /* and extract field */ + scratch.opcode = EEOP_FIELDSELECT; + scratch.d.fieldselect.fieldnum = fselect->fieldnum; + scratch.d.fieldselect.resulttype = fselect->resulttype; + scratch.d.fieldselect.argdesc = NULL; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_FieldStore: + { + FieldStore *fstore = (FieldStore *) node; + TupleDesc tupDesc; + TupleDesc *descp; + Datum *values; + bool *nulls; + int ncolumns; + ListCell *l1, + *l2; + + /* find out the number of columns in the composite type */ + tupDesc = lookup_rowtype_tupdesc(fstore->resulttype, -1); + ncolumns = tupDesc->natts; + DecrTupleDescRefCount(tupDesc); + + /* create workspace for column values */ + values = (Datum *) palloc(sizeof(Datum) * ncolumns); + nulls = (bool *) palloc(sizeof(bool) * ncolumns); + + /* create workspace for runtime tupdesc cache */ + descp = (TupleDesc *) palloc(sizeof(TupleDesc)); + *descp = NULL; + + /* emit code to evaluate the composite input value */ + ExecInitExprRec(fstore->arg, parent, state, resv, resnull); + + /* next, deform the input tuple into our workspace */ + scratch.opcode = EEOP_FIELDSTORE_DEFORM; + scratch.d.fieldstore.fstore = fstore; + scratch.d.fieldstore.argdesc = descp; + scratch.d.fieldstore.values = values; + scratch.d.fieldstore.nulls = nulls; + scratch.d.fieldstore.ncolumns = ncolumns; + ExprEvalPushStep(state, &scratch); + + /* evaluate new field values, store in workspace columns */ + forboth(l1, fstore->newvals, l2, fstore->fieldnums) + { + Expr *e = (Expr *) lfirst(l1); + AttrNumber fieldnum = lfirst_int(l2); + Datum *save_innermost_caseval; + bool *save_innermost_casenull; + + if (fieldnum <= 0 || fieldnum > ncolumns) + elog(ERROR, "field number %d is out of range in FieldStore", + fieldnum); + + /* + * Use the CaseTestExpr mechanism to pass down the old + * value of the field being replaced; this is needed in + * case the newval is itself a FieldStore or ArrayRef that + * has to obtain and modify the old value. It's safe to + * reuse the CASE mechanism because there cannot be a CASE + * between here and where the value would be needed, and a + * field assignment can't be within a CASE either. (So + * saving and restoring innermost_caseval is just + * paranoia, but let's do it anyway.) + */ + save_innermost_caseval = state->innermost_caseval; + save_innermost_casenull = state->innermost_casenull; + state->innermost_caseval = &values[fieldnum - 1]; + state->innermost_casenull = &nulls[fieldnum - 1]; + + ExecInitExprRec(e, parent, state, + &values[fieldnum - 1], + &nulls[fieldnum - 1]); + + state->innermost_caseval = save_innermost_caseval; + state->innermost_casenull = save_innermost_casenull; + } + + /* finally, form result tuple */ + scratch.opcode = EEOP_FIELDSTORE_FORM; + scratch.d.fieldstore.fstore = fstore; + scratch.d.fieldstore.argdesc = descp; + scratch.d.fieldstore.values = values; + scratch.d.fieldstore.nulls = nulls; + scratch.d.fieldstore.ncolumns = ncolumns; + ExprEvalPushStep(state, &scratch); + break; + } + + case T_RelabelType: + { + /* relabel doesn't need to do anything at runtime */ + RelabelType *relabel = (RelabelType *) node; + + ExecInitExprRec(relabel->arg, parent, state, resv, resnull); + break; + } + + case T_CoerceViaIO: + { + CoerceViaIO *iocoerce = (CoerceViaIO *) node; + Oid iofunc; + bool typisvarlena; + Oid typioparam; + FunctionCallInfo fcinfo_in; + + /* evaluate argument into step's result area */ + ExecInitExprRec(iocoerce->arg, parent, state, resv, resnull); + + /* + * Prepare both output and input function calls, to be + * evaluated inside a single evaluation step for speed - this + * can be a very common operation. + * + * We don't check permissions here as a type's input/output + * function are assumed to be executable by everyone. + */ + scratch.opcode = EEOP_IOCOERCE; + + /* lookup the source type's output function */ + scratch.d.iocoerce.finfo_out = palloc0(sizeof(FmgrInfo)); + scratch.d.iocoerce.fcinfo_data_out = palloc0(sizeof(FunctionCallInfoData)); + + getTypeOutputInfo(exprType((Node *) iocoerce->arg), + &iofunc, &typisvarlena); + fmgr_info(iofunc, scratch.d.iocoerce.finfo_out); + fmgr_info_set_expr((Node *) node, scratch.d.iocoerce.finfo_out); + InitFunctionCallInfoData(*scratch.d.iocoerce.fcinfo_data_out, + scratch.d.iocoerce.finfo_out, + 1, InvalidOid, NULL, NULL); + + /* lookup the result type's input function */ + scratch.d.iocoerce.finfo_in = palloc0(sizeof(FmgrInfo)); + scratch.d.iocoerce.fcinfo_data_in = palloc0(sizeof(FunctionCallInfoData)); + + getTypeInputInfo(iocoerce->resulttype, + &iofunc, &typioparam); + fmgr_info(iofunc, scratch.d.iocoerce.finfo_in); + fmgr_info_set_expr((Node *) node, scratch.d.iocoerce.finfo_in); + InitFunctionCallInfoData(*scratch.d.iocoerce.fcinfo_data_in, + scratch.d.iocoerce.finfo_in, + 3, InvalidOid, NULL, NULL); + + /* + * We can preload the second and third arguments for the input + * function, since they're constants. + */ + fcinfo_in = scratch.d.iocoerce.fcinfo_data_in; + fcinfo_in->arg[1] = ObjectIdGetDatum(typioparam); + fcinfo_in->argnull[1] = false; + fcinfo_in->arg[2] = Int32GetDatum(-1); + fcinfo_in->argnull[2] = false; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_ArrayCoerceExpr: + { + ArrayCoerceExpr *acoerce = (ArrayCoerceExpr *) node; + Oid resultelemtype; + + /* evaluate argument into step's result area */ + ExecInitExprRec(acoerce->arg, parent, state, resv, resnull); + + resultelemtype = get_element_type(acoerce->resulttype); + if (!OidIsValid(resultelemtype)) + ereport(ERROR, + (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("target type is not an array"))); + /* Arrays over domains aren't supported yet */ + Assert(getBaseType(resultelemtype) == resultelemtype); + + scratch.opcode = EEOP_ARRAYCOERCE; + scratch.d.arraycoerce.coerceexpr = acoerce; + scratch.d.arraycoerce.resultelemtype = resultelemtype; + + if (OidIsValid(acoerce->elemfuncid)) + { + AclResult aclresult; + + /* Check permission to call function */ + aclresult = pg_proc_aclcheck(acoerce->elemfuncid, + GetUserId(), + ACL_EXECUTE); + if (aclresult != ACLCHECK_OK) + aclcheck_error(aclresult, ACL_KIND_PROC, + get_func_name(acoerce->elemfuncid)); + InvokeFunctionExecuteHook(acoerce->elemfuncid); + + /* Set up the primary fmgr lookup information */ + scratch.d.arraycoerce.elemfunc = + (FmgrInfo *) palloc0(sizeof(FmgrInfo)); + fmgr_info(acoerce->elemfuncid, + scratch.d.arraycoerce.elemfunc); + fmgr_info_set_expr((Node *) acoerce, + scratch.d.arraycoerce.elemfunc); + + /* Set up workspace for array_map */ + scratch.d.arraycoerce.amstate = + (ArrayMapState *) palloc0(sizeof(ArrayMapState)); + } + else + { + /* Don't need workspace if there's no conversion func */ + scratch.d.arraycoerce.elemfunc = NULL; + scratch.d.arraycoerce.amstate = NULL; + } + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_ConvertRowtypeExpr: + { + ConvertRowtypeExpr *convert = (ConvertRowtypeExpr *) node; + + /* evaluate argument into step's result area */ + ExecInitExprRec(convert->arg, parent, state, resv, resnull); + + /* and push conversion step */ + scratch.opcode = EEOP_CONVERT_ROWTYPE; + scratch.d.convert_rowtype.convert = convert; + scratch.d.convert_rowtype.indesc = NULL; + scratch.d.convert_rowtype.outdesc = NULL; + scratch.d.convert_rowtype.map = NULL; + scratch.d.convert_rowtype.initialized = false; + + ExprEvalPushStep(state, &scratch); + break; + } + + /* note that CaseWhen expressions are handled within this block */ + case T_CaseExpr: + { + CaseExpr *caseExpr = (CaseExpr *) node; + List *adjust_jumps = NIL; + Datum *caseval = NULL; + bool *casenull = NULL; + ListCell *lc; + + /* + * If there's a test expression, we have to evaluate it and + * save the value where the CaseTestExpr placeholders can find + * it. + */ + if (caseExpr->arg != NULL) + { + /* Evaluate testexpr into caseval/casenull workspace */ + caseval = palloc(sizeof(Datum)); + casenull = palloc(sizeof(bool)); + + ExecInitExprRec(caseExpr->arg, parent, state, + caseval, casenull); + + /* + * Since value might be read multiple times, force to R/O + * - but only if it could be an expanded datum. + */ + if (get_typlen(exprType((Node *) caseExpr->arg)) == -1) + { + /* change caseval in-place */ + scratch.opcode = EEOP_MAKE_READONLY; + scratch.resvalue = caseval; + scratch.resnull = casenull; + scratch.d.make_readonly.value = caseval; + scratch.d.make_readonly.isnull = casenull; + ExprEvalPushStep(state, &scratch); + /* restore normal settings of scratch fields */ + scratch.resvalue = resv; + scratch.resnull = resnull; + } + } + + /* + * Prepare to evaluate each of the WHEN clauses in turn; as + * soon as one is true we return the value of the + * corresponding THEN clause. If none are true then we return + * the value of the ELSE clause, or NULL if there is none. + */ + foreach(lc, caseExpr->args) + { + CaseWhen *when = (CaseWhen *) lfirst(lc); + Datum *save_innermost_caseval; + bool *save_innermost_casenull; + int whenstep; + + /* + * Make testexpr result available to CaseTestExpr nodes + * within the condition. We must save and restore prior + * setting of innermost_caseval fields, in case this node + * is itself within a larger CASE. + * + * If there's no test expression, we don't actually need + * to save and restore these fields; but it's less code to + * just do so unconditionally. + */ + save_innermost_caseval = state->innermost_caseval; + save_innermost_casenull = state->innermost_casenull; + state->innermost_caseval = caseval; + state->innermost_casenull = casenull; + + /* evaluate condition into CASE's result variables */ + ExecInitExprRec(when->expr, parent, state, resv, resnull); + + state->innermost_caseval = save_innermost_caseval; + state->innermost_casenull = save_innermost_casenull; + + /* If WHEN result isn't true, jump to next CASE arm */ + scratch.opcode = EEOP_JUMP_IF_NOT_TRUE; + scratch.d.jump.jumpdone = -1; /* computed later */ + ExprEvalPushStep(state, &scratch); + whenstep = state->steps_len - 1; + + /* + * If WHEN result is true, evaluate THEN result, storing + * it into the CASE's result variables. + */ + ExecInitExprRec(when->result, parent, state, resv, resnull); + + /* Emit JUMP step to jump to end of CASE's code */ + scratch.opcode = EEOP_JUMP; + scratch.d.jump.jumpdone = -1; /* computed later */ + ExprEvalPushStep(state, &scratch); + + /* + * Don't know address for that jump yet, compute once the + * whole CASE expression is built. + */ + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + + /* + * But we can set WHEN test's jump target now, to make it + * jump to the next WHEN subexpression or the ELSE. + */ + state->steps[whenstep].d.jump.jumpdone = state->steps_len; + } + + /* transformCaseExpr always adds a default */ + Assert(caseExpr->defresult); + + /* evaluate ELSE expr into CASE's result variables */ + ExecInitExprRec(caseExpr->defresult, parent, state, + resv, resnull); + + /* adjust jump targets */ + foreach(lc, adjust_jumps) + { + ExprEvalStep *as = &state->steps[lfirst_int(lc)]; + + Assert(as->opcode == EEOP_JUMP); + Assert(as->d.jump.jumpdone == -1); + as->d.jump.jumpdone = state->steps_len; + } + + break; + } + + case T_CaseTestExpr: + { + /* + * Read from location identified by innermost_caseval. Note + * that innermost_caseval could be NULL, if this node isn't + * actually within a CASE structure; some parts of the system + * abuse CaseTestExpr to cause a read of a value externally + * supplied in econtext->caseValue_datum. We'll take care of + * that scenario at runtime. + */ + scratch.opcode = EEOP_CASE_TESTVAL; + scratch.d.casetest.value = state->innermost_caseval; + scratch.d.casetest.isnull = state->innermost_casenull; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_ArrayExpr: + { + ArrayExpr *arrayexpr = (ArrayExpr *) node; + int nelems = list_length(arrayexpr->elements); + ListCell *lc; + int elemoff; + + /* + * Evaluate by computing each element, and then forming the + * array. Elements are computed into scratch arrays + * associated with the ARRAYEXPR step. + */ + scratch.opcode = EEOP_ARRAYEXPR; + scratch.d.arrayexpr.elemvalues = + (Datum *) palloc(sizeof(Datum) * nelems); + scratch.d.arrayexpr.elemnulls = + (bool *) palloc(sizeof(bool) * nelems); + scratch.d.arrayexpr.nelems = nelems; + + /* fill remaining fields of step */ + scratch.d.arrayexpr.multidims = arrayexpr->multidims; + scratch.d.arrayexpr.elemtype = arrayexpr->element_typeid; + + /* do one-time catalog lookup for type info */ + get_typlenbyvalalign(arrayexpr->element_typeid, + &scratch.d.arrayexpr.elemlength, + &scratch.d.arrayexpr.elembyval, + &scratch.d.arrayexpr.elemalign); + + /* prepare to evaluate all arguments */ + elemoff = 0; + foreach(lc, arrayexpr->elements) + { + Expr *e = (Expr *) lfirst(lc); + + ExecInitExprRec(e, parent, state, + &scratch.d.arrayexpr.elemvalues[elemoff], + &scratch.d.arrayexpr.elemnulls[elemoff]); + elemoff++; + } + + /* and then collect all into an array */ + ExprEvalPushStep(state, &scratch); + break; + } + + case T_RowExpr: + { + RowExpr *rowexpr = (RowExpr *) node; + int nelems = list_length(rowexpr->args); + TupleDesc tupdesc; + Form_pg_attribute *attrs; + int i; + ListCell *l; + + /* Build tupdesc to describe result tuples */ + if (rowexpr->row_typeid == RECORDOID) + { + /* generic record, use types of given expressions */ + tupdesc = ExecTypeFromExprList(rowexpr->args); + } + else + { + /* it's been cast to a named type, use that */ + tupdesc = lookup_rowtype_tupdesc_copy(rowexpr->row_typeid, -1); + } + /* In either case, adopt RowExpr's column aliases */ + ExecTypeSetColNames(tupdesc, rowexpr->colnames); + /* Bless the tupdesc in case it's now of type RECORD */ + BlessTupleDesc(tupdesc); + + /* + * In the named-type case, the tupdesc could have more columns + * than are in the args list, since the type might have had + * columns added since the ROW() was parsed. We want those + * extra columns to go to nulls, so we make sure that the + * workspace arrays are large enough and then initialize any + * extra columns to read as NULLs. + */ + Assert(nelems <= tupdesc->natts); + nelems = Max(nelems, tupdesc->natts); + + /* + * Evaluate by first building datums for each field, and then + * a final step forming the composite datum. + */ + scratch.opcode = EEOP_ROW; + scratch.d.row.tupdesc = tupdesc; + + /* space for the individual field datums */ + scratch.d.row.elemvalues = + (Datum *) palloc(sizeof(Datum) * nelems); + scratch.d.row.elemnulls = + (bool *) palloc(sizeof(bool) * nelems); + /* as explained above, make sure any extra columns are null */ + memset(scratch.d.row.elemnulls, true, sizeof(bool) * nelems); + + /* Set up evaluation, skipping any deleted columns */ + attrs = tupdesc->attrs; + i = 0; + foreach(l, rowexpr->args) + { + Expr *e = (Expr *) lfirst(l); + + if (!attrs[i]->attisdropped) + { + /* + * Guard against ALTER COLUMN TYPE on rowtype since + * the RowExpr was created. XXX should we check + * typmod too? Not sure we can be sure it'll be the + * same. + */ + if (exprType((Node *) e) != attrs[i]->atttypid) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("ROW() column has type %s instead of type %s", + format_type_be(exprType((Node *) e)), + format_type_be(attrs[i]->atttypid)))); + } + else + { + /* + * Ignore original expression and insert a NULL. We + * don't really care what type of NULL it is, so + * always make an int4 NULL. + */ + e = (Expr *) makeNullConst(INT4OID, -1, InvalidOid); + } + + /* Evaluate column expr into appropriate workspace slot */ + ExecInitExprRec(e, parent, state, + &scratch.d.row.elemvalues[i], + &scratch.d.row.elemnulls[i]); + i++; + } + + /* And finally build the row value */ + ExprEvalPushStep(state, &scratch); + break; + } + + case T_RowCompareExpr: + { + RowCompareExpr *rcexpr = (RowCompareExpr *) node; + int nopers = list_length(rcexpr->opnos); + List *adjust_jumps = NIL; + ListCell *l_left_expr, + *l_right_expr, + *l_opno, + *l_opfamily, + *l_inputcollid; + ListCell *lc; + int off; + + /* + * Iterate over each field, prepare comparisons. To handle + * NULL results, prepare jumps to after the expression. If a + * comparison yields a != 0 result, jump to the final step. + */ + Assert(list_length(rcexpr->largs) == nopers); + Assert(list_length(rcexpr->rargs) == nopers); + Assert(list_length(rcexpr->opfamilies) == nopers); + Assert(list_length(rcexpr->inputcollids) == nopers); + + off = 0; + for (off = 0, + l_left_expr = list_head(rcexpr->largs), + l_right_expr = list_head(rcexpr->rargs), + l_opno = list_head(rcexpr->opnos), + l_opfamily = list_head(rcexpr->opfamilies), + l_inputcollid = list_head(rcexpr->inputcollids); + off < nopers; + off++, + l_left_expr = lnext(l_left_expr), + l_right_expr = lnext(l_right_expr), + l_opno = lnext(l_opno), + l_opfamily = lnext(l_opfamily), + l_inputcollid = lnext(l_inputcollid)) + { + Expr *left_expr = (Expr *) lfirst(l_left_expr); + Expr *right_expr = (Expr *) lfirst(l_right_expr); + Oid opno = lfirst_oid(l_opno); + Oid opfamily = lfirst_oid(l_opfamily); + Oid inputcollid = lfirst_oid(l_inputcollid); + int strategy; + Oid lefttype; + Oid righttype; + Oid proc; + FmgrInfo *finfo; + FunctionCallInfo fcinfo; + + get_op_opfamily_properties(opno, opfamily, false, + &strategy, + &lefttype, + &righttype); + proc = get_opfamily_proc(opfamily, + lefttype, + righttype, + BTORDER_PROC); + + /* Set up the primary fmgr lookup information */ + finfo = palloc0(sizeof(FmgrInfo)); + fcinfo = palloc0(sizeof(FunctionCallInfoData)); + fmgr_info(proc, finfo); + fmgr_info_set_expr((Node *) node, finfo); + InitFunctionCallInfoData(*fcinfo, finfo, 2, + inputcollid, NULL, NULL); + + /* + * If we enforced permissions checks on index support + * functions, we'd need to make a check here. But the + * index support machinery doesn't do that, and thus + * neither does this code. + */ + + /* evaluate left and right args directly into fcinfo */ + ExecInitExprRec(left_expr, parent, state, + &fcinfo->arg[0], &fcinfo->argnull[0]); + ExecInitExprRec(right_expr, parent, state, + &fcinfo->arg[1], &fcinfo->argnull[1]); + + scratch.opcode = EEOP_ROWCOMPARE_STEP; + scratch.d.rowcompare_step.finfo = finfo; + scratch.d.rowcompare_step.fcinfo_data = fcinfo; + scratch.d.rowcompare_step.fn_addr = finfo->fn_addr; + /* jump targets filled below */ + scratch.d.rowcompare_step.jumpnull = -1; + scratch.d.rowcompare_step.jumpdone = -1; + + ExprEvalPushStep(state, &scratch); + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + } + + /* + * We could have a zero-column rowtype, in which case the rows + * necessarily compare equal. + */ + if (nopers == 0) + { + scratch.opcode = EEOP_CONST; + scratch.d.constval.value = Int32GetDatum(0); + scratch.d.constval.isnull = false; + ExprEvalPushStep(state, &scratch); + } + + /* Finally, examine the last comparison result */ + scratch.opcode = EEOP_ROWCOMPARE_FINAL; + scratch.d.rowcompare_final.rctype = rcexpr->rctype; + ExprEvalPushStep(state, &scratch); + + /* adjust jump targetss */ + foreach(lc, adjust_jumps) + { + ExprEvalStep *as = &state->steps[lfirst_int(lc)]; + + Assert(as->opcode == EEOP_ROWCOMPARE_STEP); + Assert(as->d.rowcompare_step.jumpdone == -1); + Assert(as->d.rowcompare_step.jumpnull == -1); + + /* jump to comparison evaluation */ + as->d.rowcompare_step.jumpdone = state->steps_len - 1; + /* jump to the following expression */ + as->d.rowcompare_step.jumpnull = state->steps_len; + } + + break; + } + + case T_CoalesceExpr: + { + CoalesceExpr *coalesce = (CoalesceExpr *) node; + List *adjust_jumps = NIL; + ListCell *lc; + + /* We assume there's at least one arg */ + Assert(coalesce->args != NIL); + + /* + * Prepare evaluation of all coalesced arguments, after each + * one push a step that short-circuits if not null. + */ + foreach(lc, coalesce->args) + { + Expr *e = (Expr *) lfirst(lc); + + /* evaluate argument, directly into result datum */ + ExecInitExprRec(e, parent, state, resv, resnull); + + /* if it's not null, skip to end of COALESCE expr */ + scratch.opcode = EEOP_JUMP_IF_NOT_NULL; + scratch.d.jump.jumpdone = -1; /* adjust later */ + ExprEvalPushStep(state, &scratch); + + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + } + + /* + * No need to add a constant NULL return - we only can get to + * the end of the expression if a NULL already is being + * returned. + */ + + /* adjust jump targets */ + foreach(lc, adjust_jumps) + { + ExprEvalStep *as = &state->steps[lfirst_int(lc)]; + + Assert(as->opcode == EEOP_JUMP_IF_NOT_NULL); + Assert(as->d.jump.jumpdone == -1); + as->d.jump.jumpdone = state->steps_len; + } + + break; + } + + case T_MinMaxExpr: + { + MinMaxExpr *minmaxexpr = (MinMaxExpr *) node; + int nelems = list_length(minmaxexpr->args); + TypeCacheEntry *typentry; + FmgrInfo *finfo; + FunctionCallInfo fcinfo; + ListCell *lc; + int off; + + /* Look up the btree comparison function for the datatype */ + typentry = lookup_type_cache(minmaxexpr->minmaxtype, + TYPECACHE_CMP_PROC); + if (!OidIsValid(typentry->cmp_proc)) + ereport(ERROR, + (errcode(ERRCODE_UNDEFINED_FUNCTION), + errmsg("could not identify a comparison function for type %s", + format_type_be(minmaxexpr->minmaxtype)))); + + /* + * If we enforced permissions checks on index support + * functions, we'd need to make a check here. But the index + * support machinery doesn't do that, and thus neither does + * this code. + */ + + /* Perform function lookup */ + finfo = palloc0(sizeof(FmgrInfo)); + fcinfo = palloc0(sizeof(FunctionCallInfoData)); + fmgr_info(typentry->cmp_proc, finfo); + fmgr_info_set_expr((Node *) node, finfo); + InitFunctionCallInfoData(*fcinfo, finfo, 2, + minmaxexpr->inputcollid, NULL, NULL); + + scratch.opcode = EEOP_MINMAX; + /* allocate space to store arguments */ + scratch.d.minmax.values = + (Datum *) palloc(sizeof(Datum) * nelems); + scratch.d.minmax.nulls = + (bool *) palloc(sizeof(bool) * nelems); + scratch.d.minmax.nelems = nelems; + + scratch.d.minmax.op = minmaxexpr->op; + scratch.d.minmax.finfo = finfo; + scratch.d.minmax.fcinfo_data = fcinfo; + + /* evaluate expressions into minmax->values/nulls */ + off = 0; + foreach(lc, minmaxexpr->args) + { + Expr *e = (Expr *) lfirst(lc); + + ExecInitExprRec(e, parent, state, + &scratch.d.minmax.values[off], + &scratch.d.minmax.nulls[off]); + off++; + } + + /* and push the final comparison */ + ExprEvalPushStep(state, &scratch); + break; + } + + case T_SQLValueFunction: + { + SQLValueFunction *svf = (SQLValueFunction *) node; + + scratch.opcode = EEOP_SQLVALUEFUNCTION; + scratch.d.sqlvaluefunction.svf = svf; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_XmlExpr: + { + XmlExpr *xexpr = (XmlExpr *) node; + int nnamed = list_length(xexpr->named_args); + int nargs = list_length(xexpr->args); + int off; + ListCell *arg; + + scratch.opcode = EEOP_XMLEXPR; + scratch.d.xmlexpr.xexpr = xexpr; + + /* allocate space for storing all the arguments */ + if (nnamed) + { + scratch.d.xmlexpr.named_argvalue = + (Datum *) palloc(sizeof(Datum) * nnamed); + scratch.d.xmlexpr.named_argnull = + (bool *) palloc(sizeof(bool) * nnamed); + } + else + { + scratch.d.xmlexpr.named_argvalue = NULL; + scratch.d.xmlexpr.named_argnull = NULL; + } + + if (nargs) + { + scratch.d.xmlexpr.argvalue = + (Datum *) palloc(sizeof(Datum) * nargs); + scratch.d.xmlexpr.argnull = + (bool *) palloc(sizeof(bool) * nargs); + } + else + { + scratch.d.xmlexpr.argvalue = NULL; + scratch.d.xmlexpr.argnull = NULL; + } + + /* prepare argument execution */ + off = 0; + foreach(arg, xexpr->named_args) + { + Expr *e = (Expr *) lfirst(arg); + + ExecInitExprRec(e, parent, state, + &scratch.d.xmlexpr.named_argvalue[off], + &scratch.d.xmlexpr.named_argnull[off]); + off++; + } + + off = 0; + foreach(arg, xexpr->args) + { + Expr *e = (Expr *) lfirst(arg); + + ExecInitExprRec(e, parent, state, + &scratch.d.xmlexpr.argvalue[off], + &scratch.d.xmlexpr.argnull[off]); + off++; + } + + /* and evaluate the actual XML expression */ + ExprEvalPushStep(state, &scratch); + break; + } + + case T_NullTest: + { + NullTest *ntest = (NullTest *) node; + + if (ntest->nulltesttype == IS_NULL) + { + if (ntest->argisrow) + scratch.opcode = EEOP_NULLTEST_ROWISNULL; + else + scratch.opcode = EEOP_NULLTEST_ISNULL; + } + else if (ntest->nulltesttype == IS_NOT_NULL) + { + if (ntest->argisrow) + scratch.opcode = EEOP_NULLTEST_ROWISNOTNULL; + else + scratch.opcode = EEOP_NULLTEST_ISNOTNULL; + } + else + { + elog(ERROR, "unrecognized nulltesttype: %d", + (int) ntest->nulltesttype); + } + /* initialize cache in case it's a row test */ + scratch.d.nulltest_row.argdesc = NULL; + + /* first evaluate argument into result variable */ + ExecInitExprRec(ntest->arg, parent, state, + resv, resnull); + + /* then push the test of that argument */ + ExprEvalPushStep(state, &scratch); + break; + } + + case T_BooleanTest: + { + BooleanTest *btest = (BooleanTest *) node; + + /* + * Evaluate argument, directly into result datum. That's ok, + * because resv/resnull is definitely not used anywhere else, + * and will get overwritten by the below EEOP_BOOLTEST_IS_* + * step. + */ + ExecInitExprRec(btest->arg, parent, state, resv, resnull); + + switch (btest->booltesttype) + { + case IS_TRUE: + scratch.opcode = EEOP_BOOLTEST_IS_TRUE; + break; + case IS_NOT_TRUE: + scratch.opcode = EEOP_BOOLTEST_IS_NOT_TRUE; + break; + case IS_FALSE: + scratch.opcode = EEOP_BOOLTEST_IS_FALSE; + break; + case IS_NOT_FALSE: + scratch.opcode = EEOP_BOOLTEST_IS_NOT_FALSE; + break; + case IS_UNKNOWN: + /* Same as scalar IS NULL test */ + scratch.opcode = EEOP_NULLTEST_ISNULL; + break; + case IS_NOT_UNKNOWN: + /* Same as scalar IS NOT NULL test */ + scratch.opcode = EEOP_NULLTEST_ISNOTNULL; + break; + default: + elog(ERROR, "unrecognized booltesttype: %d", + (int) btest->booltesttype); + } + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_CoerceToDomain: + { + CoerceToDomain *ctest = (CoerceToDomain *) node; + + ExecInitCoerceToDomain(&scratch, ctest, parent, state, + resv, resnull); + break; + } + + case T_CoerceToDomainValue: + { + /* + * Read from location identified by innermost_domainval. Note + * that innermost_domainval could be NULL, if we're compiling + * a standalone domain check rather than one embedded in a + * larger expression. In that case we must read from + * econtext->domainValue_datum. We'll take care of that + * scenario at runtime. + */ + scratch.opcode = EEOP_DOMAIN_TESTVAL; + /* we share instruction union variant with case testval */ + scratch.d.casetest.value = state->innermost_domainval; + scratch.d.casetest.isnull = state->innermost_domainnull; + + ExprEvalPushStep(state, &scratch); + break; + } + + case T_CurrentOfExpr: + { + scratch.opcode = EEOP_CURRENTOFEXPR; + ExprEvalPushStep(state, &scratch); + break; + } + + case T_NextValueExpr: + { + NextValueExpr *nve = (NextValueExpr *) node; + + scratch.opcode = EEOP_NEXTVALUEEXPR; + scratch.d.nextvalueexpr.seqid = nve->seqid; + scratch.d.nextvalueexpr.seqtypid = nve->typeId; + + ExprEvalPushStep(state, &scratch); + break; + } + + default: + elog(ERROR, "unrecognized node type: %d", + (int) nodeTag(node)); + break; + } +} + +/* + * Add another expression evaluation step to ExprState->steps. + * + * Note that this potentially re-allocates es->steps, therefore no pointer + * into that array may be used while the expression is still being built. + */ +static void +ExprEvalPushStep(ExprState *es, const ExprEvalStep *s) +{ + if (es->steps_alloc == 0) + { + es->steps_alloc = 16; + es->steps = palloc(sizeof(ExprEvalStep) * es->steps_alloc); + } + else if (es->steps_alloc == es->steps_len) + { + es->steps_alloc *= 2; + es->steps = repalloc(es->steps, + sizeof(ExprEvalStep) * es->steps_alloc); + } + + memcpy(&es->steps[es->steps_len++], s, sizeof(ExprEvalStep)); +} + +/* + * Perform setup necessary for the evaluation of a function-like expression, + * appending argument evaluation steps to the steps list in *state, and + * setting up *scratch so it is ready to be pushed. + * + * *scratch is not pushed here, so that callers may override the opcode, + * which is useful for function-like cases like DISTINCT. + */ +static void +ExecInitFunc(ExprEvalStep *scratch, Expr *node, List *args, Oid funcid, + Oid inputcollid, PlanState *parent, ExprState *state) +{ + int nargs = list_length(args); + AclResult aclresult; + FmgrInfo *flinfo; + FunctionCallInfo fcinfo; + int argno; + ListCell *lc; + + /* Check permission to call function */ + aclresult = pg_proc_aclcheck(funcid, GetUserId(), ACL_EXECUTE); + if (aclresult != ACLCHECK_OK) + aclcheck_error(aclresult, ACL_KIND_PROC, get_func_name(funcid)); + InvokeFunctionExecuteHook(funcid); + + /* + * Safety check on nargs. Under normal circumstances this should never + * fail, as parser should check sooner. But possibly it might fail if + * server has been compiled with FUNC_MAX_ARGS smaller than some functions + * declared in pg_proc? + */ + if (nargs > FUNC_MAX_ARGS) + ereport(ERROR, + (errcode(ERRCODE_TOO_MANY_ARGUMENTS), + errmsg_plural("cannot pass more than %d argument to a function", + "cannot pass more than %d arguments to a function", + FUNC_MAX_ARGS, + FUNC_MAX_ARGS))); + + /* Allocate function lookup data and parameter workspace for this call */ + scratch->d.func.finfo = palloc0(sizeof(FmgrInfo)); + scratch->d.func.fcinfo_data = palloc0(sizeof(FunctionCallInfoData)); + flinfo = scratch->d.func.finfo; + fcinfo = scratch->d.func.fcinfo_data; + + /* Set up the primary fmgr lookup information */ + fmgr_info(funcid, flinfo); + fmgr_info_set_expr((Node *) node, flinfo); + + /* Initialize function call parameter structure too */ + InitFunctionCallInfoData(*fcinfo, flinfo, + nargs, inputcollid, NULL, NULL); + + /* Keep extra copies of this info to save an indirection at runtime */ + scratch->d.func.fn_addr = flinfo->fn_addr; + scratch->d.func.nargs = nargs; + + /* We only support non-set functions here */ + if (flinfo->fn_retset) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("set-valued function called in context that cannot accept a set"), + parent ? executor_errposition(parent->state, + exprLocation((Node *) node)) : 0)); + + /* Build code to evaluate arguments directly into the fcinfo struct */ + argno = 0; + foreach(lc, args) + { + Expr *arg = (Expr *) lfirst(lc); + + if (IsA(arg, Const)) + { + /* + * Don't evaluate const arguments every round; especially + * interesting for constants in comparisons. + */ + Const *con = (Const *) arg; + + fcinfo->arg[argno] = con->constvalue; + fcinfo->argnull[argno] = con->constisnull; + } + else + { + ExecInitExprRec(arg, parent, state, + &fcinfo->arg[argno], &fcinfo->argnull[argno]); + } + argno++; + } + + /* Insert appropriate opcode depending on strictness and stats level */ + if (pgstat_track_functions <= flinfo->fn_stats) + { + if (flinfo->fn_strict && nargs > 0) + scratch->opcode = EEOP_FUNCEXPR_STRICT; + else + scratch->opcode = EEOP_FUNCEXPR; + } + else + { + if (flinfo->fn_strict && nargs > 0) + scratch->opcode = EEOP_FUNCEXPR_STRICT_FUSAGE; + else + scratch->opcode = EEOP_FUNCEXPR_FUSAGE; + } +} + +/* + * Add expression steps deforming the ExprState's inner/outer/scan slots + * as much as required by the expression. + */ +static void +ExecInitExprSlots(ExprState *state, Node *node) +{ + LastAttnumInfo info = {0, 0, 0}; + ExprEvalStep scratch; + + /* + * Figure out which attributes we're going to need. + */ + get_last_attnums_walker(node, &info); + + /* Emit steps as needed */ + if (info.last_inner > 0) + { + scratch.opcode = EEOP_INNER_FETCHSOME; + scratch.d.fetch.last_var = info.last_inner; + ExprEvalPushStep(state, &scratch); + } + if (info.last_outer > 0) + { + scratch.opcode = EEOP_OUTER_FETCHSOME; + scratch.d.fetch.last_var = info.last_outer; + ExprEvalPushStep(state, &scratch); + } + if (info.last_scan > 0) + { + scratch.opcode = EEOP_SCAN_FETCHSOME; + scratch.d.fetch.last_var = info.last_scan; + ExprEvalPushStep(state, &scratch); + } +} + +/* + * get_last_attnums_walker: expression walker for ExecInitExprSlots + */ +static bool +get_last_attnums_walker(Node *node, LastAttnumInfo *info) +{ + if (node == NULL) + return false; + if (IsA(node, Var)) + { + Var *variable = (Var *) node; + AttrNumber attnum = variable->varattno; + + switch (variable->varno) + { + case INNER_VAR: + info->last_inner = Max(info->last_inner, attnum); + break; + + case OUTER_VAR: + info->last_outer = Max(info->last_outer, attnum); + break; + + /* INDEX_VAR is handled by default case */ + + default: + info->last_scan = Max(info->last_scan, attnum); + break; + } + return false; + } + + /* + * Don't examine the arguments or filters of Aggrefs or WindowFuncs, + * because those do not represent expressions to be evaluated within the + * calling expression's econtext. GroupingFunc arguments are never + * evaluated at all. + */ + if (IsA(node, Aggref)) + return false; + if (IsA(node, WindowFunc)) + return false; + if (IsA(node, GroupingFunc)) + return false; + return expression_tree_walker(node, get_last_attnums_walker, + (void *) info); +} + +/* + * Prepare step for the evaluation of a whole-row variable. + * The caller still has to push the step. + */ +static void +ExecInitWholeRowVar(ExprEvalStep *scratch, Var *variable, PlanState *parent) +{ + /* fill in all but the target */ + scratch->opcode = EEOP_WHOLEROW; + scratch->d.wholerow.var = variable; + scratch->d.wholerow.first = true; + scratch->d.wholerow.slow = false; + scratch->d.wholerow.tupdesc = NULL; /* filled at runtime */ + scratch->d.wholerow.junkFilter = NULL; + + /* + * If the input tuple came from a subquery, it might contain "resjunk" + * columns (such as GROUP BY or ORDER BY columns), which we don't want to + * keep in the whole-row result. We can get rid of such columns by + * passing the tuple through a JunkFilter --- but to make one, we have to + * lay our hands on the subquery's targetlist. Fortunately, there are not + * very many cases where this can happen, and we can identify all of them + * by examining our parent PlanState. We assume this is not an issue in + * standalone expressions that don't have parent plans. (Whole-row Vars + * can occur in such expressions, but they will always be referencing + * table rows.) + */ + if (parent) + { + PlanState *subplan = NULL; + + switch (nodeTag(parent)) + { + case T_SubqueryScanState: + subplan = ((SubqueryScanState *) parent)->subplan; + break; + case T_CteScanState: + subplan = ((CteScanState *) parent)->cteplanstate; + break; + default: + break; + } + + if (subplan) + { + bool junk_filter_needed = false; + ListCell *tlist; + + /* Detect whether subplan tlist actually has any junk columns */ + foreach(tlist, subplan->plan->targetlist) + { + TargetEntry *tle = (TargetEntry *) lfirst(tlist); + + if (tle->resjunk) + { + junk_filter_needed = true; + break; + } + } + + /* If so, build the junkfilter now */ + if (junk_filter_needed) + { + scratch->d.wholerow.junkFilter = + ExecInitJunkFilter(subplan->plan->targetlist, + ExecGetResultType(subplan)->tdhasoid, + ExecInitExtraTupleSlot(parent->state)); + } + } + } +} + +/* + * Prepare evaluation of an ArrayRef expression. + */ +static void +ExecInitArrayRef(ExprEvalStep *scratch, ArrayRef *aref, PlanState *parent, + ExprState *state, Datum *resv, bool *resnull) +{ + bool isAssignment = (aref->refassgnexpr != NULL); + ArrayRefState *arefstate = palloc0(sizeof(ArrayRefState)); + List *adjust_jumps = NIL; + ListCell *lc; + int i; + + /* Fill constant fields of ArrayRefState */ + arefstate->isassignment = isAssignment; + arefstate->refelemtype = aref->refelemtype; + arefstate->refattrlength = get_typlen(aref->refarraytype); + get_typlenbyvalalign(aref->refelemtype, + &arefstate->refelemlength, + &arefstate->refelembyval, + &arefstate->refelemalign); + + /* + * Evaluate array input. It's safe to do so into resv/resnull, because we + * won't use that as target for any of the other subexpressions, and it'll + * be overwritten by the final EEOP_ARRAYREF_FETCH/ASSIGN step, which is + * pushed last. + */ + ExecInitExprRec(aref->refexpr, parent, state, resv, resnull); + + /* + * If refexpr yields NULL, and it's a fetch, then result is NULL. We can + * implement this with just JUMP_IF_NULL, since we evaluated the array + * into the desired target location. + */ + if (!isAssignment) + { + scratch->opcode = EEOP_JUMP_IF_NULL; + scratch->d.jump.jumpdone = -1; /* adjust later */ + ExprEvalPushStep(state, scratch); + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + } + + /* Verify subscript list lengths are within limit */ + if (list_length(aref->refupperindexpr) > MAXDIM) + ereport(ERROR, + (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED), + errmsg("number of array dimensions (%d) exceeds the maximum allowed (%d)", + list_length(aref->refupperindexpr), MAXDIM))); + + if (list_length(aref->reflowerindexpr) > MAXDIM) + ereport(ERROR, + (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED), + errmsg("number of array dimensions (%d) exceeds the maximum allowed (%d)", + list_length(aref->reflowerindexpr), MAXDIM))); + + /* Evaluate upper subscripts */ + i = 0; + foreach(lc, aref->refupperindexpr) + { + Expr *e = (Expr *) lfirst(lc); + + /* When slicing, individual subscript bounds can be omitted */ + if (!e) + { + arefstate->upperprovided[i] = false; + i++; + continue; + } + + arefstate->upperprovided[i] = true; + + /* Each subscript is evaluated into subscriptvalue/subscriptnull */ + ExecInitExprRec(e, parent, state, + &arefstate->subscriptvalue, &arefstate->subscriptnull); + + /* ... and then ARRAYREF_SUBSCRIPT saves it into step's workspace */ + scratch->opcode = EEOP_ARRAYREF_SUBSCRIPT; + scratch->d.arrayref_subscript.state = arefstate; + scratch->d.arrayref_subscript.off = i; + scratch->d.arrayref_subscript.isupper = true; + scratch->d.arrayref_subscript.jumpdone = -1; /* adjust later */ + ExprEvalPushStep(state, scratch); + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + i++; + } + arefstate->numupper = i; + + /* Evaluate lower subscripts similarly */ + i = 0; + foreach(lc, aref->reflowerindexpr) + { + Expr *e = (Expr *) lfirst(lc); + + /* When slicing, individual subscript bounds can be omitted */ + if (!e) + { + arefstate->lowerprovided[i] = false; + i++; + continue; + } + + arefstate->lowerprovided[i] = true; + + /* Each subscript is evaluated into subscriptvalue/subscriptnull */ + ExecInitExprRec(e, parent, state, + &arefstate->subscriptvalue, &arefstate->subscriptnull); + + /* ... and then ARRAYREF_SUBSCRIPT saves it into step's workspace */ + scratch->opcode = EEOP_ARRAYREF_SUBSCRIPT; + scratch->d.arrayref_subscript.state = arefstate; + scratch->d.arrayref_subscript.off = i; + scratch->d.arrayref_subscript.isupper = false; + scratch->d.arrayref_subscript.jumpdone = -1; /* adjust later */ + ExprEvalPushStep(state, scratch); + adjust_jumps = lappend_int(adjust_jumps, + state->steps_len - 1); + i++; + } + arefstate->numlower = i; + + /* Should be impossible if parser is sane, but check anyway: */ + if (arefstate->numlower != 0 && + arefstate->numupper != arefstate->numlower) + elog(ERROR, "upper and lower index lists are not same length"); + + if (isAssignment) + { + Datum *save_innermost_caseval; + bool *save_innermost_casenull; + + /* + * We might have a nested-assignment situation, in which the + * refassgnexpr is itself a FieldStore or ArrayRef that needs to + * obtain and modify the previous value of the array element or slice + * being replaced. If so, we have to extract that value from the + * array and pass it down via the CaseTextExpr mechanism. It's safe + * to reuse the CASE mechanism because there cannot be a CASE between + * here and where the value would be needed, and an array assignment + * can't be within a CASE either. (So saving and restoring + * innermost_caseval is just paranoia, but let's do it anyway.) + * + * Since fetching the old element might be a nontrivial expense, do it + * only if the argument appears to actually need it. + */ + if (isAssignmentIndirectionExpr(aref->refassgnexpr)) + { + scratch->opcode = EEOP_ARRAYREF_OLD; + scratch->d.arrayref.state = arefstate; + ExprEvalPushStep(state, scratch); + } + + /* ARRAYREF_OLD puts extracted value into prevvalue/prevnull */ + save_innermost_caseval = state->innermost_caseval; + save_innermost_casenull = state->innermost_casenull; + state->innermost_caseval = &arefstate->prevvalue; + state->innermost_casenull = &arefstate->prevnull; + + /* evaluate replacement value into replacevalue/replacenull */ + ExecInitExprRec(aref->refassgnexpr, parent, state, + &arefstate->replacevalue, &arefstate->replacenull); + + state->innermost_caseval = save_innermost_caseval; + state->innermost_casenull = save_innermost_casenull; + + /* and perform the assignment */ + scratch->opcode = EEOP_ARRAYREF_ASSIGN; + scratch->d.arrayref.state = arefstate; + ExprEvalPushStep(state, scratch); + } + else + { + /* array fetch is much simpler */ + scratch->opcode = EEOP_ARRAYREF_FETCH; + scratch->d.arrayref.state = arefstate; + ExprEvalPushStep(state, scratch); + } + + /* adjust jump targets */ + foreach(lc, adjust_jumps) + { + ExprEvalStep *as = &state->steps[lfirst_int(lc)]; + + if (as->opcode == EEOP_ARRAYREF_SUBSCRIPT) + { + Assert(as->d.arrayref_subscript.jumpdone == -1); + as->d.arrayref_subscript.jumpdone = state->steps_len; + } + else + { + Assert(as->opcode == EEOP_JUMP_IF_NULL); + Assert(as->d.jump.jumpdone == -1); + as->d.jump.jumpdone = state->steps_len; + } + } +} + +/* + * Helper for preparing ArrayRef expressions for evaluation: is expr a nested + * FieldStore or ArrayRef that might need the old element value passed down? + * + * (We could use this in FieldStore too, but in that case passing the old + * value is so cheap there's no need.) + */ +static bool +isAssignmentIndirectionExpr(Expr *expr) +{ + if (expr == NULL) + return false; /* just paranoia */ + if (IsA(expr, FieldStore)) + { + FieldStore *fstore = (FieldStore *) expr; + + if (fstore->arg && IsA(fstore->arg, CaseTestExpr)) + return true; + } + else if (IsA(expr, ArrayRef)) + { + ArrayRef *arrayRef = (ArrayRef *) expr; + + if (arrayRef->refexpr && IsA(arrayRef->refexpr, CaseTestExpr)) + return true; + } + return false; +} + +/* + * Prepare evaluation of a CoerceToDomain expression. + */ +static void +ExecInitCoerceToDomain(ExprEvalStep *scratch, CoerceToDomain *ctest, + PlanState *parent, ExprState *state, + Datum *resv, bool *resnull) +{ + ExprEvalStep scratch2; + DomainConstraintRef *constraint_ref; + Datum *domainval = NULL; + bool *domainnull = NULL; + Datum *save_innermost_domainval; + bool *save_innermost_domainnull; + ListCell *l; + + scratch->d.domaincheck.resulttype = ctest->resulttype; + /* we'll allocate workspace only if needed */ + scratch->d.domaincheck.checkvalue = NULL; + scratch->d.domaincheck.checknull = NULL; + + /* + * Evaluate argument - it's fine to directly store it into resv/resnull, + * if there's constraint failures there'll be errors, otherwise it's what + * needs to be returned. + */ + ExecInitExprRec(ctest->arg, parent, state, resv, resnull); + + /* + * Note: if the argument is of varlena type, it could be a R/W expanded + * object. We want to return the R/W pointer as the final result, but we + * have to pass a R/O pointer as the value to be tested by any functions + * in check expressions. We don't bother to emit a MAKE_READONLY step + * unless there's actually at least one check expression, though. Until + * we've tested that, domainval/domainnull are NULL. + */ + + /* + * Collect the constraints associated with the domain. + * + * Note: before PG v10 we'd recheck the set of constraints during each + * evaluation of the expression. Now we bake them into the ExprState + * during executor initialization. That means we don't need typcache.c to + * provide compiled exprs. + */ + constraint_ref = (DomainConstraintRef *) + palloc(sizeof(DomainConstraintRef)); + InitDomainConstraintRef(ctest->resulttype, + constraint_ref, + CurrentMemoryContext, + false); + + /* + * Compile code to check each domain constraint. NOTNULL constraints can + * just be applied on the resv/resnull value, but for CHECK constraints we + * need more pushups. + */ + foreach(l, constraint_ref->constraints) + { + DomainConstraintState *con = (DomainConstraintState *) lfirst(l); + + scratch->d.domaincheck.constraintname = con->name; + + switch (con->constrainttype) + { + case DOM_CONSTRAINT_NOTNULL: + scratch->opcode = EEOP_DOMAIN_NOTNULL; + ExprEvalPushStep(state, scratch); + break; + case DOM_CONSTRAINT_CHECK: + /* Allocate workspace for CHECK output if we didn't yet */ + if (scratch->d.domaincheck.checkvalue == NULL) + { + scratch->d.domaincheck.checkvalue = + (Datum *) palloc(sizeof(Datum)); + scratch->d.domaincheck.checknull = + (bool *) palloc(sizeof(bool)); + } + + /* + * If first time through, determine where CoerceToDomainValue + * nodes should read from. + */ + if (domainval == NULL) + { + /* + * Since value might be read multiple times, force to R/O + * - but only if it could be an expanded datum. + */ + if (get_typlen(ctest->resulttype) == -1) + { + /* Yes, so make output workspace for MAKE_READONLY */ + domainval = (Datum *) palloc(sizeof(Datum)); + domainnull = (bool *) palloc(sizeof(bool)); + + /* Emit MAKE_READONLY */ + scratch2.opcode = EEOP_MAKE_READONLY; + scratch2.resvalue = domainval; + scratch2.resnull = domainnull; + scratch2.d.make_readonly.value = resv; + scratch2.d.make_readonly.isnull = resnull; + ExprEvalPushStep(state, &scratch2); + } + else + { + /* No, so it's fine to read from resv/resnull */ + domainval = resv; + domainnull = resnull; + } + } + + /* + * Set up value to be returned by CoerceToDomainValue nodes. + * We must save and restore innermost_domainval/null fields, + * in case this node is itself within a check expression for + * another domain. + */ + save_innermost_domainval = state->innermost_domainval; + save_innermost_domainnull = state->innermost_domainnull; + state->innermost_domainval = domainval; + state->innermost_domainnull = domainnull; + + /* evaluate check expression value */ + ExecInitExprRec(con->check_expr, parent, state, + scratch->d.domaincheck.checkvalue, + scratch->d.domaincheck.checknull); + + state->innermost_domainval = save_innermost_domainval; + state->innermost_domainnull = save_innermost_domainnull; + + /* now test result */ + scratch->opcode = EEOP_DOMAIN_CHECK; + ExprEvalPushStep(state, scratch); + + break; + default: + elog(ERROR, "unrecognized constraint type: %d", + (int) con->constrainttype); + break; + } + } +} diff --git a/src/backend/executor/execExprInterp.c b/src/backend/executor/execExprInterp.c new file mode 100644 index 0000000000..fed0052fc6 --- /dev/null +++ b/src/backend/executor/execExprInterp.c @@ -0,0 +1,3565 @@ +/*------------------------------------------------------------------------- + * + * execExprInterp.c + * Interpreted evaluation of an expression step list. + * + * This file provides either a "direct threaded" (for gcc, clang and + * compatible) or a "switch threaded" (for all compilers) implementation of + * expression evaluation. The former is amongst the fastest known methods + * of interpreting programs without resorting to assembly level work, or + * just-in-time compilation, but it requires support for computed gotos. + * The latter is amongst the fastest approaches doable in standard C. + * + * In either case we use ExprEvalStep->opcode to dispatch to the code block + * within ExecInterpExpr() that implements the specific opcode type. + * + * Switch-threading uses a plain switch() statement to perform the + * dispatch. This has the advantages of being plain C and allowing the + * compiler to warn if implementation of a specific opcode has been forgotten. + * The disadvantage is that dispatches will, as commonly implemented by + * compilers, happen from a single location, requiring more jumps and causing + * bad branch prediction. + * + * In direct threading, we use gcc's label-as-values extension - also adopted + * by some other compilers - to replace ExprEvalStep->opcode with the address + * of the block implementing the instruction. Dispatch to the next instruction + * is done by a "computed goto". This allows for better branch prediction + * (as the jumps are happening from different locations) and fewer jumps + * (as no preparatory jump to a common dispatch location is needed). + * + * When using direct threading, ExecReadyInterpretedExpr will replace + * each step's opcode field with the address of the relevant code block and + * ExprState->flags will contain EEO_FLAG_DIRECT_THREADED to remember that + * that's been done. + * + * For very simple instructions the overhead of the full interpreter + * "startup", as minimal as it is, is noticeable. Therefore + * ExecReadyInterpretedExpr will choose to implement simple scalar Var + * and Const expressions using special fast-path routines (ExecJust*). + * Benchmarking shows anything more complex than those may as well use the + * "full interpreter". + * + * Complex or uncommon instructions are not implemented in-line in + * ExecInterpExpr(), rather we call out to a helper function appearing later + * in this file. For one reason, there'd not be a noticeable performance + * benefit, but more importantly those complex routines are intended to be + * shared between different expression evaluation approaches. For instance + * a JIT compiler would generate calls to them. (This is why they are + * exported rather than being "static" in this file.) + * + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * IDENTIFICATION + * src/backend/executor/execExprInterp.c + * + *------------------------------------------------------------------------- + */ +#include "postgres.h" + +#include "access/tuptoaster.h" +#include "catalog/pg_type.h" +#include "commands/sequence.h" +#include "executor/execExpr.h" +#include "executor/nodeSubplan.h" +#include "funcapi.h" +#include "miscadmin.h" +#include "nodes/nodeFuncs.h" +#include "parser/parsetree.h" +#include "pgstat.h" +#include "utils/builtins.h" +#include "utils/date.h" +#include "utils/lsyscache.h" +#include "utils/timestamp.h" +#include "utils/typcache.h" +#include "utils/xml.h" + + +/* + * Use computed-goto-based opcode dispatch when computed gotos are available. + * But use a separate symbol so that it's easy to adjust locally in this file + * for development and testing. + */ +#ifdef HAVE_COMPUTED_GOTO +#define EEO_USE_COMPUTED_GOTO +#endif /* HAVE_COMPUTED_GOTO */ + +/* + * Macros for opcode dispatch. + * + * EEO_SWITCH - just hides the switch if not in use. + * EEO_CASE - labels the implementation of named expression step type. + * EEO_DISPATCH - jump to the implementation of the step type for 'op'. + * EEO_OPCODE - compute opcode required by used expression evaluation method. + * EEO_NEXT - increment 'op' and jump to correct next step type. + * EEO_JUMP - jump to the specified step number within the current expression. + */ +#if defined(EEO_USE_COMPUTED_GOTO) + +/* to make dispatch_table accessible outside ExecInterpExpr() */ +static const void **dispatch_table = NULL; + +#define EEO_SWITCH() +#define EEO_CASE(name) CASE_##name: +#define EEO_DISPATCH() goto *((void *) op->opcode) +#define EEO_OPCODE(opcode) ((intptr_t) dispatch_table[opcode]) + +#else /* !EEO_USE_COMPUTED_GOTO */ + +#define EEO_SWITCH() starteval: switch ((ExprEvalOp) op->opcode) +#define EEO_CASE(name) case name: +#define EEO_DISPATCH() goto starteval +#define EEO_OPCODE(opcode) (opcode) + +#endif /* EEO_USE_COMPUTED_GOTO */ + +#define EEO_NEXT() \ + do { \ + op++; \ + EEO_DISPATCH(); \ + } while (0) + +#define EEO_JUMP(stepno) \ + do { \ + op = &state->steps[stepno]; \ + EEO_DISPATCH(); \ + } while (0) + + +static Datum ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull); +static void ExecInitInterpreter(void); + +/* support functions */ +static void CheckVarSlotCompatibility(TupleTableSlot *slot, int attnum, Oid vartype); +static TupleDesc get_cached_rowtype(Oid type_id, int32 typmod, + TupleDesc *cache_field, ExprContext *econtext); +static void ShutdownTupleDescRef(Datum arg); +static void ExecEvalRowNullInt(ExprState *state, ExprEvalStep *op, + ExprContext *econtext, bool checkisnull); + +/* fast-path evaluation functions */ +static Datum ExecJustInnerVarFirst(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustInnerVar(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustOuterVarFirst(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustOuterVar(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustScanVarFirst(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustScanVar(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustConst(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustAssignInnerVar(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustAssignOuterVar(ExprState *state, ExprContext *econtext, bool *isnull); +static Datum ExecJustAssignScanVar(ExprState *state, ExprContext *econtext, bool *isnull); + + +/* + * Prepare ExprState for interpreted execution. + */ +void +ExecReadyInterpretedExpr(ExprState *state) +{ + /* Ensure one-time interpreter setup has been done */ + ExecInitInterpreter(); + + /* Simple validity checks on expression */ + Assert(state->steps_len >= 1); + Assert(state->steps[state->steps_len - 1].opcode == EEOP_DONE); + + /* + * Don't perform redundant initialization. This is unreachable in current + * cases, but might be hit if there's additional expression evaluation + * methods that rely on interpreted execution to work. + */ + if (state->flags & EEO_FLAG_INTERPRETER_INITIALIZED) + return; + + /* DIRECT_THREADED should not already be set */ + Assert((state->flags & EEO_FLAG_DIRECT_THREADED) == 0); + + /* + * There shouldn't be any errors before the expression is fully + * initialized, and even if so, it'd lead to the expression being + * abandoned. So we can set the flag now and save some code. + */ + state->flags |= EEO_FLAG_INTERPRETER_INITIALIZED; + + /* + * Select fast-path evalfuncs for very simple expressions. "Starting up" + * the full interpreter is a measurable overhead for these. Plain Vars + * and Const seem to be the only ones where the intrinsic cost is small + * enough that the overhead of ExecInterpExpr matters. For more complex + * expressions it's cheaper to use ExecInterpExpr always. + */ + if (state->steps_len == 3) + { + ExprEvalOp step0 = state->steps[0].opcode; + ExprEvalOp step1 = state->steps[1].opcode; + + if (step0 == EEOP_INNER_FETCHSOME && + step1 == EEOP_INNER_VAR_FIRST) + { + state->evalfunc = ExecJustInnerVarFirst; + return; + } + else if (step0 == EEOP_OUTER_FETCHSOME && + step1 == EEOP_OUTER_VAR_FIRST) + { + state->evalfunc = ExecJustOuterVarFirst; + return; + } + else if (step0 == EEOP_SCAN_FETCHSOME && + step1 == EEOP_SCAN_VAR_FIRST) + { + state->evalfunc = ExecJustScanVarFirst; + return; + } + else if (step0 == EEOP_INNER_FETCHSOME && + step1 == EEOP_ASSIGN_INNER_VAR) + { + state->evalfunc = ExecJustAssignInnerVar; + return; + } + else if (step0 == EEOP_OUTER_FETCHSOME && + step1 == EEOP_ASSIGN_OUTER_VAR) + { + state->evalfunc = ExecJustAssignOuterVar; + return; + } + else if (step0 == EEOP_SCAN_FETCHSOME && + step1 == EEOP_ASSIGN_SCAN_VAR) + { + state->evalfunc = ExecJustAssignScanVar; + return; + } + } + else if (state->steps_len == 2 && + state->steps[0].opcode == EEOP_CONST) + { + state->evalfunc = ExecJustConst; + return; + } + +#if defined(EEO_USE_COMPUTED_GOTO) + + /* + * In the direct-threaded implementation, replace each opcode with the + * address to jump to. (Use ExecEvalStepOp() to get back the opcode.) + */ + { + int off; + + for (off = 0; off < state->steps_len; off++) + { + ExprEvalStep *op = &state->steps[off]; + + op->opcode = EEO_OPCODE(op->opcode); + } + + state->flags |= EEO_FLAG_DIRECT_THREADED; + } +#endif /* EEO_USE_COMPUTED_GOTO */ + + state->evalfunc = ExecInterpExpr; +} + + +/* + * Evaluate expression identified by "state" in the execution context + * given by "econtext". *isnull is set to the is-null flag for the result, + * and the Datum value is the function result. + * + * As a special case, return the dispatch table's address if state is NULL. + * This is used by ExecInitInterpreter to set up the dispatch_table global. + * (Only applies when EEO_USE_COMPUTED_GOTO is defined.) + */ +static Datum +ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op; + TupleTableSlot *resultslot; + TupleTableSlot *innerslot; + TupleTableSlot *outerslot; + TupleTableSlot *scanslot; + + /* + * This array has to be in the same order as enum ExprEvalOp. + */ +#if defined(EEO_USE_COMPUTED_GOTO) + static const void *const dispatch_table[] = { + &&CASE_EEOP_DONE, + &&CASE_EEOP_INNER_FETCHSOME, + &&CASE_EEOP_OUTER_FETCHSOME, + &&CASE_EEOP_SCAN_FETCHSOME, + &&CASE_EEOP_INNER_VAR_FIRST, + &&CASE_EEOP_INNER_VAR, + &&CASE_EEOP_OUTER_VAR_FIRST, + &&CASE_EEOP_OUTER_VAR, + &&CASE_EEOP_SCAN_VAR_FIRST, + &&CASE_EEOP_SCAN_VAR, + &&CASE_EEOP_INNER_SYSVAR, + &&CASE_EEOP_OUTER_SYSVAR, + &&CASE_EEOP_SCAN_SYSVAR, + &&CASE_EEOP_WHOLEROW, + &&CASE_EEOP_ASSIGN_INNER_VAR, + &&CASE_EEOP_ASSIGN_OUTER_VAR, + &&CASE_EEOP_ASSIGN_SCAN_VAR, + &&CASE_EEOP_ASSIGN_TMP, + &&CASE_EEOP_ASSIGN_TMP_MAKE_RO, + &&CASE_EEOP_CONST, + &&CASE_EEOP_FUNCEXPR, + &&CASE_EEOP_FUNCEXPR_STRICT, + &&CASE_EEOP_FUNCEXPR_FUSAGE, + &&CASE_EEOP_FUNCEXPR_STRICT_FUSAGE, + &&CASE_EEOP_BOOL_AND_STEP_FIRST, + &&CASE_EEOP_BOOL_AND_STEP, + &&CASE_EEOP_BOOL_AND_STEP_LAST, + &&CASE_EEOP_BOOL_OR_STEP_FIRST, + &&CASE_EEOP_BOOL_OR_STEP, + &&CASE_EEOP_BOOL_OR_STEP_LAST, + &&CASE_EEOP_BOOL_NOT_STEP, + &&CASE_EEOP_QUAL, + &&CASE_EEOP_JUMP, + &&CASE_EEOP_JUMP_IF_NULL, + &&CASE_EEOP_JUMP_IF_NOT_NULL, + &&CASE_EEOP_JUMP_IF_NOT_TRUE, + &&CASE_EEOP_NULLTEST_ISNULL, + &&CASE_EEOP_NULLTEST_ISNOTNULL, + &&CASE_EEOP_NULLTEST_ROWISNULL, + &&CASE_EEOP_NULLTEST_ROWISNOTNULL, + &&CASE_EEOP_BOOLTEST_IS_TRUE, + &&CASE_EEOP_BOOLTEST_IS_NOT_TRUE, + &&CASE_EEOP_BOOLTEST_IS_FALSE, + &&CASE_EEOP_BOOLTEST_IS_NOT_FALSE, + &&CASE_EEOP_PARAM_EXEC, + &&CASE_EEOP_PARAM_EXTERN, + &&CASE_EEOP_CASE_TESTVAL, + &&CASE_EEOP_MAKE_READONLY, + &&CASE_EEOP_IOCOERCE, + &&CASE_EEOP_DISTINCT, + &&CASE_EEOP_NULLIF, + &&CASE_EEOP_SQLVALUEFUNCTION, + &&CASE_EEOP_CURRENTOFEXPR, + &&CASE_EEOP_NEXTVALUEEXPR, + &&CASE_EEOP_ARRAYEXPR, + &&CASE_EEOP_ARRAYCOERCE, + &&CASE_EEOP_ROW, + &&CASE_EEOP_ROWCOMPARE_STEP, + &&CASE_EEOP_ROWCOMPARE_FINAL, + &&CASE_EEOP_MINMAX, + &&CASE_EEOP_FIELDSELECT, + &&CASE_EEOP_FIELDSTORE_DEFORM, + &&CASE_EEOP_FIELDSTORE_FORM, + &&CASE_EEOP_ARRAYREF_SUBSCRIPT, + &&CASE_EEOP_ARRAYREF_OLD, + &&CASE_EEOP_ARRAYREF_ASSIGN, + &&CASE_EEOP_ARRAYREF_FETCH, + &&CASE_EEOP_DOMAIN_TESTVAL, + &&CASE_EEOP_DOMAIN_NOTNULL, + &&CASE_EEOP_DOMAIN_CHECK, + &&CASE_EEOP_CONVERT_ROWTYPE, + &&CASE_EEOP_SCALARARRAYOP, + &&CASE_EEOP_XMLEXPR, + &&CASE_EEOP_AGGREF, + &&CASE_EEOP_GROUPING_FUNC, + &&CASE_EEOP_WINDOW_FUNC, + &&CASE_EEOP_SUBPLAN, + &&CASE_EEOP_ALTERNATIVE_SUBPLAN, + &&CASE_EEOP_LAST + }; + + StaticAssertStmt(EEOP_LAST + 1 == lengthof(dispatch_table), + "dispatch_table out of whack with ExprEvalOp"); + + if (unlikely(state == NULL)) + return PointerGetDatum(dispatch_table); +#else + Assert(state != NULL); +#endif /* EEO_USE_COMPUTED_GOTO */ + + /* setup state */ + op = state->steps; + resultslot = state->resultslot; + innerslot = econtext->ecxt_innertuple; + outerslot = econtext->ecxt_outertuple; + scanslot = econtext->ecxt_scantuple; + +#if defined(EEO_USE_COMPUTED_GOTO) + EEO_DISPATCH(); +#endif + + EEO_SWITCH() + { + EEO_CASE(EEOP_DONE) + { + goto out; + } + + EEO_CASE(EEOP_INNER_FETCHSOME) + { + /* XXX: worthwhile to check tts_nvalid inline first? */ + slot_getsomeattrs(innerslot, op->d.fetch.last_var); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_OUTER_FETCHSOME) + { + slot_getsomeattrs(outerslot, op->d.fetch.last_var); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_SCAN_FETCHSOME) + { + slot_getsomeattrs(scanslot, op->d.fetch.last_var); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_INNER_VAR_FIRST) + { + int attnum = op->d.var.attnum; + + /* + * First time through, check whether attribute matches Var. Might + * not be ok anymore, due to schema changes. + */ + CheckVarSlotCompatibility(innerslot, attnum + 1, op->d.var.vartype); + + /* Skip that check on subsequent evaluations */ + op->opcode = EEO_OPCODE(EEOP_INNER_VAR); + + /* FALL THROUGH to EEOP_INNER_VAR */ + } + + EEO_CASE(EEOP_INNER_VAR) + { + int attnum = op->d.var.attnum; + + /* + * Since we already extracted all referenced columns from the + * tuple with a FETCHSOME step, we can just grab the value + * directly out of the slot's decomposed-data arrays. But let's + * have an Assert to check that that did happen. + */ + Assert(attnum >= 0 && attnum < innerslot->tts_nvalid); + *op->resvalue = innerslot->tts_values[attnum]; + *op->resnull = innerslot->tts_isnull[attnum]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_OUTER_VAR_FIRST) + { + int attnum = op->d.var.attnum; + + /* See EEOP_INNER_VAR_FIRST comments */ + + CheckVarSlotCompatibility(outerslot, attnum + 1, op->d.var.vartype); + op->opcode = EEO_OPCODE(EEOP_OUTER_VAR); + + /* FALL THROUGH to EEOP_OUTER_VAR */ + } + + EEO_CASE(EEOP_OUTER_VAR) + { + int attnum = op->d.var.attnum; + + /* See EEOP_INNER_VAR comments */ + + Assert(attnum >= 0 && attnum < outerslot->tts_nvalid); + *op->resvalue = outerslot->tts_values[attnum]; + *op->resnull = outerslot->tts_isnull[attnum]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_SCAN_VAR_FIRST) + { + int attnum = op->d.var.attnum; + + /* See EEOP_INNER_VAR_FIRST comments */ + + CheckVarSlotCompatibility(scanslot, attnum + 1, op->d.var.vartype); + op->opcode = EEO_OPCODE(EEOP_SCAN_VAR); + + /* FALL THROUGH to EEOP_SCAN_VAR */ + } + + EEO_CASE(EEOP_SCAN_VAR) + { + int attnum = op->d.var.attnum; + + /* See EEOP_INNER_VAR comments */ + + Assert(attnum >= 0 && attnum < scanslot->tts_nvalid); + *op->resvalue = scanslot->tts_values[attnum]; + *op->resnull = scanslot->tts_isnull[attnum]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_INNER_SYSVAR) + { + int attnum = op->d.var.attnum; + + /* these asserts must match defenses in slot_getattr */ + Assert(innerslot->tts_tuple != NULL); + Assert(innerslot->tts_tuple != &(innerslot->tts_minhdr)); + /* heap_getsysattr has sufficient defenses against bad attnums */ + + *op->resvalue = heap_getsysattr(innerslot->tts_tuple, attnum, + innerslot->tts_tupleDescriptor, + op->resnull); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_OUTER_SYSVAR) + { + int attnum = op->d.var.attnum; + + /* these asserts must match defenses in slot_getattr */ + Assert(outerslot->tts_tuple != NULL); + Assert(outerslot->tts_tuple != &(outerslot->tts_minhdr)); + + /* heap_getsysattr has sufficient defenses against bad attnums */ + *op->resvalue = heap_getsysattr(outerslot->tts_tuple, attnum, + outerslot->tts_tupleDescriptor, + op->resnull); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_SCAN_SYSVAR) + { + int attnum = op->d.var.attnum; + + /* these asserts must match defenses in slot_getattr */ + Assert(scanslot->tts_tuple != NULL); + Assert(scanslot->tts_tuple != &(scanslot->tts_minhdr)); + /* heap_getsysattr has sufficient defenses against bad attnums */ + + *op->resvalue = heap_getsysattr(scanslot->tts_tuple, attnum, + scanslot->tts_tupleDescriptor, + op->resnull); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_WHOLEROW) + { + /* too complex for an inline implementation */ + ExecEvalWholeRowVar(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ASSIGN_INNER_VAR) + { + int resultnum = op->d.assign_var.resultnum; + int attnum = op->d.assign_var.attnum; + + /* + * We do not need CheckVarSlotCompatibility here; that was taken + * care of at compilation time. But see EEOP_INNER_VAR comments. + */ + Assert(attnum >= 0 && attnum < innerslot->tts_nvalid); + resultslot->tts_values[resultnum] = innerslot->tts_values[attnum]; + resultslot->tts_isnull[resultnum] = innerslot->tts_isnull[attnum]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ASSIGN_OUTER_VAR) + { + int resultnum = op->d.assign_var.resultnum; + int attnum = op->d.assign_var.attnum; + + /* + * We do not need CheckVarSlotCompatibility here; that was taken + * care of at compilation time. But see EEOP_INNER_VAR comments. + */ + Assert(attnum >= 0 && attnum < outerslot->tts_nvalid); + resultslot->tts_values[resultnum] = outerslot->tts_values[attnum]; + resultslot->tts_isnull[resultnum] = outerslot->tts_isnull[attnum]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ASSIGN_SCAN_VAR) + { + int resultnum = op->d.assign_var.resultnum; + int attnum = op->d.assign_var.attnum; + + /* + * We do not need CheckVarSlotCompatibility here; that was taken + * care of at compilation time. But see EEOP_INNER_VAR comments. + */ + Assert(attnum >= 0 && attnum < scanslot->tts_nvalid); + resultslot->tts_values[resultnum] = scanslot->tts_values[attnum]; + resultslot->tts_isnull[resultnum] = scanslot->tts_isnull[attnum]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ASSIGN_TMP) + { + int resultnum = op->d.assign_tmp.resultnum; + + resultslot->tts_values[resultnum] = state->resvalue; + resultslot->tts_isnull[resultnum] = state->resnull; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ASSIGN_TMP_MAKE_RO) + { + int resultnum = op->d.assign_tmp.resultnum; + + resultslot->tts_isnull[resultnum] = state->resnull; + if (!resultslot->tts_isnull[resultnum]) + resultslot->tts_values[resultnum] = + MakeExpandedObjectReadOnlyInternal(state->resvalue); + else + resultslot->tts_values[resultnum] = state->resvalue; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_CONST) + { + *op->resnull = op->d.constval.isnull; + *op->resvalue = op->d.constval.value; + + EEO_NEXT(); + } + + /* + * Function-call implementations. Arguments have previously been + * evaluated directly into fcinfo->args. + * + * As both STRICT checks and function-usage are noticeable performance + * wise, and function calls are a very hot-path (they also back + * operators!), it's worth having so many separate opcodes. + */ + EEO_CASE(EEOP_FUNCEXPR) + { + FunctionCallInfo fcinfo = op->d.func.fcinfo_data; + + fcinfo->isnull = false; + *op->resvalue = (op->d.func.fn_addr) (fcinfo); + *op->resnull = fcinfo->isnull; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_FUNCEXPR_STRICT) + { + FunctionCallInfo fcinfo = op->d.func.fcinfo_data; + bool *argnull = fcinfo->argnull; + int argno; + + /* strict function, so check for NULL args */ + for (argno = 0; argno < op->d.func.nargs; argno++) + { + if (argnull[argno]) + { + *op->resnull = true; + goto strictfail; + } + } + fcinfo->isnull = false; + *op->resvalue = (op->d.func.fn_addr) (fcinfo); + *op->resnull = fcinfo->isnull; + + strictfail: + EEO_NEXT(); + } + + EEO_CASE(EEOP_FUNCEXPR_FUSAGE) + { + FunctionCallInfo fcinfo = op->d.func.fcinfo_data; + PgStat_FunctionCallUsage fcusage; + + pgstat_init_function_usage(fcinfo, &fcusage); + + fcinfo->isnull = false; + *op->resvalue = (op->d.func.fn_addr) (fcinfo); + *op->resnull = fcinfo->isnull; + + pgstat_end_function_usage(&fcusage, true); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_FUNCEXPR_STRICT_FUSAGE) + { + FunctionCallInfo fcinfo = op->d.func.fcinfo_data; + PgStat_FunctionCallUsage fcusage; + bool *argnull = fcinfo->argnull; + int argno; + + /* strict function, so check for NULL args */ + for (argno = 0; argno < op->d.func.nargs; argno++) + { + if (argnull[argno]) + { + *op->resnull = true; + goto strictfail_fusage; + } + } + + pgstat_init_function_usage(fcinfo, &fcusage); + + fcinfo->isnull = false; + *op->resvalue = (op->d.func.fn_addr) (fcinfo); + *op->resnull = fcinfo->isnull; + + pgstat_end_function_usage(&fcusage, true); + + strictfail_fusage: + EEO_NEXT(); + } + + /* + * If any of its clauses is FALSE, an AND's result is FALSE regardless + * of the states of the rest of the clauses, so we can stop evaluating + * and return FALSE immediately. If none are FALSE and one or more is + * NULL, we return NULL; otherwise we return TRUE. This makes sense + * when you interpret NULL as "don't know": perhaps one of the "don't + * knows" would have been FALSE if we'd known its value. Only when + * all the inputs are known to be TRUE can we state confidently that + * the AND's result is TRUE. + */ + EEO_CASE(EEOP_BOOL_AND_STEP_FIRST) + { + *op->d.boolexpr.anynull = false; + + /* + * EEOP_BOOL_AND_STEP_FIRST resets anynull, otherwise it's the + * same as EEOP_BOOL_AND_STEP - so fall through to that. + */ + + /* FALL THROUGH */ + } + + EEO_CASE(EEOP_BOOL_AND_STEP) + { + if (*op->resnull) + { + *op->d.boolexpr.anynull = true; + } + else if (!DatumGetBool(*op->resvalue)) + { + /* result is already set to FALSE, need not change it */ + /* bail out early */ + EEO_JUMP(op->d.boolexpr.jumpdone); + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_BOOL_AND_STEP_LAST) + { + if (*op->resnull) + { + /* result is already set to NULL, need not change it */ + } + else if (!DatumGetBool(*op->resvalue)) + { + /* result is already set to FALSE, need not change it */ + + /* + * No point jumping early to jumpdone - would be same target + * (as this is the last argument to the AND expression), + * except more expensive. + */ + } + else if (*op->d.boolexpr.anynull) + { + *op->resvalue = (Datum) 0; + *op->resnull = true; + } + else + { + /* result is already set to TRUE, need not change it */ + } + + EEO_NEXT(); + } + + /* + * If any of its clauses is TRUE, an OR's result is TRUE regardless of + * the states of the rest of the clauses, so we can stop evaluating + * and return TRUE immediately. If none are TRUE and one or more is + * NULL, we return NULL; otherwise we return FALSE. This makes sense + * when you interpret NULL as "don't know": perhaps one of the "don't + * knows" would have been TRUE if we'd known its value. Only when all + * the inputs are known to be FALSE can we state confidently that the + * OR's result is FALSE. + */ + EEO_CASE(EEOP_BOOL_OR_STEP_FIRST) + { + *op->d.boolexpr.anynull = false; + + /* + * EEOP_BOOL_OR_STEP_FIRST resets anynull, otherwise it's the same + * as EEOP_BOOL_OR_STEP - so fall through to that. + */ + + /* FALL THROUGH */ + } + + EEO_CASE(EEOP_BOOL_OR_STEP) + { + if (*op->resnull) + { + *op->d.boolexpr.anynull = true; + } + else if (DatumGetBool(*op->resvalue)) + { + /* result is already set to TRUE, need not change it */ + /* bail out early */ + EEO_JUMP(op->d.boolexpr.jumpdone); + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_BOOL_OR_STEP_LAST) + { + if (*op->resnull) + { + /* result is already set to NULL, need not change it */ + } + else if (DatumGetBool(*op->resvalue)) + { + /* result is already set to TRUE, need not change it */ + + /* + * No point jumping to jumpdone - would be same target (as + * this is the last argument to the AND expression), except + * more expensive. + */ + } + else if (*op->d.boolexpr.anynull) + { + *op->resvalue = (Datum) 0; + *op->resnull = true; + } + else + { + /* result is already set to FALSE, need not change it */ + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_BOOL_NOT_STEP) + { + /* + * Evaluation of 'not' is simple... if expr is false, then return + * 'true' and vice versa. It's safe to do this even on a + * nominally null value, so we ignore resnull; that means that + * NULL in produces NULL out, which is what we want. + */ + *op->resvalue = BoolGetDatum(!DatumGetBool(*op->resvalue)); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_QUAL) + { + /* simplified version of BOOL_AND_STEP for use by ExecQual() */ + + /* If argument (also result) is false or null ... */ + if (*op->resnull || + !DatumGetBool(*op->resvalue)) + { + /* ... bail out early, returning FALSE */ + *op->resnull = false; + *op->resvalue = BoolGetDatum(false); + EEO_JUMP(op->d.qualexpr.jumpdone); + } + + /* + * Otherwise, leave the TRUE value in place, in case this is the + * last qual. Then, TRUE is the correct answer. + */ + + EEO_NEXT(); + } + + EEO_CASE(EEOP_JUMP) + { + /* Unconditionally jump to target step */ + EEO_JUMP(op->d.jump.jumpdone); + } + + EEO_CASE(EEOP_JUMP_IF_NULL) + { + /* Transfer control if current result is null */ + if (*op->resnull) + EEO_JUMP(op->d.jump.jumpdone); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_JUMP_IF_NOT_NULL) + { + /* Transfer control if current result is non-null */ + if (!*op->resnull) + EEO_JUMP(op->d.jump.jumpdone); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_JUMP_IF_NOT_TRUE) + { + /* Transfer control if current result is null or false */ + if (*op->resnull || !DatumGetBool(*op->resvalue)) + EEO_JUMP(op->d.jump.jumpdone); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_NULLTEST_ISNULL) + { + *op->resvalue = BoolGetDatum(*op->resnull); + *op->resnull = false; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_NULLTEST_ISNOTNULL) + { + *op->resvalue = BoolGetDatum(!*op->resnull); + *op->resnull = false; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_NULLTEST_ROWISNULL) + { + /* out of line implementation: too large */ + ExecEvalRowNull(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_NULLTEST_ROWISNOTNULL) + { + /* out of line implementation: too large */ + ExecEvalRowNotNull(state, op, econtext); + + EEO_NEXT(); + } + + /* BooleanTest implementations for all booltesttypes */ + + EEO_CASE(EEOP_BOOLTEST_IS_TRUE) + { + if (*op->resnull) + { + *op->resvalue = BoolGetDatum(false); + *op->resnull = false; + } + /* else, input value is the correct output as well */ + + EEO_NEXT(); + } + + EEO_CASE(EEOP_BOOLTEST_IS_NOT_TRUE) + { + if (*op->resnull) + { + *op->resvalue = BoolGetDatum(true); + *op->resnull = false; + } + else + *op->resvalue = BoolGetDatum(!DatumGetBool(*op->resvalue)); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_BOOLTEST_IS_FALSE) + { + if (*op->resnull) + { + *op->resvalue = BoolGetDatum(false); + *op->resnull = false; + } + else + *op->resvalue = BoolGetDatum(!DatumGetBool(*op->resvalue)); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_BOOLTEST_IS_NOT_FALSE) + { + if (*op->resnull) + { + *op->resvalue = BoolGetDatum(true); + *op->resnull = false; + } + /* else, input value is the correct output as well */ + + EEO_NEXT(); + } + + EEO_CASE(EEOP_PARAM_EXEC) + { + /* out of line implementation: too large */ + ExecEvalParamExec(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_PARAM_EXTERN) + { + /* out of line implementation: too large */ + ExecEvalParamExtern(state, op, econtext); + EEO_NEXT(); + } + + EEO_CASE(EEOP_CASE_TESTVAL) + { + /* + * Normally upper parts of the expression tree have setup the + * values to be returned here, but some parts of the system + * currently misuse {caseValue,domainValue}_{datum,isNull} to set + * run-time data. So if no values have been set-up, use + * ExprContext's. This isn't pretty, but also not *that* ugly, + * and this is unlikely to be performance sensitive enough to + * worry about an extra branch. + */ + if (op->d.casetest.value) + { + *op->resvalue = *op->d.casetest.value; + *op->resnull = *op->d.casetest.isnull; + } + else + { + *op->resvalue = econtext->caseValue_datum; + *op->resnull = econtext->caseValue_isNull; + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_DOMAIN_TESTVAL) + { + /* + * See EEOP_CASE_TESTVAL comment. + */ + if (op->d.casetest.value) + { + *op->resvalue = *op->d.casetest.value; + *op->resnull = *op->d.casetest.isnull; + } + else + { + *op->resvalue = econtext->domainValue_datum; + *op->resnull = econtext->domainValue_isNull; + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_MAKE_READONLY) + { + /* + * Force a varlena value that might be read multiple times to R/O + */ + if (!*op->d.make_readonly.isnull) + *op->resvalue = + MakeExpandedObjectReadOnlyInternal(*op->d.make_readonly.value); + *op->resnull = *op->d.make_readonly.isnull; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_IOCOERCE) + { + /* + * Evaluate a CoerceViaIO node. This can be quite a hot path, so + * inline as much work as possible. The source value is in our + * result variable. + */ + char *str; + + /* call output function (similar to OutputFunctionCall) */ + if (*op->resnull) + { + /* output functions are not called on nulls */ + str = NULL; + } + else + { + FunctionCallInfo fcinfo_out; + + fcinfo_out = op->d.iocoerce.fcinfo_data_out; + fcinfo_out->arg[0] = *op->resvalue; + fcinfo_out->argnull[0] = false; + + fcinfo_out->isnull = false; + str = DatumGetCString(FunctionCallInvoke(fcinfo_out)); + + /* OutputFunctionCall assumes result isn't null */ + Assert(!fcinfo_out->isnull); + } + + /* call input function (similar to InputFunctionCall) */ + if (!op->d.iocoerce.finfo_in->fn_strict || str != NULL) + { + FunctionCallInfo fcinfo_in; + + fcinfo_in = op->d.iocoerce.fcinfo_data_in; + fcinfo_in->arg[0] = PointerGetDatum(str); + fcinfo_in->argnull[0] = *op->resnull; + /* second and third arguments are already set up */ + + fcinfo_in->isnull = false; + *op->resvalue = FunctionCallInvoke(fcinfo_in); + + /* Should get null result if and only if str is NULL */ + if (str == NULL) + { + Assert(*op->resnull); + Assert(fcinfo_in->isnull); + } + else + { + Assert(!*op->resnull); + Assert(!fcinfo_in->isnull); + } + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_DISTINCT) + { + /* + * IS DISTINCT FROM must evaluate arguments (already done into + * fcinfo->arg/argnull) to determine whether they are NULL; if + * either is NULL then the result is determined. If neither is + * NULL, then proceed to evaluate the comparison function, which + * is just the type's standard equality operator. We need not + * care whether that function is strict. Because the handling of + * nulls is different, we can't just reuse EEOP_FUNCEXPR. + */ + FunctionCallInfo fcinfo = op->d.func.fcinfo_data; + + /* check function arguments for NULLness */ + if (fcinfo->argnull[0] && fcinfo->argnull[1]) + { + /* Both NULL? Then is not distinct... */ + *op->resvalue = BoolGetDatum(false); + *op->resnull = false; + } + else if (fcinfo->argnull[0] || fcinfo->argnull[1]) + { + /* Only one is NULL? Then is distinct... */ + *op->resvalue = BoolGetDatum(true); + *op->resnull = false; + } + else + { + /* Neither null, so apply the equality function */ + Datum eqresult; + + fcinfo->isnull = false; + eqresult = (op->d.func.fn_addr) (fcinfo); + /* Must invert result of "="; safe to do even if null */ + *op->resvalue = BoolGetDatum(!DatumGetBool(eqresult)); + *op->resnull = fcinfo->isnull; + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_NULLIF) + { + /* + * The arguments are already evaluated into fcinfo->arg/argnull. + */ + FunctionCallInfo fcinfo = op->d.func.fcinfo_data; + + /* if either argument is NULL they can't be equal */ + if (!fcinfo->argnull[0] && !fcinfo->argnull[1]) + { + Datum result; + + fcinfo->isnull = false; + result = (op->d.func.fn_addr) (fcinfo); + + /* if the arguments are equal return null */ + if (!fcinfo->isnull && DatumGetBool(result)) + { + *op->resvalue = (Datum) 0; + *op->resnull = true; + + EEO_NEXT(); + } + } + + /* Arguments aren't equal, so return the first one */ + *op->resvalue = fcinfo->arg[0]; + *op->resnull = fcinfo->argnull[0]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_SQLVALUEFUNCTION) + { + /* + * Doesn't seem worthwhile to have an inline implementation + * efficiency-wise. + */ + ExecEvalSQLValueFunction(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_CURRENTOFEXPR) + { + /* error invocation uses space, and shouldn't ever occur */ + ExecEvalCurrentOfExpr(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_NEXTVALUEEXPR) + { + switch (op->d.nextvalueexpr.seqtypid) + { + case INT2OID: + *op->resvalue = Int16GetDatum((int16) nextval_internal(op->d.nextvalueexpr.seqid, false)); + break; + case INT4OID: + *op->resvalue = Int32GetDatum((int32) nextval_internal(op->d.nextvalueexpr.seqid, false)); + break; + case INT8OID: + *op->resvalue = Int64GetDatum((int64) nextval_internal(op->d.nextvalueexpr.seqid, false)); + break; + default: + elog(ERROR, "unsupported sequence type %u", op->d.nextvalueexpr.seqtypid); + } + *op->resnull = false; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ARRAYEXPR) + { + /* too complex for an inline implementation */ + ExecEvalArrayExpr(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ARRAYCOERCE) + { + /* too complex for an inline implementation */ + ExecEvalArrayCoerce(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ROW) + { + /* too complex for an inline implementation */ + ExecEvalRow(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ROWCOMPARE_STEP) + { + FunctionCallInfo fcinfo = op->d.rowcompare_step.fcinfo_data; + + /* force NULL result if strict fn and NULL input */ + if (op->d.rowcompare_step.finfo->fn_strict && + (fcinfo->argnull[0] || fcinfo->argnull[1])) + { + *op->resnull = true; + EEO_JUMP(op->d.rowcompare_step.jumpnull); + } + + /* Apply comparison function */ + fcinfo->isnull = false; + *op->resvalue = (op->d.rowcompare_step.fn_addr) (fcinfo); + + /* force NULL result if NULL function result */ + if (fcinfo->isnull) + { + *op->resnull = true; + EEO_JUMP(op->d.rowcompare_step.jumpnull); + } + *op->resnull = false; + + /* If unequal, no need to compare remaining columns */ + if (DatumGetInt32(*op->resvalue) != 0) + { + EEO_JUMP(op->d.rowcompare_step.jumpdone); + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ROWCOMPARE_FINAL) + { + int32 cmpresult = DatumGetInt32(*op->resvalue); + RowCompareType rctype = op->d.rowcompare_final.rctype; + + *op->resnull = false; + switch (rctype) + { + /* EQ and NE cases aren't allowed here */ + case ROWCOMPARE_LT: + *op->resvalue = BoolGetDatum(cmpresult < 0); + break; + case ROWCOMPARE_LE: + *op->resvalue = BoolGetDatum(cmpresult <= 0); + break; + case ROWCOMPARE_GE: + *op->resvalue = BoolGetDatum(cmpresult >= 0); + break; + case ROWCOMPARE_GT: + *op->resvalue = BoolGetDatum(cmpresult > 0); + break; + default: + Assert(false); + break; + } + + EEO_NEXT(); + } + + EEO_CASE(EEOP_MINMAX) + { + /* too complex for an inline implementation */ + ExecEvalMinMax(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_FIELDSELECT) + { + /* too complex for an inline implementation */ + ExecEvalFieldSelect(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_FIELDSTORE_DEFORM) + { + /* too complex for an inline implementation */ + ExecEvalFieldStoreDeForm(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_FIELDSTORE_FORM) + { + /* too complex for an inline implementation */ + ExecEvalFieldStoreForm(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ARRAYREF_SUBSCRIPT) + { + /* Process an array subscript */ + + /* too complex for an inline implementation */ + if (ExecEvalArrayRefSubscript(state, op)) + { + EEO_NEXT(); + } + else + { + /* Subscript is null, short-circuit ArrayRef to NULL */ + EEO_JUMP(op->d.arrayref_subscript.jumpdone); + } + } + + EEO_CASE(EEOP_ARRAYREF_OLD) + { + /* + * Fetch the old value in an arrayref assignment, in case it's + * referenced (via a CaseTestExpr) inside the assignment + * expression. + */ + + /* too complex for an inline implementation */ + ExecEvalArrayRefOld(state, op); + + EEO_NEXT(); + } + + /* + * Perform ArrayRef assignment + */ + EEO_CASE(EEOP_ARRAYREF_ASSIGN) + { + /* too complex for an inline implementation */ + ExecEvalArrayRefAssign(state, op); + + EEO_NEXT(); + } + + /* + * Fetch subset of an array. + */ + EEO_CASE(EEOP_ARRAYREF_FETCH) + { + /* too complex for an inline implementation */ + ExecEvalArrayRefFetch(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_CONVERT_ROWTYPE) + { + /* too complex for an inline implementation */ + ExecEvalConvertRowtype(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_SCALARARRAYOP) + { + /* too complex for an inline implementation */ + ExecEvalScalarArrayOp(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_DOMAIN_NOTNULL) + { + /* too complex for an inline implementation */ + ExecEvalConstraintNotNull(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_DOMAIN_CHECK) + { + /* too complex for an inline implementation */ + ExecEvalConstraintCheck(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_XMLEXPR) + { + /* too complex for an inline implementation */ + ExecEvalXmlExpr(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_AGGREF) + { + /* + * Returns a Datum whose value is the precomputed aggregate value + * found in the given expression context. + */ + AggrefExprState *aggref = op->d.aggref.astate; + + Assert(econtext->ecxt_aggvalues != NULL); + + *op->resvalue = econtext->ecxt_aggvalues[aggref->aggno]; + *op->resnull = econtext->ecxt_aggnulls[aggref->aggno]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_GROUPING_FUNC) + { + /* too complex/uncommon for an inline implementation */ + ExecEvalGroupingFunc(state, op); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_WINDOW_FUNC) + { + /* + * Like Aggref, just return a precomputed value from the econtext. + */ + WindowFuncExprState *wfunc = op->d.window_func.wfstate; + + Assert(econtext->ecxt_aggvalues != NULL); + + *op->resvalue = econtext->ecxt_aggvalues[wfunc->wfuncno]; + *op->resnull = econtext->ecxt_aggnulls[wfunc->wfuncno]; + + EEO_NEXT(); + } + + EEO_CASE(EEOP_SUBPLAN) + { + /* too complex for an inline implementation */ + ExecEvalSubPlan(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_ALTERNATIVE_SUBPLAN) + { + /* too complex for an inline implementation */ + ExecEvalAlternativeSubPlan(state, op, econtext); + + EEO_NEXT(); + } + + EEO_CASE(EEOP_LAST) + { + /* unreachable */ + Assert(false); + goto out; + } + } + +out: + *isnull = state->resnull; + return state->resvalue; +} + +/* + * Check whether a user attribute in a slot can be referenced by a Var + * expression. This should succeed unless there have been schema changes + * since the expression tree has been created. + */ +static void +CheckVarSlotCompatibility(TupleTableSlot *slot, int attnum, Oid vartype) +{ + /* + * What we have to check for here is the possibility of an attribute + * having been dropped or changed in type since the plan tree was created. + * Ideally the plan will get invalidated and not re-used, but just in + * case, we keep these defenses. Fortunately it's sufficient to check + * once on the first time through. + * + * Note: ideally we'd check typmod as well as typid, but that seems + * impractical at the moment: in many cases the tupdesc will have been + * generated by ExecTypeFromTL(), and that can't guarantee to generate an + * accurate typmod in all cases, because some expression node types don't + * carry typmod. Fortunately, for precisely that reason, there should be + * no places with a critical dependency on the typmod of a value. + * + * System attributes don't require checking since their types never + * change. + */ + if (attnum > 0) + { + TupleDesc slot_tupdesc = slot->tts_tupleDescriptor; + Form_pg_attribute attr; + + if (attnum > slot_tupdesc->natts) /* should never happen */ + elog(ERROR, "attribute number %d exceeds number of columns %d", + attnum, slot_tupdesc->natts); + + attr = slot_tupdesc->attrs[attnum - 1]; + + if (attr->attisdropped) + ereport(ERROR, + (errcode(ERRCODE_UNDEFINED_COLUMN), + errmsg("attribute %d of type %s has been dropped", + attnum, format_type_be(slot_tupdesc->tdtypeid)))); + + if (vartype != attr->atttypid) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("attribute %d of type %s has wrong type", + attnum, format_type_be(slot_tupdesc->tdtypeid)), + errdetail("Table has type %s, but query expects %s.", + format_type_be(attr->atttypid), + format_type_be(vartype)))); + } +} + +/* + * get_cached_rowtype: utility function to lookup a rowtype tupdesc + * + * type_id, typmod: identity of the rowtype + * cache_field: where to cache the TupleDesc pointer in expression state node + * (field must be initialized to NULL) + * econtext: expression context we are executing in + * + * NOTE: because the shutdown callback will be called during plan rescan, + * must be prepared to re-do this during any node execution; cannot call + * just once during expression initialization. + */ +static TupleDesc +get_cached_rowtype(Oid type_id, int32 typmod, + TupleDesc *cache_field, ExprContext *econtext) +{ + TupleDesc tupDesc = *cache_field; + + /* Do lookup if no cached value or if requested type changed */ + if (tupDesc == NULL || + type_id != tupDesc->tdtypeid || + typmod != tupDesc->tdtypmod) + { + tupDesc = lookup_rowtype_tupdesc(type_id, typmod); + + if (*cache_field) + { + /* Release old tupdesc; but callback is already registered */ + ReleaseTupleDesc(*cache_field); + } + else + { + /* Need to register shutdown callback to release tupdesc */ + RegisterExprContextCallback(econtext, + ShutdownTupleDescRef, + PointerGetDatum(cache_field)); + } + *cache_field = tupDesc; + } + return tupDesc; +} + +/* + * Callback function to release a tupdesc refcount at econtext shutdown + */ +static void +ShutdownTupleDescRef(Datum arg) +{ + TupleDesc *cache_field = (TupleDesc *) DatumGetPointer(arg); + + if (*cache_field) + ReleaseTupleDesc(*cache_field); + *cache_field = NULL; +} + +/* + * Fast-path functions, for very simple expressions + */ + +/* Simple reference to inner Var, first time through */ +static Datum +ExecJustInnerVarFirst(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.var.attnum + 1; + TupleTableSlot *slot = econtext->ecxt_innertuple; + + /* See ExecInterpExpr()'s comments for EEOP_INNER_VAR_FIRST */ + + CheckVarSlotCompatibility(slot, attnum, op->d.var.vartype); + op->opcode = EEOP_INNER_VAR; /* just for cleanliness */ + state->evalfunc = ExecJustInnerVar; + + /* + * Since we use slot_getattr(), we don't need to implement the FETCHSOME + * step explicitly, and we also needn't Assert that the attnum is in range + * --- slot_getattr() will take care of any problems. + */ + return slot_getattr(slot, attnum, isnull); +} + +/* Simple reference to inner Var */ +static Datum +ExecJustInnerVar(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.var.attnum + 1; + TupleTableSlot *slot = econtext->ecxt_innertuple; + + /* See comments in ExecJustInnerVarFirst */ + return slot_getattr(slot, attnum, isnull); +} + +/* Simple reference to outer Var, first time through */ +static Datum +ExecJustOuterVarFirst(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.var.attnum + 1; + TupleTableSlot *slot = econtext->ecxt_outertuple; + + CheckVarSlotCompatibility(slot, attnum, op->d.var.vartype); + op->opcode = EEOP_OUTER_VAR; /* just for cleanliness */ + state->evalfunc = ExecJustOuterVar; + + /* See comments in ExecJustInnerVarFirst */ + return slot_getattr(slot, attnum, isnull); +} + +/* Simple reference to outer Var */ +static Datum +ExecJustOuterVar(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.var.attnum + 1; + TupleTableSlot *slot = econtext->ecxt_outertuple; + + /* See comments in ExecJustInnerVarFirst */ + return slot_getattr(slot, attnum, isnull); +} + +/* Simple reference to scan Var, first time through */ +static Datum +ExecJustScanVarFirst(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.var.attnum + 1; + TupleTableSlot *slot = econtext->ecxt_scantuple; + + CheckVarSlotCompatibility(slot, attnum, op->d.var.vartype); + op->opcode = EEOP_SCAN_VAR; /* just for cleanliness */ + state->evalfunc = ExecJustScanVar; + + /* See comments in ExecJustInnerVarFirst */ + return slot_getattr(slot, attnum, isnull); +} + +/* Simple reference to scan Var */ +static Datum +ExecJustScanVar(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.var.attnum + 1; + TupleTableSlot *slot = econtext->ecxt_scantuple; + + /* See comments in ExecJustInnerVarFirst */ + return slot_getattr(slot, attnum, isnull); +} + +/* Simple Const expression */ +static Datum +ExecJustConst(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[0]; + + *isnull = op->d.constval.isnull; + return op->d.constval.value; +} + +/* Evaluate inner Var and assign to appropriate column of result tuple */ +static Datum +ExecJustAssignInnerVar(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.assign_var.attnum + 1; + int resultnum = op->d.assign_var.resultnum; + TupleTableSlot *inslot = econtext->ecxt_innertuple; + TupleTableSlot *outslot = state->resultslot; + + /* + * We do not need CheckVarSlotCompatibility here; that was taken care of + * at compilation time. + * + * Since we use slot_getattr(), we don't need to implement the FETCHSOME + * step explicitly, and we also needn't Assert that the attnum is in range + * --- slot_getattr() will take care of any problems. + */ + outslot->tts_values[resultnum] = + slot_getattr(inslot, attnum, &outslot->tts_isnull[resultnum]); + return 0; +} + +/* Evaluate outer Var and assign to appropriate column of result tuple */ +static Datum +ExecJustAssignOuterVar(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.assign_var.attnum + 1; + int resultnum = op->d.assign_var.resultnum; + TupleTableSlot *inslot = econtext->ecxt_outertuple; + TupleTableSlot *outslot = state->resultslot; + + /* See comments in ExecJustAssignInnerVar */ + outslot->tts_values[resultnum] = + slot_getattr(inslot, attnum, &outslot->tts_isnull[resultnum]); + return 0; +} + +/* Evaluate scan Var and assign to appropriate column of result tuple */ +static Datum +ExecJustAssignScanVar(ExprState *state, ExprContext *econtext, bool *isnull) +{ + ExprEvalStep *op = &state->steps[1]; + int attnum = op->d.assign_var.attnum + 1; + int resultnum = op->d.assign_var.resultnum; + TupleTableSlot *inslot = econtext->ecxt_scantuple; + TupleTableSlot *outslot = state->resultslot; + + /* See comments in ExecJustAssignInnerVar */ + outslot->tts_values[resultnum] = + slot_getattr(inslot, attnum, &outslot->tts_isnull[resultnum]); + return 0; +} + + +/* + * Do one-time initialization of interpretation machinery. + */ +static void +ExecInitInterpreter(void) +{ +#if defined(EEO_USE_COMPUTED_GOTO) + /* Set up externally-visible pointer to dispatch table */ + if (dispatch_table == NULL) + dispatch_table = (const void **) + DatumGetPointer(ExecInterpExpr(NULL, NULL, NULL)); +#endif +} + +/* + * Function to return the opcode of an expression step. + * + * When direct-threading is in use, ExprState->opcode isn't easily + * decipherable. This function returns the appropriate enum member. + * + * This currently is only supposed to be used in paths that aren't critical + * performance-wise. If that changes, we could add an inverse dispatch_table + * that's sorted on the address, so a binary search can be performed. + */ +ExprEvalOp +ExecEvalStepOp(ExprState *state, ExprEvalStep *op) +{ +#if defined(EEO_USE_COMPUTED_GOTO) + if (state->flags & EEO_FLAG_DIRECT_THREADED) + { + int i; + + for (i = 0; i < EEOP_LAST; i++) + { + if ((void *) op->opcode == dispatch_table[i]) + { + return (ExprEvalOp) i; + } + } + elog(ERROR, "unknown opcode"); + } +#endif + return (ExprEvalOp) op->opcode; +} + + +/* + * Out-of-line helper functions for complex instructions. + */ + +/* + * Evaluate a PARAM_EXEC parameter. + * + * PARAM_EXEC params (internal executor parameters) are stored in the + * ecxt_param_exec_vals array, and can be accessed by array index. + */ +void +ExecEvalParamExec(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + ParamExecData *prm; + + prm = &(econtext->ecxt_param_exec_vals[op->d.param.paramid]); + if (unlikely(prm->execPlan != NULL)) + { + /* Parameter not evaluated yet, so go do it */ + ExecSetParamPlan(prm->execPlan, econtext); + /* ExecSetParamPlan should have processed this param... */ + Assert(prm->execPlan == NULL); + } + *op->resvalue = prm->value; + *op->resnull = prm->isnull; +} + +/* + * Evaluate a PARAM_EXTERN parameter. + * + * PARAM_EXTERN parameters must be sought in ecxt_param_list_info. + */ +void +ExecEvalParamExtern(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + ParamListInfo paramInfo = econtext->ecxt_param_list_info; + int paramId = op->d.param.paramid; + + if (likely(paramInfo && + paramId > 0 && paramId <= paramInfo->numParams)) + { + ParamExternData *prm = ¶mInfo->params[paramId - 1]; + + /* give hook a chance in case parameter is dynamic */ + if (!OidIsValid(prm->ptype) && paramInfo->paramFetch != NULL) + (*paramInfo->paramFetch) (paramInfo, paramId); + + if (likely(OidIsValid(prm->ptype))) + { + /* safety check in case hook did something unexpected */ + if (unlikely(prm->ptype != op->d.param.paramtype)) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("type of parameter %d (%s) does not match that when preparing the plan (%s)", + paramId, + format_type_be(prm->ptype), + format_type_be(op->d.param.paramtype)))); + *op->resvalue = prm->value; + *op->resnull = prm->isnull; + return; + } + } + + ereport(ERROR, + (errcode(ERRCODE_UNDEFINED_OBJECT), + errmsg("no value found for parameter %d", paramId))); +} + +/* + * Evaluate a SQLValueFunction expression. + */ +void +ExecEvalSQLValueFunction(ExprState *state, ExprEvalStep *op) +{ + SQLValueFunction *svf = op->d.sqlvaluefunction.svf; + FunctionCallInfoData fcinfo; + + *op->resnull = false; + + /* + * Note: current_schema() can return NULL. current_user() etc currently + * cannot, but might as well code those cases the same way for safety. + */ + switch (svf->op) + { + case SVFOP_CURRENT_DATE: + *op->resvalue = DateADTGetDatum(GetSQLCurrentDate()); + break; + case SVFOP_CURRENT_TIME: + case SVFOP_CURRENT_TIME_N: + *op->resvalue = TimeTzADTPGetDatum(GetSQLCurrentTime(svf->typmod)); + break; + case SVFOP_CURRENT_TIMESTAMP: + case SVFOP_CURRENT_TIMESTAMP_N: + *op->resvalue = TimestampTzGetDatum(GetSQLCurrentTimestamp(svf->typmod)); + break; + case SVFOP_LOCALTIME: + case SVFOP_LOCALTIME_N: + *op->resvalue = TimeADTGetDatum(GetSQLLocalTime(svf->typmod)); + break; + case SVFOP_LOCALTIMESTAMP: + case SVFOP_LOCALTIMESTAMP_N: + *op->resvalue = TimestampGetDatum(GetSQLLocalTimestamp(svf->typmod)); + break; + case SVFOP_CURRENT_ROLE: + case SVFOP_CURRENT_USER: + case SVFOP_USER: + InitFunctionCallInfoData(fcinfo, NULL, 0, InvalidOid, NULL, NULL); + *op->resvalue = current_user(&fcinfo); + *op->resnull = fcinfo.isnull; + break; + case SVFOP_SESSION_USER: + InitFunctionCallInfoData(fcinfo, NULL, 0, InvalidOid, NULL, NULL); + *op->resvalue = session_user(&fcinfo); + *op->resnull = fcinfo.isnull; + break; + case SVFOP_CURRENT_CATALOG: + InitFunctionCallInfoData(fcinfo, NULL, 0, InvalidOid, NULL, NULL); + *op->resvalue = current_database(&fcinfo); + *op->resnull = fcinfo.isnull; + break; + case SVFOP_CURRENT_SCHEMA: + InitFunctionCallInfoData(fcinfo, NULL, 0, InvalidOid, NULL, NULL); + *op->resvalue = current_schema(&fcinfo); + *op->resnull = fcinfo.isnull; + break; + } +} + +/* + * Raise error if a CURRENT OF expression is evaluated. + * + * The planner should convert CURRENT OF into a TidScan qualification, or some + * other special handling in a ForeignScan node. So we have to be able to do + * ExecInitExpr on a CurrentOfExpr, but we shouldn't ever actually execute it. + * If we get here, we suppose we must be dealing with CURRENT OF on a foreign + * table whose FDW doesn't handle it, and complain accordingly. + */ +void +ExecEvalCurrentOfExpr(ExprState *state, ExprEvalStep *op) +{ + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("WHERE CURRENT OF is not supported for this table type"))); +} + +/* + * Evaluate NullTest / IS NULL for rows. + */ +void +ExecEvalRowNull(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + ExecEvalRowNullInt(state, op, econtext, true); +} + +/* + * Evaluate NullTest / IS NOT NULL for rows. + */ +void +ExecEvalRowNotNull(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + ExecEvalRowNullInt(state, op, econtext, false); +} + +/* Common code for IS [NOT] NULL on a row value */ +static void +ExecEvalRowNullInt(ExprState *state, ExprEvalStep *op, + ExprContext *econtext, bool checkisnull) +{ + Datum value = *op->resvalue; + bool isnull = *op->resnull; + HeapTupleHeader tuple; + Oid tupType; + int32 tupTypmod; + TupleDesc tupDesc; + HeapTupleData tmptup; + int att; + + *op->resnull = false; + + /* NULL row variables are treated just as NULL scalar columns */ + if (isnull) + { + *op->resvalue = BoolGetDatum(checkisnull); + return; + } + + /* + * The SQL standard defines IS [NOT] NULL for a non-null rowtype argument + * as: + * + * "R IS NULL" is true if every field is the null value. + * + * "R IS NOT NULL" is true if no field is the null value. + * + * This definition is (apparently intentionally) not recursive; so our + * tests on the fields are primitive attisnull tests, not recursive checks + * to see if they are all-nulls or no-nulls rowtypes. + * + * The standard does not consider the possibility of zero-field rows, but + * here we consider them to vacuously satisfy both predicates. + */ + + tuple = DatumGetHeapTupleHeader(value); + + tupType = HeapTupleHeaderGetTypeId(tuple); + tupTypmod = HeapTupleHeaderGetTypMod(tuple); + + /* Lookup tupdesc if first time through or if type changes */ + tupDesc = get_cached_rowtype(tupType, tupTypmod, + &op->d.nulltest_row.argdesc, + econtext); + + /* + * heap_attisnull needs a HeapTuple not a bare HeapTupleHeader. + */ + tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); + tmptup.t_data = tuple; + + for (att = 1; att <= tupDesc->natts; att++) + { + /* ignore dropped columns */ + if (tupDesc->attrs[att - 1]->attisdropped) + continue; + if (heap_attisnull(&tmptup, att)) + { + /* null field disproves IS NOT NULL */ + if (!checkisnull) + { + *op->resvalue = BoolGetDatum(false); + return; + } + } + else + { + /* non-null field disproves IS NULL */ + if (checkisnull) + { + *op->resvalue = BoolGetDatum(false); + return; + } + } + } + + *op->resvalue = BoolGetDatum(true); +} + +/* + * Evaluate an ARRAY[] expression. + * + * The individual array elements (or subarrays) have already been evaluated + * into op->d.arrayexpr.elemvalues[]/elemnulls[]. + */ +void +ExecEvalArrayExpr(ExprState *state, ExprEvalStep *op) +{ + ArrayType *result; + Oid element_type = op->d.arrayexpr.elemtype; + int nelems = op->d.arrayexpr.nelems; + int ndims = 0; + int dims[MAXDIM]; + int lbs[MAXDIM]; + + /* Set non-null as default */ + *op->resnull = false; + + if (!op->d.arrayexpr.multidims) + { + /* Elements are presumably of scalar type */ + Datum *dvalues = op->d.arrayexpr.elemvalues; + bool *dnulls = op->d.arrayexpr.elemnulls; + + /* Shouldn't happen here, but if length is 0, return empty array */ + if (nelems == 0) + { + *op->resvalue = + PointerGetDatum(construct_empty_array(element_type)); + return; + } + + /* setup for 1-D array of the given length */ + ndims = 1; + dims[0] = nelems; + lbs[0] = 1; + + result = construct_md_array(dvalues, dnulls, ndims, dims, lbs, + element_type, + op->d.arrayexpr.elemlength, + op->d.arrayexpr.elembyval, + op->d.arrayexpr.elemalign); + } + else + { + /* Must be nested array expressions */ + int nbytes = 0; + int nitems = 0; + int outer_nelems = 0; + int elem_ndims = 0; + int *elem_dims = NULL; + int *elem_lbs = NULL; + bool firstone = true; + bool havenulls = false; + bool haveempty = false; + char **subdata; + bits8 **subbitmaps; + int *subbytes; + int *subnitems; + int32 dataoffset; + char *dat; + int iitem; + int elemoff; + int i; + + subdata = (char **) palloc(nelems * sizeof(char *)); + subbitmaps = (bits8 **) palloc(nelems * sizeof(bits8 *)); + subbytes = (int *) palloc(nelems * sizeof(int)); + subnitems = (int *) palloc(nelems * sizeof(int)); + + /* loop through and get data area from each element */ + for (elemoff = 0; elemoff < nelems; elemoff++) + { + Datum arraydatum; + bool eisnull; + ArrayType *array; + int this_ndims; + + arraydatum = op->d.arrayexpr.elemvalues[elemoff]; + eisnull = op->d.arrayexpr.elemnulls[elemoff]; + + /* temporarily ignore null subarrays */ + if (eisnull) + { + haveempty = true; + continue; + } + + array = DatumGetArrayTypeP(arraydatum); + + /* run-time double-check on element type */ + if (element_type != ARR_ELEMTYPE(array)) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("cannot merge incompatible arrays"), + errdetail("Array with element type %s cannot be " + "included in ARRAY construct with element type %s.", + format_type_be(ARR_ELEMTYPE(array)), + format_type_be(element_type)))); + + this_ndims = ARR_NDIM(array); + /* temporarily ignore zero-dimensional subarrays */ + if (this_ndims <= 0) + { + haveempty = true; + continue; + } + + if (firstone) + { + /* Get sub-array details from first member */ + elem_ndims = this_ndims; + ndims = elem_ndims + 1; + if (ndims <= 0 || ndims > MAXDIM) + ereport(ERROR, + (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED), + errmsg("number of array dimensions (%d) exceeds " \ + "the maximum allowed (%d)", ndims, MAXDIM))); + + elem_dims = (int *) palloc(elem_ndims * sizeof(int)); + memcpy(elem_dims, ARR_DIMS(array), elem_ndims * sizeof(int)); + elem_lbs = (int *) palloc(elem_ndims * sizeof(int)); + memcpy(elem_lbs, ARR_LBOUND(array), elem_ndims * sizeof(int)); + + firstone = false; + } + else + { + /* Check other sub-arrays are compatible */ + if (elem_ndims != this_ndims || + memcmp(elem_dims, ARR_DIMS(array), + elem_ndims * sizeof(int)) != 0 || + memcmp(elem_lbs, ARR_LBOUND(array), + elem_ndims * sizeof(int)) != 0) + ereport(ERROR, + (errcode(ERRCODE_ARRAY_SUBSCRIPT_ERROR), + errmsg("multidimensional arrays must have array " + "expressions with matching dimensions"))); + } + + subdata[outer_nelems] = ARR_DATA_PTR(array); + subbitmaps[outer_nelems] = ARR_NULLBITMAP(array); + subbytes[outer_nelems] = ARR_SIZE(array) - ARR_DATA_OFFSET(array); + nbytes += subbytes[outer_nelems]; + subnitems[outer_nelems] = ArrayGetNItems(this_ndims, + ARR_DIMS(array)); + nitems += subnitems[outer_nelems]; + havenulls |= ARR_HASNULL(array); + outer_nelems++; + } + + /* + * If all items were null or empty arrays, return an empty array; + * otherwise, if some were and some weren't, raise error. (Note: we + * must special-case this somehow to avoid trying to generate a 1-D + * array formed from empty arrays. It's not ideal...) + */ + if (haveempty) + { + if (ndims == 0) /* didn't find any nonempty array */ + { + *op->resvalue = PointerGetDatum(construct_empty_array(element_type)); + return; + } + ereport(ERROR, + (errcode(ERRCODE_ARRAY_SUBSCRIPT_ERROR), + errmsg("multidimensional arrays must have array " + "expressions with matching dimensions"))); + } + + /* setup for multi-D array */ + dims[0] = outer_nelems; + lbs[0] = 1; + for (i = 1; i < ndims; i++) + { + dims[i] = elem_dims[i - 1]; + lbs[i] = elem_lbs[i - 1]; + } + + if (havenulls) + { + dataoffset = ARR_OVERHEAD_WITHNULLS(ndims, nitems); + nbytes += dataoffset; + } + else + { + dataoffset = 0; /* marker for no null bitmap */ + nbytes += ARR_OVERHEAD_NONULLS(ndims); + } + + result = (ArrayType *) palloc(nbytes); + SET_VARSIZE(result, nbytes); + result->ndim = ndims; + result->dataoffset = dataoffset; + result->elemtype = element_type; + memcpy(ARR_DIMS(result), dims, ndims * sizeof(int)); + memcpy(ARR_LBOUND(result), lbs, ndims * sizeof(int)); + + dat = ARR_DATA_PTR(result); + iitem = 0; + for (i = 0; i < outer_nelems; i++) + { + memcpy(dat, subdata[i], subbytes[i]); + dat += subbytes[i]; + if (havenulls) + array_bitmap_copy(ARR_NULLBITMAP(result), iitem, + subbitmaps[i], 0, + subnitems[i]); + iitem += subnitems[i]; + } + } + + *op->resvalue = PointerGetDatum(result); +} + +/* + * Evaluate an ArrayCoerceExpr expression. + * + * Source array is in step's result variable. + */ +void +ExecEvalArrayCoerce(ExprState *state, ExprEvalStep *op) +{ + ArrayCoerceExpr *acoerce = op->d.arraycoerce.coerceexpr; + Datum arraydatum; + FunctionCallInfoData locfcinfo; + + /* NULL array -> NULL result */ + if (*op->resnull) + return; + + arraydatum = *op->resvalue; + + /* + * If it's binary-compatible, modify the element type in the array header, + * but otherwise leave the array as we received it. + */ + if (!OidIsValid(acoerce->elemfuncid)) + { + /* Detoast input array if necessary, and copy in any case */ + ArrayType *array = DatumGetArrayTypePCopy(arraydatum); + + ARR_ELEMTYPE(array) = op->d.arraycoerce.resultelemtype; + *op->resvalue = PointerGetDatum(array); + return; + } + + /* + * Use array_map to apply the function to each array element. + * + * We pass on the desttypmod and isExplicit flags whether or not the + * function wants them. + * + * Note: coercion functions are assumed to not use collation. + */ + InitFunctionCallInfoData(locfcinfo, op->d.arraycoerce.elemfunc, 3, + InvalidOid, NULL, NULL); + locfcinfo.arg[0] = arraydatum; + locfcinfo.arg[1] = Int32GetDatum(acoerce->resulttypmod); + locfcinfo.arg[2] = BoolGetDatum(acoerce->isExplicit); + locfcinfo.argnull[0] = false; + locfcinfo.argnull[1] = false; + locfcinfo.argnull[2] = false; + + *op->resvalue = array_map(&locfcinfo, op->d.arraycoerce.resultelemtype, + op->d.arraycoerce.amstate); +} + +/* + * Evaluate a ROW() expression. + * + * The individual columns have already been evaluated into + * op->d.row.elemvalues[]/elemnulls[]. + */ +void +ExecEvalRow(ExprState *state, ExprEvalStep *op) +{ + HeapTuple tuple; + + /* build tuple from evaluated field values */ + tuple = heap_form_tuple(op->d.row.tupdesc, + op->d.row.elemvalues, + op->d.row.elemnulls); + + *op->resvalue = HeapTupleGetDatum(tuple); + *op->resnull = false; +} + +/* + * Evaluate GREATEST() or LEAST() expression (note this is *not* MIN()/MAX()). + * + * All of the to-be-compared expressions have already been evaluated into + * op->d.minmax.values[]/nulls[]. + */ +void +ExecEvalMinMax(ExprState *state, ExprEvalStep *op) +{ + Datum *values = op->d.minmax.values; + bool *nulls = op->d.minmax.nulls; + FunctionCallInfo fcinfo = op->d.minmax.fcinfo_data; + MinMaxOp operator = op->d.minmax.op; + int off; + + /* set at initialization */ + Assert(fcinfo->argnull[0] == false); + Assert(fcinfo->argnull[1] == false); + + /* default to null result */ + *op->resnull = true; + + for (off = 0; off < op->d.minmax.nelems; off++) + { + /* ignore NULL inputs */ + if (nulls[off]) + continue; + + if (*op->resnull) + { + /* first nonnull input, adopt value */ + *op->resvalue = values[off]; + *op->resnull = false; + } + else + { + int cmpresult; + + /* apply comparison function */ + fcinfo->arg[0] = *op->resvalue; + fcinfo->arg[1] = values[off]; + + fcinfo->isnull = false; + cmpresult = DatumGetInt32(FunctionCallInvoke(fcinfo)); + if (fcinfo->isnull) /* probably should not happen */ + continue; + + if (cmpresult > 0 && operator == IS_LEAST) + *op->resvalue = values[off]; + else if (cmpresult < 0 && operator == IS_GREATEST) + *op->resvalue = values[off]; + } + } +} + +/* + * Evaluate a FieldSelect node. + * + * Source record is in step's result variable. + */ +void +ExecEvalFieldSelect(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + AttrNumber fieldnum = op->d.fieldselect.fieldnum; + Datum tupDatum; + HeapTupleHeader tuple; + Oid tupType; + int32 tupTypmod; + TupleDesc tupDesc; + Form_pg_attribute attr; + HeapTupleData tmptup; + + /* NULL record -> NULL result */ + if (*op->resnull) + return; + + /* Get the composite datum and extract its type fields */ + tupDatum = *op->resvalue; + tuple = DatumGetHeapTupleHeader(tupDatum); + + tupType = HeapTupleHeaderGetTypeId(tuple); + tupTypmod = HeapTupleHeaderGetTypMod(tuple); + + /* Lookup tupdesc if first time through or if type changes */ + tupDesc = get_cached_rowtype(tupType, tupTypmod, + &op->d.fieldselect.argdesc, + econtext); + + /* + * Find field's attr record. Note we don't support system columns here: a + * datum tuple doesn't have valid values for most of the interesting + * system columns anyway. + */ + if (fieldnum <= 0) /* should never happen */ + elog(ERROR, "unsupported reference to system column %d in FieldSelect", + fieldnum); + if (fieldnum > tupDesc->natts) /* should never happen */ + elog(ERROR, "attribute number %d exceeds number of columns %d", + fieldnum, tupDesc->natts); + attr = tupDesc->attrs[fieldnum - 1]; + + /* Check for dropped column, and force a NULL result if so */ + if (attr->attisdropped) + { + *op->resnull = true; + return; + } + + /* Check for type mismatch --- possible after ALTER COLUMN TYPE? */ + /* As in CheckVarSlotCompatibility, we should but can't check typmod */ + if (op->d.fieldselect.resulttype != attr->atttypid) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("attribute %d has wrong type", fieldnum), + errdetail("Table has type %s, but query expects %s.", + format_type_be(attr->atttypid), + format_type_be(op->d.fieldselect.resulttype)))); + + /* heap_getattr needs a HeapTuple not a bare HeapTupleHeader */ + tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); + tmptup.t_data = tuple; + + /* extract the field */ + *op->resvalue = heap_getattr(&tmptup, + fieldnum, + tupDesc, + op->resnull); +} + +/* + * Deform source tuple, filling in the step's values/nulls arrays, before + * evaluating individual new values as part of a FieldStore expression. + * Subsequent steps will overwrite individual elements of the values/nulls + * arrays with the new field values, and then FIELDSTORE_FORM will build the + * new tuple value. + * + * Source record is in step's result variable. + */ +void +ExecEvalFieldStoreDeForm(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + TupleDesc tupDesc; + + /* Lookup tupdesc if first time through or after rescan */ + tupDesc = get_cached_rowtype(op->d.fieldstore.fstore->resulttype, -1, + op->d.fieldstore.argdesc, econtext); + + /* Check that current tupdesc doesn't have more fields than we allocated */ + if (unlikely(tupDesc->natts > op->d.fieldstore.ncolumns)) + elog(ERROR, "too many columns in composite type %u", + op->d.fieldstore.fstore->resulttype); + + if (*op->resnull) + { + /* Convert null input tuple into an all-nulls row */ + memset(op->d.fieldstore.nulls, true, + op->d.fieldstore.ncolumns * sizeof(bool)); + } + else + { + /* + * heap_deform_tuple needs a HeapTuple not a bare HeapTupleHeader. We + * set all the fields in the struct just in case. + */ + Datum tupDatum = *op->resvalue; + HeapTupleHeader tuphdr; + HeapTupleData tmptup; + + tuphdr = DatumGetHeapTupleHeader(tupDatum); + tmptup.t_len = HeapTupleHeaderGetDatumLength(tuphdr); + ItemPointerSetInvalid(&(tmptup.t_self)); + tmptup.t_tableOid = InvalidOid; + tmptup.t_data = tuphdr; + + heap_deform_tuple(&tmptup, tupDesc, + op->d.fieldstore.values, + op->d.fieldstore.nulls); + } +} + +/* + * Compute the new composite datum after each individual field value of a + * FieldStore expression has been evaluated. + */ +void +ExecEvalFieldStoreForm(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + HeapTuple tuple; + + /* argdesc should already be valid from the DeForm step */ + tuple = heap_form_tuple(*op->d.fieldstore.argdesc, + op->d.fieldstore.values, + op->d.fieldstore.nulls); + + *op->resvalue = HeapTupleGetDatum(tuple); + *op->resnull = false; +} + +/* + * Process a subscript in an ArrayRef expression. + * + * If subscript is NULL, throw error in assignment case, or in fetch case + * set result to NULL and return false (instructing caller to skip the rest + * of the ArrayRef sequence). + * + * Subscript expression result is in subscriptvalue/subscriptnull. + * On success, integer subscript value has been saved in upperindex[] or + * lowerindex[] for use later. + */ +bool +ExecEvalArrayRefSubscript(ExprState *state, ExprEvalStep *op) +{ + ArrayRefState *arefstate = op->d.arrayref_subscript.state; + int *indexes; + int off; + + /* If any index expr yields NULL, result is NULL or error */ + if (arefstate->subscriptnull) + { + if (arefstate->isassignment) + ereport(ERROR, + (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("array subscript in assignment must not be null"))); + *op->resnull = true; + return false; + } + + /* Convert datum to int, save in appropriate place */ + if (op->d.arrayref_subscript.isupper) + indexes = arefstate->upperindex; + else + indexes = arefstate->lowerindex; + off = op->d.arrayref_subscript.off; + + indexes[off] = DatumGetInt32(arefstate->subscriptvalue); + + return true; +} + +/* + * Evaluate ArrayRef fetch. + * + * Source array is in step's result variable. + */ +void +ExecEvalArrayRefFetch(ExprState *state, ExprEvalStep *op) +{ + ArrayRefState *arefstate = op->d.arrayref.state; + + /* Should not get here if source array (or any subscript) is null */ + Assert(!(*op->resnull)); + + if (arefstate->numlower == 0) + { + /* Scalar case */ + *op->resvalue = array_get_element(*op->resvalue, + arefstate->numupper, + arefstate->upperindex, + arefstate->refattrlength, + arefstate->refelemlength, + arefstate->refelembyval, + arefstate->refelemalign, + op->resnull); + } + else + { + /* Slice case */ + *op->resvalue = array_get_slice(*op->resvalue, + arefstate->numupper, + arefstate->upperindex, + arefstate->lowerindex, + arefstate->upperprovided, + arefstate->lowerprovided, + arefstate->refattrlength, + arefstate->refelemlength, + arefstate->refelembyval, + arefstate->refelemalign); + } +} + +/* + * Compute old array element/slice value for an ArrayRef assignment + * expression. Will only be generated if the new-value subexpression + * contains ArrayRef or FieldStore. The value is stored into the + * ArrayRefState's prevvalue/prevnull fields. + */ +void +ExecEvalArrayRefOld(ExprState *state, ExprEvalStep *op) +{ + ArrayRefState *arefstate = op->d.arrayref.state; + + if (*op->resnull) + { + /* whole array is null, so any element or slice is too */ + arefstate->prevvalue = (Datum) 0; + arefstate->prevnull = true; + } + else if (arefstate->numlower == 0) + { + /* Scalar case */ + arefstate->prevvalue = array_get_element(*op->resvalue, + arefstate->numupper, + arefstate->upperindex, + arefstate->refattrlength, + arefstate->refelemlength, + arefstate->refelembyval, + arefstate->refelemalign, + &arefstate->prevnull); + } + else + { + /* Slice case */ + /* this is currently unreachable */ + arefstate->prevvalue = array_get_slice(*op->resvalue, + arefstate->numupper, + arefstate->upperindex, + arefstate->lowerindex, + arefstate->upperprovided, + arefstate->lowerprovided, + arefstate->refattrlength, + arefstate->refelemlength, + arefstate->refelembyval, + arefstate->refelemalign); + arefstate->prevnull = false; + } +} + +/* + * Evaluate ArrayRef assignment. + * + * Input array (possibly null) is in result area, replacement value is in + * ArrayRefState's replacevalue/replacenull. + */ +void +ExecEvalArrayRefAssign(ExprState *state, ExprEvalStep *op) +{ + ArrayRefState *arefstate = op->d.arrayref.state; + + /* + * For an assignment to a fixed-length array type, both the original array + * and the value to be assigned into it must be non-NULL, else we punt and + * return the original array. + */ + if (arefstate->refattrlength > 0) /* fixed-length array? */ + { + if (*op->resnull || arefstate->replacenull) + return; + } + + /* + * For assignment to varlena arrays, we handle a NULL original array by + * substituting an empty (zero-dimensional) array; insertion of the new + * element will result in a singleton array value. It does not matter + * whether the new element is NULL. + */ + if (*op->resnull) + { + *op->resvalue = PointerGetDatum(construct_empty_array(arefstate->refelemtype)); + *op->resnull = false; + } + + if (arefstate->numlower == 0) + { + /* Scalar case */ + *op->resvalue = array_set_element(*op->resvalue, + arefstate->numupper, + arefstate->upperindex, + arefstate->replacevalue, + arefstate->replacenull, + arefstate->refattrlength, + arefstate->refelemlength, + arefstate->refelembyval, + arefstate->refelemalign); + } + else + { + /* Slice case */ + *op->resvalue = array_set_slice(*op->resvalue, + arefstate->numupper, + arefstate->upperindex, + arefstate->lowerindex, + arefstate->upperprovided, + arefstate->lowerprovided, + arefstate->replacevalue, + arefstate->replacenull, + arefstate->refattrlength, + arefstate->refelemlength, + arefstate->refelembyval, + arefstate->refelemalign); + } +} + +/* + * Evaluate a rowtype coercion operation. + * This may require rearranging field positions. + * + * Source record is in step's result variable. + */ +void +ExecEvalConvertRowtype(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + ConvertRowtypeExpr *convert = op->d.convert_rowtype.convert; + HeapTuple result; + Datum tupDatum; + HeapTupleHeader tuple; + HeapTupleData tmptup; + TupleDesc indesc, + outdesc; + + /* NULL in -> NULL out */ + if (*op->resnull) + return; + + tupDatum = *op->resvalue; + tuple = DatumGetHeapTupleHeader(tupDatum); + + /* Lookup tupdescs if first time through or after rescan */ + if (op->d.convert_rowtype.indesc == NULL) + { + get_cached_rowtype(exprType((Node *) convert->arg), -1, + &op->d.convert_rowtype.indesc, + econtext); + op->d.convert_rowtype.initialized = false; + } + if (op->d.convert_rowtype.outdesc == NULL) + { + get_cached_rowtype(convert->resulttype, -1, + &op->d.convert_rowtype.outdesc, + econtext); + op->d.convert_rowtype.initialized = false; + } + + indesc = op->d.convert_rowtype.indesc; + outdesc = op->d.convert_rowtype.outdesc; + + /* + * We used to be able to assert that incoming tuples are marked with + * exactly the rowtype of indesc. However, now that ExecEvalWholeRowVar + * might change the tuples' marking to plain RECORD due to inserting + * aliases, we can only make this weak test: + */ + Assert(HeapTupleHeaderGetTypeId(tuple) == indesc->tdtypeid || + HeapTupleHeaderGetTypeId(tuple) == RECORDOID); + + /* if first time through, initialize conversion map */ + if (!op->d.convert_rowtype.initialized) + { + MemoryContext old_cxt; + + /* allocate map in long-lived memory context */ + old_cxt = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + + /* prepare map from old to new attribute numbers */ + op->d.convert_rowtype.map = + convert_tuples_by_name(indesc, outdesc, + gettext_noop("could not convert row type")); + op->d.convert_rowtype.initialized = true; + + MemoryContextSwitchTo(old_cxt); + } + + /* Following steps need a HeapTuple not a bare HeapTupleHeader */ + tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); + tmptup.t_data = tuple; + + if (op->d.convert_rowtype.map != NULL) + { + /* Full conversion with attribute rearrangement needed */ + result = do_convert_tuple(&tmptup, op->d.convert_rowtype.map); + /* Result already has appropriate composite-datum header fields */ + *op->resvalue = HeapTupleGetDatum(result); + } + else + { + /* + * The tuple is physically compatible as-is, but we need to insert the + * destination rowtype OID in its composite-datum header field, so we + * have to copy it anyway. heap_copy_tuple_as_datum() is convenient + * for this since it will both make the physical copy and insert the + * correct composite header fields. Note that we aren't expecting to + * have to flatten any toasted fields: the input was a composite + * datum, so it shouldn't contain any. So heap_copy_tuple_as_datum() + * is overkill here, but its check for external fields is cheap. + */ + *op->resvalue = heap_copy_tuple_as_datum(&tmptup, outdesc); + } +} + +/* + * Evaluate "scalar op ANY/ALL (array)". + * + * Source array is in our result area, scalar arg is already evaluated into + * fcinfo->arg[0]/argnull[0]. + * + * The operator always yields boolean, and we combine the results across all + * array elements using OR and AND (for ANY and ALL respectively). Of course + * we short-circuit as soon as the result is known. + */ +void +ExecEvalScalarArrayOp(ExprState *state, ExprEvalStep *op) +{ + FunctionCallInfo fcinfo = op->d.scalararrayop.fcinfo_data; + bool useOr = op->d.scalararrayop.useOr; + bool strictfunc = op->d.scalararrayop.finfo->fn_strict; + ArrayType *arr; + int nitems; + Datum result; + bool resultnull; + int i; + int16 typlen; + bool typbyval; + char typalign; + char *s; + bits8 *bitmap; + int bitmask; + + /* + * If the array is NULL then we return NULL --- it's not very meaningful + * to do anything else, even if the operator isn't strict. + */ + if (*op->resnull) + return; + + /* Else okay to fetch and detoast the array */ + arr = DatumGetArrayTypeP(*op->resvalue); + + /* + * If the array is empty, we return either FALSE or TRUE per the useOr + * flag. This is correct even if the scalar is NULL; since we would + * evaluate the operator zero times, it matters not whether it would want + * to return NULL. + */ + nitems = ArrayGetNItems(ARR_NDIM(arr), ARR_DIMS(arr)); + if (nitems <= 0) + { + *op->resvalue = BoolGetDatum(!useOr); + *op->resnull = false; + return; + } + + /* + * If the scalar is NULL, and the function is strict, return NULL; no + * point in iterating the loop. + */ + if (fcinfo->argnull[0] && strictfunc) + { + *op->resnull = true; + return; + } + + /* + * We arrange to look up info about the element type only once per series + * of calls, assuming the element type doesn't change underneath us. + */ + if (op->d.scalararrayop.element_type != ARR_ELEMTYPE(arr)) + { + get_typlenbyvalalign(ARR_ELEMTYPE(arr), + &op->d.scalararrayop.typlen, + &op->d.scalararrayop.typbyval, + &op->d.scalararrayop.typalign); + op->d.scalararrayop.element_type = ARR_ELEMTYPE(arr); + } + + typlen = op->d.scalararrayop.typlen; + typbyval = op->d.scalararrayop.typbyval; + typalign = op->d.scalararrayop.typalign; + + /* Initialize result appropriately depending on useOr */ + result = BoolGetDatum(!useOr); + resultnull = false; + + /* Loop over the array elements */ + s = (char *) ARR_DATA_PTR(arr); + bitmap = ARR_NULLBITMAP(arr); + bitmask = 1; + + for (i = 0; i < nitems; i++) + { + Datum elt; + Datum thisresult; + + /* Get array element, checking for NULL */ + if (bitmap && (*bitmap & bitmask) == 0) + { + fcinfo->arg[1] = (Datum) 0; + fcinfo->argnull[1] = true; + } + else + { + elt = fetch_att(s, typbyval, typlen); + s = att_addlength_pointer(s, typlen, s); + s = (char *) att_align_nominal(s, typalign); + fcinfo->arg[1] = elt; + fcinfo->argnull[1] = false; + } + + /* Call comparison function */ + if (fcinfo->argnull[1] && strictfunc) + { + fcinfo->isnull = true; + thisresult = (Datum) 0; + } + else + { + fcinfo->isnull = false; + thisresult = (op->d.scalararrayop.fn_addr) (fcinfo); + } + + /* Combine results per OR or AND semantics */ + if (fcinfo->isnull) + resultnull = true; + else if (useOr) + { + if (DatumGetBool(thisresult)) + { + result = BoolGetDatum(true); + resultnull = false; + break; /* needn't look at any more elements */ + } + } + else + { + if (!DatumGetBool(thisresult)) + { + result = BoolGetDatum(false); + resultnull = false; + break; /* needn't look at any more elements */ + } + } + + /* advance bitmap pointer if any */ + if (bitmap) + { + bitmask <<= 1; + if (bitmask == 0x100) + { + bitmap++; + bitmask = 1; + } + } + } + + *op->resvalue = result; + *op->resnull = resultnull; +} + +/* + * Evaluate a NOT NULL domain constraint. + */ +void +ExecEvalConstraintNotNull(ExprState *state, ExprEvalStep *op) +{ + if (*op->resnull) + ereport(ERROR, + (errcode(ERRCODE_NOT_NULL_VIOLATION), + errmsg("domain %s does not allow null values", + format_type_be(op->d.domaincheck.resulttype)), + errdatatype(op->d.domaincheck.resulttype))); +} + +/* + * Evaluate a CHECK domain constraint. + */ +void +ExecEvalConstraintCheck(ExprState *state, ExprEvalStep *op) +{ + if (!*op->d.domaincheck.checknull && + !DatumGetBool(*op->d.domaincheck.checkvalue)) + ereport(ERROR, + (errcode(ERRCODE_CHECK_VIOLATION), + errmsg("value for domain %s violates check constraint \"%s\"", + format_type_be(op->d.domaincheck.resulttype), + op->d.domaincheck.constraintname), + errdomainconstraint(op->d.domaincheck.resulttype, + op->d.domaincheck.constraintname))); +} + +/* + * Evaluate the various forms of XmlExpr. + * + * Arguments have been evaluated into named_argvalue/named_argnull + * and/or argvalue/argnull arrays. + */ +void +ExecEvalXmlExpr(ExprState *state, ExprEvalStep *op) +{ + XmlExpr *xexpr = op->d.xmlexpr.xexpr; + Datum value; + int i; + + *op->resnull = true; /* until we get a result */ + *op->resvalue = (Datum) 0; + + switch (xexpr->op) + { + case IS_XMLCONCAT: + { + Datum *argvalue = op->d.xmlexpr.argvalue; + bool *argnull = op->d.xmlexpr.argnull; + List *values = NIL; + + for (i = 0; i < list_length(xexpr->args); i++) + { + if (!argnull[i]) + values = lappend(values, DatumGetPointer(argvalue[i])); + } + + if (values != NIL) + { + *op->resvalue = PointerGetDatum(xmlconcat(values)); + *op->resnull = false; + } + } + break; + + case IS_XMLFOREST: + { + Datum *argvalue = op->d.xmlexpr.named_argvalue; + bool *argnull = op->d.xmlexpr.named_argnull; + StringInfoData buf; + ListCell *lc; + ListCell *lc2; + + initStringInfo(&buf); + + i = 0; + forboth(lc, xexpr->named_args, lc2, xexpr->arg_names) + { + Expr *e = (Expr *) lfirst(lc); + char *argname = strVal(lfirst(lc2)); + + if (!argnull[i]) + { + value = argvalue[i]; + appendStringInfo(&buf, "<%s>%s</%s>", + argname, + map_sql_value_to_xml_value(value, + exprType((Node *) e), true), + argname); + *op->resnull = false; + } + i++; + } + + if (!*op->resnull) + { + text *result; + + result = cstring_to_text_with_len(buf.data, buf.len); + *op->resvalue = PointerGetDatum(result); + } + + pfree(buf.data); + } + break; + + case IS_XMLELEMENT: + *op->resvalue = PointerGetDatum(xmlelement(xexpr, + op->d.xmlexpr.named_argvalue, + op->d.xmlexpr.named_argnull, + op->d.xmlexpr.argvalue, + op->d.xmlexpr.argnull)); + *op->resnull = false; + break; + + case IS_XMLPARSE: + { + Datum *argvalue = op->d.xmlexpr.argvalue; + bool *argnull = op->d.xmlexpr.argnull; + text *data; + bool preserve_whitespace; + + /* arguments are known to be text, bool */ + Assert(list_length(xexpr->args) == 2); + + if (argnull[0]) + return; + value = argvalue[0]; + data = DatumGetTextPP(value); + + if (argnull[1]) /* probably can't happen */ + return; + value = argvalue[1]; + preserve_whitespace = DatumGetBool(value); + + *op->resvalue = PointerGetDatum(xmlparse(data, + xexpr->xmloption, + preserve_whitespace)); + *op->resnull = false; + } + break; + + case IS_XMLPI: + { + text *arg; + bool isnull; + + /* optional argument is known to be text */ + Assert(list_length(xexpr->args) <= 1); + + if (xexpr->args) + { + isnull = op->d.xmlexpr.argnull[0]; + if (isnull) + arg = NULL; + else + arg = DatumGetTextPP(op->d.xmlexpr.argvalue[0]); + } + else + { + arg = NULL; + isnull = false; + } + + *op->resvalue = PointerGetDatum(xmlpi(xexpr->name, + arg, + isnull, + op->resnull)); + } + break; + + case IS_XMLROOT: + { + Datum *argvalue = op->d.xmlexpr.argvalue; + bool *argnull = op->d.xmlexpr.argnull; + xmltype *data; + text *version; + int standalone; + + /* arguments are known to be xml, text, int */ + Assert(list_length(xexpr->args) == 3); + + if (argnull[0]) + return; + data = DatumGetXmlP(argvalue[0]); + + if (argnull[1]) + version = NULL; + else + version = DatumGetTextPP(argvalue[1]); + + Assert(!argnull[2]); /* always present */ + standalone = DatumGetInt32(argvalue[2]); + + *op->resvalue = PointerGetDatum(xmlroot(data, + version, + standalone)); + *op->resnull = false; + } + break; + + case IS_XMLSERIALIZE: + { + Datum *argvalue = op->d.xmlexpr.argvalue; + bool *argnull = op->d.xmlexpr.argnull; + + /* argument type is known to be xml */ + Assert(list_length(xexpr->args) == 1); + + if (argnull[0]) + return; + value = argvalue[0]; + + *op->resvalue = PointerGetDatum( + xmltotext_with_xmloption(DatumGetXmlP(value), + xexpr->xmloption)); + *op->resnull = false; + } + break; + + case IS_DOCUMENT: + { + Datum *argvalue = op->d.xmlexpr.argvalue; + bool *argnull = op->d.xmlexpr.argnull; + + /* optional argument is known to be xml */ + Assert(list_length(xexpr->args) == 1); + + if (argnull[0]) + return; + value = argvalue[0]; + + *op->resvalue = + BoolGetDatum(xml_is_document(DatumGetXmlP(value))); + *op->resnull = false; + } + break; + + default: + elog(ERROR, "unrecognized XML operation"); + break; + } +} + +/* + * ExecEvalGroupingFunc + * + * Computes a bitmask with a bit for each (unevaluated) argument expression + * (rightmost arg is least significant bit). + * + * A bit is set if the corresponding expression is NOT part of the set of + * grouping expressions in the current grouping set. + */ +void +ExecEvalGroupingFunc(ExprState *state, ExprEvalStep *op) +{ + int result = 0; + Bitmapset *grouped_cols = op->d.grouping_func.parent->grouped_cols; + ListCell *lc; + + foreach(lc, op->d.grouping_func.clauses) + { + int attnum = lfirst_int(lc); + + result <<= 1; + + if (!bms_is_member(attnum, grouped_cols)) + result |= 1; + } + + *op->resvalue = Int32GetDatum(result); + *op->resnull = false; +} + +/* + * Hand off evaluation of a subplan to nodeSubplan.c + */ +void +ExecEvalSubPlan(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + SubPlanState *sstate = op->d.subplan.sstate; + + /* could potentially be nested, so make sure there's enough stack */ + check_stack_depth(); + + *op->resvalue = ExecSubPlan(sstate, econtext, op->resnull); +} + +/* + * Hand off evaluation of an alternative subplan to nodeSubplan.c + */ +void +ExecEvalAlternativeSubPlan(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + AlternativeSubPlanState *asstate = op->d.alternative_subplan.asstate; + + /* could potentially be nested, so make sure there's enough stack */ + check_stack_depth(); + + *op->resvalue = ExecAlternativeSubPlan(asstate, econtext, op->resnull); +} + +/* + * Evaluate a wholerow Var expression. + * + * Returns a Datum whose value is the value of a whole-row range variable + * with respect to given expression context. + */ +void +ExecEvalWholeRowVar(ExprState *state, ExprEvalStep *op, ExprContext *econtext) +{ + Var *variable = op->d.wholerow.var; + TupleTableSlot *slot; + TupleDesc output_tupdesc; + MemoryContext oldcontext; + HeapTupleHeader dtuple; + HeapTuple tuple; + + /* This was checked by ExecInitExpr */ + Assert(variable->varattno == InvalidAttrNumber); + + /* Get the input slot we want */ + switch (variable->varno) + { + case INNER_VAR: + /* get the tuple from the inner node */ + slot = econtext->ecxt_innertuple; + break; + + case OUTER_VAR: + /* get the tuple from the outer node */ + slot = econtext->ecxt_outertuple; + break; + + /* INDEX_VAR is handled by default case */ + + default: + /* get the tuple from the relation being scanned */ + slot = econtext->ecxt_scantuple; + break; + } + + /* Apply the junkfilter if any */ + if (op->d.wholerow.junkFilter != NULL) + slot = ExecFilterJunk(op->d.wholerow.junkFilter, slot); + + /* + * If first time through, obtain tuple descriptor and check compatibility. + * + * XXX: It'd be great if this could be moved to the expression + * initialization phase, but due to using slots that's currently not + * feasible. + */ + if (op->d.wholerow.first) + { + /* optimistically assume we don't need slow path */ + op->d.wholerow.slow = false; + + /* + * If the Var identifies a named composite type, we must check that + * the actual tuple type is compatible with it. + */ + if (variable->vartype != RECORDOID) + { + TupleDesc var_tupdesc; + TupleDesc slot_tupdesc; + int i; + + /* + * We really only care about numbers of attributes and data types. + * Also, we can ignore type mismatch on columns that are dropped + * in the destination type, so long as (1) the physical storage + * matches or (2) the actual column value is NULL. Case (1) is + * helpful in some cases involving out-of-date cached plans, while + * case (2) is expected behavior in situations such as an INSERT + * into a table with dropped columns (the planner typically + * generates an INT4 NULL regardless of the dropped column type). + * If we find a dropped column and cannot verify that case (1) + * holds, we have to use the slow path to check (2) for each row. + */ + var_tupdesc = lookup_rowtype_tupdesc(variable->vartype, -1); + + slot_tupdesc = slot->tts_tupleDescriptor; + + if (var_tupdesc->natts != slot_tupdesc->natts) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("table row type and query-specified row type do not match"), + errdetail_plural("Table row contains %d attribute, but query expects %d.", + "Table row contains %d attributes, but query expects %d.", + slot_tupdesc->natts, + slot_tupdesc->natts, + var_tupdesc->natts))); + + for (i = 0; i < var_tupdesc->natts; i++) + { + Form_pg_attribute vattr = var_tupdesc->attrs[i]; + Form_pg_attribute sattr = slot_tupdesc->attrs[i]; + + if (vattr->atttypid == sattr->atttypid) + continue; /* no worries */ + if (!vattr->attisdropped) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("table row type and query-specified row type do not match"), + errdetail("Table has type %s at ordinal position %d, but query expects %s.", + format_type_be(sattr->atttypid), + i + 1, + format_type_be(vattr->atttypid)))); + + if (vattr->attlen != sattr->attlen || + vattr->attalign != sattr->attalign) + op->d.wholerow.slow = true; /* need to check for nulls */ + } + + /* + * Use the variable's declared rowtype as the descriptor for the + * output values, modulo possibly assigning new column names + * below. In particular, we *must* absorb any attisdropped + * markings. + */ + oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + output_tupdesc = CreateTupleDescCopy(var_tupdesc); + MemoryContextSwitchTo(oldcontext); + + ReleaseTupleDesc(var_tupdesc); + } + else + { + /* + * In the RECORD case, we use the input slot's rowtype as the + * descriptor for the output values, modulo possibly assigning new + * column names below. + */ + oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + output_tupdesc = CreateTupleDescCopy(slot->tts_tupleDescriptor); + MemoryContextSwitchTo(oldcontext); + } + + /* + * Construct a tuple descriptor for the composite values we'll + * produce, and make sure its record type is "blessed". The main + * reason to do this is to be sure that operations such as + * row_to_json() will see the desired column names when they look up + * the descriptor from the type information embedded in the composite + * values. + * + * We already got the correct physical datatype info above, but now we + * should try to find the source RTE and adopt its column aliases, in + * case they are different from the original rowtype's names. For + * example, in "SELECT foo(t) FROM tab t(x,y)", the first two columns + * in the composite output should be named "x" and "y" regardless of + * tab's column names. + * + * If we can't locate the RTE, assume the column names we've got are + * OK. (As of this writing, the only cases where we can't locate the + * RTE are in execution of trigger WHEN clauses, and then the Var will + * have the trigger's relation's rowtype, so its names are fine.) + * Also, if the creator of the RTE didn't bother to fill in an eref + * field, assume our column names are OK. (This happens in COPY, and + * perhaps other places.) + */ + if (econtext->ecxt_estate && + variable->varno <= list_length(econtext->ecxt_estate->es_range_table)) + { + RangeTblEntry *rte = rt_fetch(variable->varno, + econtext->ecxt_estate->es_range_table); + + if (rte->eref) + ExecTypeSetColNames(output_tupdesc, rte->eref->colnames); + } + + /* Bless the tupdesc if needed, and save it in the execution state */ + op->d.wholerow.tupdesc = BlessTupleDesc(output_tupdesc); + + op->d.wholerow.first = false; + } + + /* + * Make sure all columns of the slot are accessible in the slot's + * Datum/isnull arrays. + */ + slot_getallattrs(slot); + + if (op->d.wholerow.slow) + { + /* Check to see if any dropped attributes are non-null */ + TupleDesc tupleDesc = slot->tts_tupleDescriptor; + TupleDesc var_tupdesc = op->d.wholerow.tupdesc; + int i; + + Assert(var_tupdesc->natts == tupleDesc->natts); + + for (i = 0; i < var_tupdesc->natts; i++) + { + Form_pg_attribute vattr = var_tupdesc->attrs[i]; + Form_pg_attribute sattr = tupleDesc->attrs[i]; + + if (!vattr->attisdropped) + continue; /* already checked non-dropped cols */ + if (slot->tts_isnull[i]) + continue; /* null is always okay */ + if (vattr->attlen != sattr->attlen || + vattr->attalign != sattr->attalign) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("table row type and query-specified row type do not match"), + errdetail("Physical storage mismatch on dropped attribute at ordinal position %d.", + i + 1))); + } + } + + /* + * Build a composite datum, making sure any toasted fields get detoasted. + * + * (Note: it is critical that we not change the slot's state here.) + */ + tuple = toast_build_flattened_tuple(slot->tts_tupleDescriptor, + slot->tts_values, + slot->tts_isnull); + dtuple = tuple->t_data; + + /* + * Label the datum with the composite type info we identified before. + * + * (Note: we could skip doing this by passing op->d.wholerow.tupdesc to + * the tuple build step; but that seems a tad risky so let's not.) + */ + HeapTupleHeaderSetTypeId(dtuple, op->d.wholerow.tupdesc->tdtypeid); + HeapTupleHeaderSetTypMod(dtuple, op->d.wholerow.tupdesc->tdtypmod); + + *op->resvalue = PointerGetDatum(dtuple); + *op->resnull = false; +} diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c index 808275a094..07c8852fca 100644 --- a/src/backend/executor/execGrouping.c +++ b/src/backend/executor/execGrouping.c @@ -7,7 +7,7 @@ * collation-sensitive, so the code in this file has no support for passing * collation settings through from callers. That may have to change someday. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -18,17 +18,32 @@ */ #include "postgres.h" +#include "access/hash.h" +#include "access/parallel.h" #include "executor/executor.h" #include "miscadmin.h" #include "utils/lsyscache.h" #include "utils/memutils.h" +static uint32 TupleHashTableHash(struct tuplehash_hash *tb, const MinimalTuple tuple); +static int TupleHashTableMatch(struct tuplehash_hash *tb, const MinimalTuple tuple1, const MinimalTuple tuple2); -static TupleHashTable CurTupleHashTable = NULL; - -static uint32 TupleHashTableHash(const void *key, Size keysize); -static int TupleHashTableMatch(const void *key1, const void *key2, - Size keysize); +/* + * Define parameters for tuple hash table code generation. The interface is + * *also* declared in execnodes.h (to generate the types, which are externally + * visible). + */ +#define SH_PREFIX tuplehash +#define SH_ELEMENT_TYPE TupleHashEntryData +#define SH_KEY_TYPE MinimalTuple +#define SH_KEY firstTuple +#define SH_HASH_KEY(tb, key) TupleHashTableHash(tb, key) +#define SH_EQUAL(tb, a, b) TupleHashTableMatch(tb, a, b) == 0 +#define SH_SCOPE extern +#define SH_STORE_HASH +#define SH_GET_HASH(tb, a) a->hash +#define SH_DEFINE +#include "lib/simplehash.h" /***************************************************************************** @@ -260,7 +275,7 @@ execTuplesHashPrepare(int numCols, * eqfunctions: equality comparison functions to use * hashfunctions: datatype-specific hashing functions to use * nbuckets: initial estimate of hashtable size - * entrysize: size of each entry (at least sizeof(TupleHashEntryData)) + * additionalsize: size of data stored in ->additional * tablecxt: memory context in which to store table and table entries * tempcxt: short-lived context for evaluation hash and comparison functions * @@ -275,20 +290,20 @@ TupleHashTable BuildTupleHashTable(int numCols, AttrNumber *keyColIdx, FmgrInfo *eqfunctions, FmgrInfo *hashfunctions, - long nbuckets, Size entrysize, - MemoryContext tablecxt, MemoryContext tempcxt) + long nbuckets, Size additionalsize, + MemoryContext tablecxt, MemoryContext tempcxt, + bool use_variable_hash_iv) { TupleHashTable hashtable; - HASHCTL hash_ctl; + Size entrysize = sizeof(TupleHashEntryData) + additionalsize; Assert(nbuckets > 0); - Assert(entrysize >= sizeof(TupleHashEntryData)); /* Limit initial table size request to not more than work_mem */ nbuckets = Min(nbuckets, (long) ((work_mem * 1024L) / entrysize)); - hashtable = (TupleHashTable) MemoryContextAlloc(tablecxt, - sizeof(TupleHashTableData)); + hashtable = (TupleHashTable) + MemoryContextAlloc(tablecxt, sizeof(TupleHashTableData)); hashtable->numCols = numCols; hashtable->keyColIdx = keyColIdx; @@ -302,15 +317,20 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx, hashtable->in_hash_funcs = NULL; hashtable->cur_eq_funcs = NULL; - MemSet(&hash_ctl, 0, sizeof(hash_ctl)); - hash_ctl.keysize = sizeof(TupleHashEntryData); - hash_ctl.entrysize = entrysize; - hash_ctl.hash = TupleHashTableHash; - hash_ctl.match = TupleHashTableMatch; - hash_ctl.hcxt = tablecxt; - hashtable->hashtab = hash_create("TupleHashTable", nbuckets, - &hash_ctl, - HASH_ELEM | HASH_FUNCTION | HASH_COMPARE | HASH_CONTEXT); + /* + * If parallelism is in use, even if the master backend is performing the + * scan itself, we don't want to create the hashtable exactly the same way + * in all workers. As hashtables are iterated over in keyspace-order, + * doing so in all processes in the same way is likely to lead to + * "unbalanced" hashtables when the table size initially is + * underestimated. + */ + if (use_variable_hash_iv) + hashtable->hash_iv = hash_uint32(ParallelWorkerNumber); + else + hashtable->hash_iv = 0; + + hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, hashtable); return hashtable; } @@ -324,18 +344,17 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx, * * If isnew isn't NULL, then a new entry is created if no existing entry * matches. On return, *isnew is true if the entry is newly created, - * false if it existed already. Any extra space in a new entry has been - * zeroed. + * false if it existed already. ->additional_data in the new entry has + * been zeroed. */ TupleHashEntry LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, bool *isnew) { - TupleHashEntry entry; + TupleHashEntryData *entry; MemoryContext oldContext; - TupleHashTable saveCurHT; - TupleHashEntryData dummy; bool found; + MinimalTuple key; /* If first time through, clone the input slot to make table slot */ if (hashtable->tableslot == NULL) @@ -356,28 +375,17 @@ LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, /* Need to run the hash functions in short-lived context */ oldContext = MemoryContextSwitchTo(hashtable->tempcxt); - /* - * Set up data needed by hash and match functions - * - * We save and restore CurTupleHashTable just in case someone manages to - * invoke this code re-entrantly. - */ + /* set up data needed by hash and match functions */ hashtable->inputslot = slot; hashtable->in_hash_funcs = hashtable->tab_hash_funcs; hashtable->cur_eq_funcs = hashtable->tab_eq_funcs; - saveCurHT = CurTupleHashTable; - CurTupleHashTable = hashtable; - - /* Search the hash table */ - dummy.firstTuple = NULL; /* flag to reference inputslot */ - entry = (TupleHashEntry) hash_search(hashtable->hashtab, - &dummy, - isnew ? HASH_ENTER : HASH_FIND, - &found); + key = NULL; /* flag to reference inputslot */ if (isnew) { + entry = tuplehash_insert(hashtable->hashtab, key, &found); + if (found) { /* found pre-existing entry */ @@ -385,24 +393,19 @@ LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, } else { - /* - * created new entry - * - * Zero any caller-requested space in the entry. (This zaps the - * "key data" dynahash.c copied into the new entry, but we don't - * care since we're about to overwrite it anyway.) - */ - MemSet(entry, 0, hashtable->entrysize); - - /* Copy the first tuple into the table context */ + /* created new entry */ + *isnew = true; + /* zero caller data */ + entry->additional = NULL; MemoryContextSwitchTo(hashtable->tablecxt); + /* Copy the first tuple into the table context */ entry->firstTuple = ExecCopySlotMinimalTuple(slot); - - *isnew = true; } } - - CurTupleHashTable = saveCurHT; + else + { + entry = tuplehash_lookup(hashtable->hashtab, key); + } MemoryContextSwitchTo(oldContext); @@ -425,34 +428,19 @@ FindTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, { TupleHashEntry entry; MemoryContext oldContext; - TupleHashTable saveCurHT; - TupleHashEntryData dummy; + MinimalTuple key; /* Need to run the hash functions in short-lived context */ oldContext = MemoryContextSwitchTo(hashtable->tempcxt); - /* - * Set up data needed by hash and match functions - * - * We save and restore CurTupleHashTable just in case someone manages to - * invoke this code re-entrantly. - */ + /* Set up data needed by hash and match functions */ hashtable->inputslot = slot; hashtable->in_hash_funcs = hashfunctions; hashtable->cur_eq_funcs = eqfunctions; - saveCurHT = CurTupleHashTable; - CurTupleHashTable = hashtable; - /* Search the hash table */ - dummy.firstTuple = NULL; /* flag to reference inputslot */ - entry = (TupleHashEntry) hash_search(hashtable->hashtab, - &dummy, - HASH_FIND, - NULL); - - CurTupleHashTable = saveCurHT; - + key = NULL; /* flag to reference inputslot */ + entry = tuplehash_lookup(hashtable->hashtab, key); MemoryContextSwitchTo(oldContext); return entry; @@ -468,22 +456,18 @@ FindTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, * This convention avoids the need to materialize virtual input tuples unless * they actually need to get copied into the table. * - * CurTupleHashTable must be set before calling this, since dynahash.c - * doesn't provide any API that would let us get at the hashtable otherwise. - * * Also, the caller must select an appropriate memory context for running * the hash functions. (dynahash.c doesn't change CurrentMemoryContext.) */ static uint32 -TupleHashTableHash(const void *key, Size keysize) +TupleHashTableHash(struct tuplehash_hash *tb, const MinimalTuple tuple) { - MinimalTuple tuple = ((const TupleHashEntryData *) key)->firstTuple; - TupleTableSlot *slot; - TupleHashTable hashtable = CurTupleHashTable; + TupleHashTable hashtable = (TupleHashTable) tb->private_data; int numCols = hashtable->numCols; AttrNumber *keyColIdx = hashtable->keyColIdx; + uint32 hashkey = hashtable->hash_iv; + TupleTableSlot *slot; FmgrInfo *hashfunctions; - uint32 hashkey = 0; int i; if (tuple == NULL) @@ -494,8 +478,12 @@ TupleHashTableHash(const void *key, Size keysize) } else { - /* Process a tuple already stored in the table */ - /* (this case never actually occurs in current dynahash.c code) */ + /* + * Process a tuple already stored in the table. + * + * (this case never actually occurs due to the way simplehash.h is + * used, as the hash-value is stored in the entries) + */ slot = hashtable->tableslot; ExecStoreMinimalTuple(tuple, slot, false); hashfunctions = hashtable->tab_hash_funcs; @@ -530,29 +518,21 @@ TupleHashTableHash(const void *key, Size keysize) * * As above, the passed pointers are pointers to TupleHashEntryData. * - * CurTupleHashTable must be set before calling this, since dynahash.c - * doesn't provide any API that would let us get at the hashtable otherwise. - * * Also, the caller must select an appropriate memory context for running * the compare functions. (dynahash.c doesn't change CurrentMemoryContext.) */ static int -TupleHashTableMatch(const void *key1, const void *key2, Size keysize) +TupleHashTableMatch(struct tuplehash_hash *tb, const MinimalTuple tuple1, const MinimalTuple tuple2) { - MinimalTuple tuple1 = ((const TupleHashEntryData *) key1)->firstTuple; - -#ifdef USE_ASSERT_CHECKING - MinimalTuple tuple2 = ((const TupleHashEntryData *) key2)->firstTuple; -#endif TupleTableSlot *slot1; TupleTableSlot *slot2; - TupleHashTable hashtable = CurTupleHashTable; + TupleHashTable hashtable = (TupleHashTable) tb->private_data; /* - * We assume that dynahash.c will only ever call us with the first + * We assume that simplehash.h will only ever call us with the first * argument being an actual table entry, and the second argument being * LookupTupleHashEntry's dummy TupleHashEntryData. The other direction - * could be supported too, but is not currently used by dynahash.c. + * could be supported too, but is not currently required. */ Assert(tuple1 != NULL); slot1 = hashtable->tableslot; diff --git a/src/backend/executor/execIndexing.c b/src/backend/executor/execIndexing.c index 0e2d834ed1..108060ac0f 100644 --- a/src/backend/executor/execIndexing.c +++ b/src/backend/executor/execIndexing.c @@ -2,7 +2,7 @@ * * execIndexing.c * routines for inserting index tuples and enforcing unique and - * exclusive constraints. + * exclusion constraints. * * ExecInsertIndexTuples() is the main entry point. It's called after * inserting a tuple to the heap, and it inserts corresponding index tuples @@ -95,7 +95,7 @@ * with the higher XID backs out. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -327,23 +327,21 @@ ExecInsertIndexTuples(TupleTableSlot *slot, /* Check for partial index */ if (indexInfo->ii_Predicate != NIL) { - List *predicate; + ExprState *predicate; /* * If predicate state not set up yet, create it (in the estate's * per-query context) */ predicate = indexInfo->ii_PredicateState; - if (predicate == NIL) + if (predicate == NULL) { - predicate = (List *) - ExecPrepareExpr((Expr *) indexInfo->ii_Predicate, - estate); + predicate = ExecPrepareQual(indexInfo->ii_Predicate, estate); indexInfo->ii_PredicateState = predicate; } /* Skip this index-update if the predicate isn't satisfied */ - if (!ExecQual(predicate, econtext, false)) + if (!ExecQual(predicate, econtext)) continue; } @@ -391,7 +389,8 @@ ExecInsertIndexTuples(TupleTableSlot *slot, isnull, /* null flags */ tupleid, /* tid of heap tuple */ heapRelation, /* heap relation */ - checkUnique); /* type of uniqueness check to do */ + checkUnique, /* type of uniqueness check to do */ + indexInfo); /* index AM may need this */ /* * If the index has an associated exclusion constraint, check that. @@ -550,23 +549,21 @@ ExecCheckIndexConstraints(TupleTableSlot *slot, /* Check for partial index */ if (indexInfo->ii_Predicate != NIL) { - List *predicate; + ExprState *predicate; /* * If predicate state not set up yet, create it (in the estate's * per-query context) */ predicate = indexInfo->ii_PredicateState; - if (predicate == NIL) + if (predicate == NULL) { - predicate = (List *) - ExecPrepareExpr((Expr *) indexInfo->ii_Predicate, - estate); + predicate = ExecPrepareQual(indexInfo->ii_Predicate, estate); indexInfo->ii_PredicateState = predicate; } /* Skip this index-update if the predicate isn't satisfied */ - if (!ExecQual(predicate, econtext, false)) + if (!ExecQual(predicate, econtext)) continue; } diff --git a/src/backend/executor/execJunk.c b/src/backend/executor/execJunk.c index 74a10048a2..a422327c88 100644 --- a/src/backend/executor/execJunk.c +++ b/src/backend/executor/execJunk.c @@ -3,7 +3,7 @@ * execJunk.c * Junk attribute support stuff.... * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c index 2eaa33455c..7232b0911f 100644 --- a/src/backend/executor/execMain.c +++ b/src/backend/executor/execMain.c @@ -27,7 +27,7 @@ * which should also omit ExecutorRun. * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -43,6 +43,8 @@ #include "access/transam.h" #include "access/xact.h" #include "catalog/namespace.h" +#include "catalog/partition.h" +#include "catalog/pg_publication.h" #include "commands/matview.h" #include "commands/trigger.h" #include "executor/execdebug.h" @@ -51,6 +53,7 @@ #include "miscadmin.h" #include "optimizer/clauses.h" #include "parser/parsetree.h" +#include "rewrite/rewriteManip.h" #include "storage/bufmgr.h" #include "storage/lmgr.h" #include "tcop/utility.h" @@ -58,6 +61,7 @@ #include "utils/lsyscache.h" #include "utils/memutils.h" #include "utils/rls.h" +#include "utils/ruleutils.h" #include "utils/snapmgr.h" #include "utils/tqual.h" #ifdef PGXC @@ -90,7 +94,8 @@ static void ExecutePlan(EState *estate, PlanState *planstate, bool sendTuples, uint64 numberTuples, ScanDirection direction, - DestReceiver *dest); + DestReceiver *dest, + bool execute_once); static bool ExecCheckRTEPerms(RangeTblEntry *rte); static bool ExecCheckRTEPermsModified(Oid relOid, Oid userid, Bitmapset *modifiedCols, @@ -101,6 +106,10 @@ static char *ExecBuildSlotValueDescription(Oid reloid, TupleDesc tupdesc, Bitmapset *modifiedCols, int maxfieldlen); +static char *ExecBuildSlotPartitionKeyDescription(Relation rel, + Datum *values, + bool *isnull, + int maxfieldlen); static void EvalPlanQualStart(EPQState *epqstate, EState *parentestate, Plan *planTree); @@ -234,6 +243,13 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags) palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData)); #endif + estate->es_sourceText = queryDesc->sourceText; + + /* + * Fill in the query environment, if any, from queryDesc. + */ + estate->es_queryEnv = queryDesc->queryEnv; + /* * If non-read-only query, set the command ID to mark output tuples with */ @@ -325,17 +341,18 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags) */ void ExecutorRun(QueryDesc *queryDesc, - ScanDirection direction, uint64 count) + ScanDirection direction, uint64 count, + bool execute_once) { if (ExecutorRun_hook) - (*ExecutorRun_hook) (queryDesc, direction, count); + (*ExecutorRun_hook) (queryDesc, direction, count, execute_once); else - standard_ExecutorRun(queryDesc, direction, count); + standard_ExecutorRun(queryDesc, direction, count, execute_once); } void standard_ExecutorRun(QueryDesc *queryDesc, - ScanDirection direction, uint64 count) + ScanDirection direction, uint64 count, bool execute_once) { EState *estate; CmdType operation; @@ -382,6 +399,11 @@ standard_ExecutorRun(QueryDesc *queryDesc, * run plan */ if (!ScanDirectionIsNoMovement(direction)) + { + if (execute_once && queryDesc->already_executed) + elog(ERROR, "can't re-execute query flagged for single execution"); + queryDesc->already_executed = true; + ExecutePlan(estate, queryDesc->planstate, queryDesc->plannedstmt->parallelModeNeeded, @@ -389,7 +411,9 @@ standard_ExecutorRun(QueryDesc *queryDesc, sendTuples, count, direction, - dest); + dest, + execute_once); + } /* * shutdown tuple receiver, if we started it @@ -628,8 +652,8 @@ ExecCheckRTEPerms(RangeTblEntry *rte) /* * Only plain-relation RTEs need to be checked here. Function RTEs are - * checked by init_fcache when the function is prepared for execution. - * Join, subquery, and special RTEs need no checks. + * checked when the function is prepared for execution. Join, subquery, + * and special RTEs need no checks. */ if (rte->rtekind != RTE_RELATION) return true; @@ -869,9 +893,11 @@ InitPlan(QueryDesc *queryDesc, int eflags) resultRelationOid = getrelid(resultRelationIndex, rangeTable); resultRelation = heap_open(resultRelationOid, RowExclusiveLock); + InitResultRelInfo(resultRelInfo, resultRelation, resultRelationIndex, + NULL, estate->es_instrument); resultRelInfo++; } @@ -879,6 +905,57 @@ InitPlan(QueryDesc *queryDesc, int eflags) estate->es_num_result_relations = numResultRelations; /* es_result_relation_info is NULL except when within ModifyTable */ estate->es_result_relation_info = NULL; + + /* + * In the partitioned result relation case, lock the non-leaf result + * relations too. A subset of these are the roots of respective + * partitioned tables, for which we also allocate ResulRelInfos. + */ + estate->es_root_result_relations = NULL; + estate->es_num_root_result_relations = 0; + if (plannedstmt->nonleafResultRelations) + { + int num_roots = list_length(plannedstmt->rootResultRelations); + + /* + * Firstly, build ResultRelInfos for all the partitioned table + * roots, because we will need them to fire the statement-level + * triggers, if any. + */ + resultRelInfos = (ResultRelInfo *) + palloc(num_roots * sizeof(ResultRelInfo)); + resultRelInfo = resultRelInfos; + foreach(l, plannedstmt->rootResultRelations) + { + Index resultRelIndex = lfirst_int(l); + Oid resultRelOid; + Relation resultRelDesc; + + resultRelOid = getrelid(resultRelIndex, rangeTable); + resultRelDesc = heap_open(resultRelOid, RowExclusiveLock); + InitResultRelInfo(resultRelInfo, + resultRelDesc, + lfirst_int(l), + NULL, + estate->es_instrument); + resultRelInfo++; + } + + estate->es_root_result_relations = resultRelInfos; + estate->es_num_root_result_relations = num_roots; + + /* Simply lock the rest of them. */ + foreach(l, plannedstmt->nonleafResultRelations) + { + Index resultRelIndex = lfirst_int(l); + + /* We locked the roots above. */ + if (!list_member_int(plannedstmt->rootResultRelations, + resultRelIndex)) + LockRelationOid(getrelid(resultRelIndex, rangeTable), + RowExclusiveLock); + } + } } else { @@ -888,12 +965,18 @@ InitPlan(QueryDesc *queryDesc, int eflags) estate->es_result_relations = NULL; estate->es_num_result_relations = 0; estate->es_result_relation_info = NULL; + estate->es_root_result_relations = NULL; + estate->es_num_root_result_relations = 0; } /* * Similarly, we have to lock relations selected FOR [KEY] UPDATE/SHARE * before we initialize the plan tree, else we'd be risking lock upgrades. - * While we are at it, build the ExecRowMark list. + * While we are at it, build the ExecRowMark list. Any partitioned child + * tables are ignored here (because isParent=true) and will be locked by + * the first Append or MergeAppend node that references them. (Note that + * the RowMarks corresponding to partitioned child tables are present in + * the same list as the rest, i.e., plannedstmt->rowMarks.) */ estate->es_rowMarks = NIL; foreach(l, plannedstmt->rowMarks) @@ -1084,7 +1167,8 @@ CheckValidResultRel(Relation resultRel, CmdType operation) switch (resultRel->rd_rel->relkind) { case RELKIND_RELATION: - /* OK */ + case RELKIND_PARTITIONED_TABLE: + CheckCmdReplicaIdentity(resultRel, operation); break; case RELKIND_SEQUENCE: ereport(ERROR, @@ -1217,6 +1301,7 @@ CheckValidRowMarkRel(Relation rel, RowMarkType markType) switch (rel->rd_rel->relkind) { case RELKIND_RELATION: + case RELKIND_PARTITIONED_TABLE: /* OK */ break; case RELKIND_SEQUENCE: @@ -1277,8 +1362,11 @@ void InitResultRelInfo(ResultRelInfo *resultRelInfo, Relation resultRelationDesc, Index resultRelationIndex, + Relation partition_root, int instrument_options) { + List *partition_check = NIL; + MemSet(resultRelInfo, 0, sizeof(ResultRelInfo)); resultRelInfo->type = T_ResultRelInfo; resultRelInfo->ri_RangeTableIndex = resultRelationIndex; @@ -1294,8 +1382,8 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo, resultRelInfo->ri_TrigFunctions = (FmgrInfo *) palloc0(n * sizeof(FmgrInfo)); - resultRelInfo->ri_TrigWhenExprs = (List **) - palloc0(n * sizeof(List *)); + resultRelInfo->ri_TrigWhenExprs = (ExprState **) + palloc0(n * sizeof(ExprState *)); if (instrument_options) resultRelInfo->ri_TrigInstrument = InstrAlloc(n, instrument_options); } @@ -1314,6 +1402,39 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo, resultRelInfo->ri_ConstraintExprs = NULL; resultRelInfo->ri_junkFilter = NULL; resultRelInfo->ri_projectReturning = NULL; + + /* + * If partition_root has been specified, that means we are building the + * ResultRelInfo for one of its leaf partitions. In that case, we need + * *not* initialize the leaf partition's constraint, but rather the + * partition_root's (if any). We must do that explicitly like this, + * because implicit partition constraints are not inherited like user- + * defined constraints and would fail to be enforced by ExecConstraints() + * after a tuple is routed to a leaf partition. + */ + if (partition_root) + { + /* + * Root table itself may or may not be a partition; partition_check + * would be NIL in the latter case. + */ + partition_check = RelationGetPartitionQual(partition_root); + + /* + * This is not our own partition constraint, but rather an ancestor's. + * So any Vars in it bear the ancestor's attribute numbers. We must + * switch them to our own. (dummy varno = 1) + */ + if (partition_check != NIL) + partition_check = map_partition_varattnos(partition_check, 1, + resultRelationDesc, + partition_root); + } + else + partition_check = RelationGetPartitionQual(resultRelationDesc); + + resultRelInfo->ri_PartitionCheck = partition_check; + resultRelInfo->ri_PartitionRoot = partition_root; } /* @@ -1376,6 +1497,7 @@ ExecGetTriggerResultRel(EState *estate, Oid relid) InitResultRelInfo(rInfo, rel, 0, /* dummy rangetable index */ + NULL, estate->es_instrument); estate->es_trig_target_relations = lappend(estate->es_trig_target_relations, rInfo); @@ -1390,6 +1512,24 @@ ExecGetTriggerResultRel(EState *estate, Oid relid) } /* + * Close any relations that have been opened by ExecGetTriggerResultRel(). + */ +void +ExecCleanUpTriggerState(EState *estate) +{ + ListCell *l; + + foreach(l, estate->es_trig_target_relations) + { + ResultRelInfo *resultRelInfo = (ResultRelInfo *) lfirst(l); + + /* Close indices and then the relation itself */ + ExecCloseIndices(resultRelInfo); + heap_close(resultRelInfo->ri_RelationDesc, NoLock); + } +} + +/* * ExecContextForcesOids * * This is pretty grotty: when doing INSERT, UPDATE, or CREATE TABLE AS, @@ -1545,17 +1685,17 @@ ExecEndPlan(PlanState *planstate, EState *estate) resultRelInfo++; } - /* - * likewise close any trigger target relations - */ - foreach(l, estate->es_trig_target_relations) + /* Close the root target relation(s). */ + resultRelInfo = estate->es_root_result_relations; + for (i = estate->es_num_root_result_relations; i > 0; i--) { - resultRelInfo = (ResultRelInfo *) lfirst(l); - /* Close indices and then the relation itself */ - ExecCloseIndices(resultRelInfo); heap_close(resultRelInfo->ri_RelationDesc, NoLock); + resultRelInfo++; } + /* likewise close any trigger target relations */ + ExecCleanUpTriggerState(estate); + /* * close any relations selected FOR [KEY] UPDATE/SHARE, again keeping * locks @@ -1589,7 +1729,8 @@ ExecutePlan(EState *estate, bool sendTuples, uint64 numberTuples, ScanDirection direction, - DestReceiver *dest) + DestReceiver *dest, + bool execute_once) { TupleTableSlot *slot; uint64 current_tuple_count; @@ -1605,16 +1746,14 @@ ExecutePlan(EState *estate, estate->es_direction = direction; /* - * If a tuple count was supplied, we must force the plan to run without - * parallelism, because we might exit early. + * If the plan might potentially be executed multiple times, we must force + * it to run without parallelism, because we might exit early. Also + * disable parallelism when writing into a relation, because no database + * changes are allowed in parallel mode. */ - if (numberTuples) + if (!execute_once || dest->mydest == DestIntoRel) use_parallel_mode = false; - /* - * If a tuple count was supplied, we must force the plan to run without - * parallelism, because we might exit early. - */ if (use_parallel_mode) EnterParallelMode(); @@ -1683,7 +1822,11 @@ ExecutePlan(EState *estate, */ current_tuple_count++; if (numberTuples && numberTuples == current_tuple_count) + { + /* Allow nodes to release or shut down resources. */ + (void) ExecShutdownNode(planstate); break; + } } if (use_parallel_mode) @@ -1705,7 +1848,6 @@ ExecRelCheck(ResultRelInfo *resultRelInfo, ConstrCheck *check = rel->rd_att->constr->check; ExprContext *econtext; MemoryContext oldContext; - List *qual; int i; /* @@ -1717,13 +1859,14 @@ ExecRelCheck(ResultRelInfo *resultRelInfo, { oldContext = MemoryContextSwitchTo(estate->es_query_cxt); resultRelInfo->ri_ConstraintExprs = - (List **) palloc(ncheck * sizeof(List *)); + (ExprState **) palloc(ncheck * sizeof(ExprState *)); for (i = 0; i < ncheck; i++) { - /* ExecQual wants implicit-AND form */ - qual = make_ands_implicit(stringToNode(check[i].ccbin)); - resultRelInfo->ri_ConstraintExprs[i] = (List *) - ExecPrepareExpr((Expr *) qual, estate); + Expr *checkconstr; + + checkconstr = stringToNode(check[i].ccbin); + resultRelInfo->ri_ConstraintExprs[i] = + ExecPrepareExpr(checkconstr, estate); } MemoryContextSwitchTo(oldContext); } @@ -1740,14 +1883,14 @@ ExecRelCheck(ResultRelInfo *resultRelInfo, /* And evaluate the constraints */ for (i = 0; i < ncheck; i++) { - qual = resultRelInfo->ri_ConstraintExprs[i]; + ExprState *checkconstr = resultRelInfo->ri_ConstraintExprs[i]; /* * NOTE: SQL specifies that a NULL result from a constraint expression - * is not to be treated as a failure. Therefore, tell ExecQual to - * return TRUE for NULL. + * is not to be treated as a failure. Therefore, use ExecCheck not + * ExecQual. */ - if (!ExecQual(qual, econtext, true)) + if (!ExecCheck(checkconstr, econtext)) return check[i].ccname; } @@ -1755,6 +1898,55 @@ ExecRelCheck(ResultRelInfo *resultRelInfo, return NULL; } +/* + * ExecPartitionCheck --- check that tuple meets the partition constraint. + * + * Note: This is called *iff* resultRelInfo is the main target table. + */ +static bool +ExecPartitionCheck(ResultRelInfo *resultRelInfo, TupleTableSlot *slot, + EState *estate) +{ + ExprContext *econtext; + + /* + * If first time through, build expression state tree for the partition + * check expression. Keep it in the per-query memory context so they'll + * survive throughout the query. + */ + if (resultRelInfo->ri_PartitionCheckExpr == NULL) + { + List *qual = resultRelInfo->ri_PartitionCheck; + + resultRelInfo->ri_PartitionCheckExpr = ExecPrepareCheck(qual, estate); + } + + /* + * We will use the EState's per-tuple context for evaluating constraint + * expressions (creating it if it's not already there). + */ + econtext = GetPerTupleExprContext(estate); + + /* Arrange for econtext's scan tuple to be the tuple under test */ + econtext->ecxt_scantuple = slot; + + /* + * As in case of the catalogued constraints, we treat a NULL result as + * success here, not a failure. + */ + return ExecCheck(resultRelInfo->ri_PartitionCheckExpr, econtext); +} + +/* + * ExecConstraints - check constraints of the tuple in 'slot' + * + * This checks the traditional NOT NULL and check constraints, as well as + * the partition constraint, if any. + * + * Note: 'slot' contains the tuple to check the constraints of, which may + * have been converted from the original input tuple after tuple routing. + * 'resultRelInfo' is the original result relation, before tuple routing. + */ void ExecConstraints(ResultRelInfo *resultRelInfo, TupleTableSlot *slot, EState *estate) @@ -1766,9 +1958,9 @@ ExecConstraints(ResultRelInfo *resultRelInfo, Bitmapset *insertedCols; Bitmapset *updatedCols; - Assert(constr); + Assert(constr || resultRelInfo->ri_PartitionCheck); - if (constr->has_not_null) + if (constr && constr->has_not_null) { int natts = tupdesc->natts; int attrChk; @@ -1779,6 +1971,32 @@ ExecConstraints(ResultRelInfo *resultRelInfo, slot_attisnull(slot, attrChk)) { char *val_desc; + Relation orig_rel = rel; + TupleDesc orig_tupdesc = RelationGetDescr(rel); + + /* + * If the tuple has been routed, it's been converted to the + * partition's rowtype, which might differ from the root + * table's. We must convert it back to the root table's + * rowtype so that val_desc shown error message matches the + * input tuple. + */ + if (resultRelInfo->ri_PartitionRoot) + { + HeapTuple tuple = ExecFetchSlotTuple(slot); + TupleConversionMap *map; + + rel = resultRelInfo->ri_PartitionRoot; + tupdesc = RelationGetDescr(rel); + /* a reverse map */ + map = convert_tuples_by_name(orig_tupdesc, tupdesc, + gettext_noop("could not convert row type")); + if (map != NULL) + { + tuple = do_convert_tuple(tuple, map); + ExecStoreTuple(tuple, slot, InvalidBuffer, false); + } + } insertedCols = GetInsertedColumns(resultRelInfo, estate); updatedCols = GetUpdatedColumns(resultRelInfo, estate); @@ -1792,20 +2010,40 @@ ExecConstraints(ResultRelInfo *resultRelInfo, ereport(ERROR, (errcode(ERRCODE_NOT_NULL_VIOLATION), errmsg("null value in column \"%s\" violates not-null constraint", - NameStr(tupdesc->attrs[attrChk - 1]->attname)), + NameStr(orig_tupdesc->attrs[attrChk - 1]->attname)), val_desc ? errdetail("Failing row contains %s.", val_desc) : 0, - errtablecol(rel, attrChk))); + errtablecol(orig_rel, attrChk))); } } } - if (constr->num_check > 0) + if (constr && constr->num_check > 0) { const char *failed; if ((failed = ExecRelCheck(resultRelInfo, slot, estate)) != NULL) { char *val_desc; + Relation orig_rel = rel; + + /* See the comment above. */ + if (resultRelInfo->ri_PartitionRoot) + { + HeapTuple tuple = ExecFetchSlotTuple(slot); + TupleDesc old_tupdesc = RelationGetDescr(rel); + TupleConversionMap *map; + + rel = resultRelInfo->ri_PartitionRoot; + tupdesc = RelationGetDescr(rel); + /* a reverse map */ + map = convert_tuples_by_name(old_tupdesc, tupdesc, + gettext_noop("could not convert row type")); + if (map != NULL) + { + tuple = do_convert_tuple(tuple, map); + ExecStoreTuple(tuple, slot, InvalidBuffer, false); + } + } insertedCols = GetInsertedColumns(resultRelInfo, estate); updatedCols = GetUpdatedColumns(resultRelInfo, estate); @@ -1818,10 +2056,50 @@ ExecConstraints(ResultRelInfo *resultRelInfo, ereport(ERROR, (errcode(ERRCODE_CHECK_VIOLATION), errmsg("new row for relation \"%s\" violates check constraint \"%s\"", - RelationGetRelationName(rel), failed), + RelationGetRelationName(orig_rel), failed), val_desc ? errdetail("Failing row contains %s.", val_desc) : 0, - errtableconstraint(rel, failed))); + errtableconstraint(orig_rel, failed))); + } + } + + if (resultRelInfo->ri_PartitionCheck && + !ExecPartitionCheck(resultRelInfo, slot, estate)) + { + char *val_desc; + Relation orig_rel = rel; + + /* See the comment above. */ + if (resultRelInfo->ri_PartitionRoot) + { + HeapTuple tuple = ExecFetchSlotTuple(slot); + TupleDesc old_tupdesc = RelationGetDescr(rel); + TupleConversionMap *map; + + rel = resultRelInfo->ri_PartitionRoot; + tupdesc = RelationGetDescr(rel); + /* a reverse map */ + map = convert_tuples_by_name(old_tupdesc, tupdesc, + gettext_noop("could not convert row type")); + if (map != NULL) + { + tuple = do_convert_tuple(tuple, map); + ExecStoreTuple(tuple, slot, InvalidBuffer, false); + } } + + insertedCols = GetInsertedColumns(resultRelInfo, estate); + updatedCols = GetUpdatedColumns(resultRelInfo, estate); + modifiedCols = bms_union(insertedCols, updatedCols); + val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel), + slot, + tupdesc, + modifiedCols, + 64); + ereport(ERROR, + (errcode(ERRCODE_CHECK_VIOLATION), + errmsg("new row for relation \"%s\" violates partition constraint", + RelationGetRelationName(orig_rel)), + val_desc ? errdetail("Failing row contains %s.", val_desc) : 0)); } } @@ -1872,11 +2150,9 @@ ExecWithCheckOptions(WCOKind kind, ResultRelInfo *resultRelInfo, * is visible (in the case of a view) or that it passes the * 'with-check' policy (in the case of row security). If the qual * evaluates to NULL or FALSE, then the new tuple won't be included in - * the view or doesn't pass the 'with-check' policy for the table. We - * need ExecQual to return FALSE for NULL to handle the view case (the - * opposite of what we do above for CHECK constraints). + * the view or doesn't pass the 'with-check' policy for the table. */ - if (!ExecQual((List *) wcoExpr, econtext, false)) + if (!ExecQual(wcoExpr, econtext)) { char *val_desc; Bitmapset *modifiedCols; @@ -2977,14 +3253,7 @@ EvalPlanQualEnd(EPQState *epqstate) ExecResetTupleTable(estate->es_tupleTable, false); /* close any trigger target relations attached to this EState */ - foreach(l, estate->es_trig_target_relations) - { - ResultRelInfo *resultRelInfo = (ResultRelInfo *) lfirst(l); - - /* Close indices and then the relation itself */ - ExecCloseIndices(resultRelInfo); - heap_close(resultRelInfo->ri_RelationDesc, NoLock); - } + ExecCleanUpTriggerState(estate); MemoryContextSwitchTo(oldcontext); @@ -2995,3 +3264,242 @@ EvalPlanQualEnd(EPQState *epqstate) epqstate->planstate = NULL; epqstate->origslot = NULL; } + +/* + * ExecSetupPartitionTupleRouting - set up information needed during + * tuple routing for partitioned tables + * + * Output arguments: + * 'pd' receives an array of PartitionDispatch objects with one entry for + * every partitioned table in the partition tree + * 'partitions' receives an array of ResultRelInfo objects with one entry for + * every leaf partition in the partition tree + * 'tup_conv_maps' receives an array of TupleConversionMap objects with one + * entry for every leaf partition (required to convert input tuple based + * on the root table's rowtype to a leaf partition's rowtype after tuple + * routing is done + * 'partition_tuple_slot' receives a standalone TupleTableSlot to be used + * to manipulate any given leaf partition's rowtype after that partition + * is chosen by tuple-routing. + * 'num_parted' receives the number of partitioned tables in the partition + * tree (= the number of entries in the 'pd' output array) + * 'num_partitions' receives the number of leaf partitions in the partition + * tree (= the number of entries in the 'partitions' and 'tup_conv_maps' + * output arrays + * + * Note that all the relations in the partition tree are locked using the + * RowExclusiveLock mode upon return from this function. + */ +void +ExecSetupPartitionTupleRouting(Relation rel, + PartitionDispatch **pd, + ResultRelInfo **partitions, + TupleConversionMap ***tup_conv_maps, + TupleTableSlot **partition_tuple_slot, + int *num_parted, int *num_partitions) +{ + TupleDesc tupDesc = RelationGetDescr(rel); + List *leaf_parts; + ListCell *cell; + int i; + ResultRelInfo *leaf_part_rri; + + /* Get the tuple-routing information and lock partitions */ + *pd = RelationGetPartitionDispatchInfo(rel, RowExclusiveLock, num_parted, + &leaf_parts); + *num_partitions = list_length(leaf_parts); + *partitions = (ResultRelInfo *) palloc(*num_partitions * + sizeof(ResultRelInfo)); + *tup_conv_maps = (TupleConversionMap **) palloc0(*num_partitions * + sizeof(TupleConversionMap *)); + + /* + * Initialize an empty slot that will be used to manipulate tuples of any + * given partition's rowtype. It is attached to the caller-specified node + * (such as ModifyTableState) and released when the node finishes + * processing. + */ + *partition_tuple_slot = MakeTupleTableSlot(); + + leaf_part_rri = *partitions; + i = 0; + foreach(cell, leaf_parts) + { + Relation partrel; + TupleDesc part_tupdesc; + + /* + * We locked all the partitions above including the leaf partitions. + * Note that each of the relations in *partitions are eventually + * closed by the caller. + */ + partrel = heap_open(lfirst_oid(cell), NoLock); + part_tupdesc = RelationGetDescr(partrel); + + /* + * Verify result relation is a valid target for the current operation. + */ + CheckValidResultRel(partrel, CMD_INSERT); + + /* + * Save a tuple conversion map to convert a tuple routed to this + * partition from the parent's type to the partition's. + */ + (*tup_conv_maps)[i] = convert_tuples_by_name(tupDesc, part_tupdesc, + gettext_noop("could not convert row type")); + + InitResultRelInfo(leaf_part_rri, + partrel, + 1, /* dummy */ + rel, + 0); + + /* + * Open partition indices (remember we do not support ON CONFLICT in + * case of partitioned tables, so we do not need support information + * for speculative insertion) + */ + if (leaf_part_rri->ri_RelationDesc->rd_rel->relhasindex && + leaf_part_rri->ri_IndexRelationDescs == NULL) + ExecOpenIndices(leaf_part_rri, false); + + leaf_part_rri++; + i++; + } +} + +/* + * ExecFindPartition -- Find a leaf partition in the partition tree rooted + * at parent, for the heap tuple contained in *slot + * + * estate must be non-NULL; we'll need it to compute any expressions in the + * partition key(s) + * + * If no leaf partition is found, this routine errors out with the appropriate + * error message, else it returns the leaf partition sequence number returned + * by get_partition_for_tuple() unchanged. + */ +int +ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd, + TupleTableSlot *slot, EState *estate) +{ + int result; + PartitionDispatchData *failed_at; + TupleTableSlot *failed_slot; + + result = get_partition_for_tuple(pd, slot, estate, + &failed_at, &failed_slot); + if (result < 0) + { + Relation failed_rel; + Datum key_values[PARTITION_MAX_KEYS]; + bool key_isnull[PARTITION_MAX_KEYS]; + char *val_desc; + ExprContext *ecxt = GetPerTupleExprContext(estate); + + failed_rel = failed_at->reldesc; + ecxt->ecxt_scantuple = failed_slot; + FormPartitionKeyDatum(failed_at, failed_slot, estate, + key_values, key_isnull); + val_desc = ExecBuildSlotPartitionKeyDescription(failed_rel, + key_values, + key_isnull, + 64); + Assert(OidIsValid(RelationGetRelid(failed_rel))); + ereport(ERROR, + (errcode(ERRCODE_CHECK_VIOLATION), + errmsg("no partition of relation \"%s\" found for row", + RelationGetRelationName(failed_rel)), + val_desc ? errdetail("Partition key of the failing row contains %s.", val_desc) : 0)); + } + + return result; +} + +/* + * BuildSlotPartitionKeyDescription + * + * This works very much like BuildIndexValueDescription() and is currently + * used for building error messages when ExecFindPartition() fails to find + * partition for a row. + */ +static char * +ExecBuildSlotPartitionKeyDescription(Relation rel, + Datum *values, + bool *isnull, + int maxfieldlen) +{ + StringInfoData buf; + PartitionKey key = RelationGetPartitionKey(rel); + int partnatts = get_partition_natts(key); + int i; + Oid relid = RelationGetRelid(rel); + AclResult aclresult; + + if (check_enable_rls(relid, InvalidOid, true) == RLS_ENABLED) + return NULL; + + /* If the user has table-level access, just go build the description. */ + aclresult = pg_class_aclcheck(relid, GetUserId(), ACL_SELECT); + if (aclresult != ACLCHECK_OK) + { + /* + * Step through the columns of the partition key and make sure the + * user has SELECT rights on all of them. + */ + for (i = 0; i < partnatts; i++) + { + AttrNumber attnum = get_partition_col_attnum(key, i); + + /* + * If this partition key column is an expression, we return no + * detail rather than try to figure out what column(s) the + * expression includes and if the user has SELECT rights on them. + */ + if (attnum == InvalidAttrNumber || + pg_attribute_aclcheck(relid, attnum, GetUserId(), + ACL_SELECT) != ACLCHECK_OK) + return NULL; + } + } + + initStringInfo(&buf); + appendStringInfo(&buf, "(%s) = (", + pg_get_partkeydef_columns(relid, true)); + + for (i = 0; i < partnatts; i++) + { + char *val; + int vallen; + + if (isnull[i]) + val = "null"; + else + { + Oid foutoid; + bool typisvarlena; + + getTypeOutputInfo(get_partition_col_typid(key, i), + &foutoid, &typisvarlena); + val = OidOutputFunctionCall(foutoid, values[i]); + } + + if (i > 0) + appendStringInfoString(&buf, ", "); + + /* truncate if needed */ + vallen = strlen(val); + if (vallen <= maxfieldlen) + appendStringInfoString(&buf, val); + else + { + vallen = pg_mbcliplen(val, vallen, maxfieldlen); + appendBinaryStringInfo(&buf, val, vallen); + appendStringInfoString(&buf, "..."); + } + } + + appendStringInfoChar(&buf, ')'); + + return buf.data; +} diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c index 380d743f6c..0610180016 100644 --- a/src/backend/executor/execParallel.c +++ b/src/backend/executor/execParallel.c @@ -3,7 +3,7 @@ * execParallel.c * Support routines for parallel execution. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * This file contains routines that are intended to support setting up, @@ -25,17 +25,22 @@ #include "executor/execParallel.h" #include "executor/executor.h" +#include "executor/nodeBitmapHeapscan.h" #include "executor/nodeCustom.h" #include "executor/nodeForeignscan.h" #include "executor/nodeSeqscan.h" +#include "executor/nodeIndexscan.h" +#include "executor/nodeIndexonlyscan.h" #include "executor/tqueue.h" #include "nodes/nodeFuncs.h" #include "optimizer/planmain.h" #include "optimizer/planner.h" #include "storage/spin.h" #include "tcop/tcopprot.h" +#include "utils/dsa.h" #include "utils/memutils.h" #include "utils/snapmgr.h" +#include "pgstat.h" /* * Magic numbers for parallel executor communication. We use constants @@ -47,6 +52,8 @@ #define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xE000000000000003) #define PARALLEL_KEY_TUPLE_QUEUE UINT64CONST(0xE000000000000004) #define PARALLEL_KEY_INSTRUMENTATION UINT64CONST(0xE000000000000005) +#define PARALLEL_KEY_DSA UINT64CONST(0xE000000000000006) +#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000007) #define PARALLEL_TUPLE_QUEUE_SIZE 65536 @@ -105,8 +112,7 @@ static shm_mq_handle **ExecParallelSetupTupleQueues(ParallelContext *pcxt, static bool ExecParallelRetrieveInstrumentation(PlanState *planstate, SharedExecutorInstrumentation *instrumentation); -/* Helper functions that run in the parallel worker. */ -static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc); +/* Helper function that runs in the parallel worker. */ static DestReceiver *ExecParallelGetReceiver(dsm_segment *seg, shm_toc *toc); /* @@ -116,7 +122,7 @@ static char * ExecSerializePlan(Plan *plan, EState *estate) { PlannedStmt *pstmt; - ListCell *tlist; + ListCell *lc; /* We can't scribble on the original plan, so make a copy. */ plan = copyObject(plan); @@ -130,9 +136,9 @@ ExecSerializePlan(Plan *plan, EState *estate) * accordingly. This is sort of a hack; there might be better ways to do * this... */ - foreach(tlist, plan->targetlist) + foreach(lc, plan->targetlist) { - TargetEntry *tle = (TargetEntry *) lfirst(tlist); + TargetEntry *tle = lfirst_node(TargetEntry, lc); tle->resjunk = false; } @@ -154,13 +160,33 @@ ExecSerializePlan(Plan *plan, EState *estate) pstmt->planTree = plan; pstmt->rtable = estate->es_range_table; pstmt->resultRelations = NIL; - pstmt->utilityStmt = NULL; + pstmt->nonleafResultRelations = NIL; + + /* + * Transfer only parallel-safe subplans, leaving a NULL "hole" in the list + * for unsafe ones (so that the list indexes of the safe ones are + * preserved). This positively ensures that the worker won't try to run, + * or even do ExecInitNode on, an unsafe subplan. That's important to + * protect, eg, non-parallel-aware FDWs from getting into trouble. + */ pstmt->subplans = NIL; + foreach(lc, estate->es_plannedstmt->subplans) + { + Plan *subplan = (Plan *) lfirst(lc); + + if (subplan && !subplan->parallel_safe) + subplan = NULL; + pstmt->subplans = lappend(pstmt->subplans, subplan); + } + pstmt->rewindPlanIDs = NULL; pstmt->rowMarks = NIL; pstmt->relationOids = NIL; pstmt->invalItems = NIL; /* workers can't replan anyway... */ pstmt->nParamExec = estate->es_plannedstmt->nParamExec; + pstmt->utilityStmt = NULL; + pstmt->stmt_location = -1; + pstmt->stmt_len = -1; /* Return serialized copy of our dummy PlannedStmt. */ return nodeToString(pstmt); @@ -193,6 +219,14 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e) ExecSeqScanEstimate((SeqScanState *) planstate, e->pcxt); break; + case T_IndexScanState: + ExecIndexScanEstimate((IndexScanState *) planstate, + e->pcxt); + break; + case T_IndexOnlyScanState: + ExecIndexOnlyScanEstimate((IndexOnlyScanState *) planstate, + e->pcxt); + break; case T_ForeignScanState: ExecForeignScanEstimate((ForeignScanState *) planstate, e->pcxt); @@ -201,6 +235,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e) ExecCustomScanEstimate((CustomScanState *) planstate, e->pcxt); break; + case T_BitmapHeapScanState: + ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate, + e->pcxt); + break; default: break; } @@ -245,6 +283,14 @@ ExecParallelInitializeDSM(PlanState *planstate, ExecSeqScanInitializeDSM((SeqScanState *) planstate, d->pcxt); break; + case T_IndexScanState: + ExecIndexScanInitializeDSM((IndexScanState *) planstate, + d->pcxt); + break; + case T_IndexOnlyScanState: + ExecIndexOnlyScanInitializeDSM((IndexOnlyScanState *) planstate, + d->pcxt); + break; case T_ForeignScanState: ExecForeignScanInitializeDSM((ForeignScanState *) planstate, d->pcxt); @@ -253,6 +299,11 @@ ExecParallelInitializeDSM(PlanState *planstate, ExecCustomScanInitializeDSM((CustomScanState *) planstate, d->pcxt); break; + case T_BitmapHeapScanState: + ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate, + d->pcxt); + break; + default: break; } @@ -345,6 +396,9 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) int param_len; int instrumentation_len = 0; int instrument_offset = 0; + Size dsa_minsize = dsa_minimum_size(); + char *query_string; + int query_len; /* Allocate object for return value. */ pei = palloc0(sizeof(ParallelExecutorInfo)); @@ -355,7 +409,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) pstmt_data = ExecSerializePlan(planstate->plan, estate); /* Create a parallel context. */ - pcxt = CreateParallelContext(ParallelQueryMain, nworkers); + pcxt = CreateParallelContext("postgres", "ParallelQueryMain", nworkers); pei->pcxt = pcxt; /* @@ -364,6 +418,11 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) * for the various things we need to store. */ + /* Estimate space for query text. */ + query_len = strlen(estate->es_sourceText); + shm_toc_estimate_chunk(&pcxt->estimator, query_len); + shm_toc_estimate_keys(&pcxt->estimator, 1); + /* Estimate space for serialized PlannedStmt. */ pstmt_len = strlen(pstmt_data) + 1; shm_toc_estimate_chunk(&pcxt->estimator, pstmt_len); @@ -413,6 +472,10 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) shm_toc_estimate_keys(&pcxt->estimator, 1); } + /* Estimate space for DSA area. */ + shm_toc_estimate_chunk(&pcxt->estimator, dsa_minsize); + shm_toc_estimate_keys(&pcxt->estimator, 1); + /* Everyone's had a chance to ask for space, so now create the DSM. */ InitializeParallelDSM(pcxt); @@ -424,6 +487,11 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) * asked for has been allocated or initialized yet, though, so do that. */ + /* Store query string */ + query_string = shm_toc_allocate(pcxt->toc, query_len); + memcpy(query_string, estate->es_sourceText, query_len); + shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, query_string); + /* Store serialized PlannedStmt. */ pstmt_space = shm_toc_allocate(pcxt->toc, pstmt_len); memcpy(pstmt_space, pstmt_data, pstmt_len); @@ -467,6 +535,28 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) } /* + * Create a DSA area that can be used by the leader and all workers. + * (However, if we failed to create a DSM and are using private memory + * instead, then skip this.) + */ + if (pcxt->seg != NULL) + { + char *area_space; + + area_space = shm_toc_allocate(pcxt->toc, dsa_minsize); + shm_toc_insert(pcxt->toc, PARALLEL_KEY_DSA, area_space); + pei->area = dsa_create_in_place(area_space, dsa_minsize, + LWTRANCHE_PARALLEL_QUERY_DSA, + pcxt->seg); + } + + /* + * Make the area available to executor nodes running in the leader. See + * also ParallelQueryMain which makes it available to workers. + */ + estate->es_query_dsa = pei->area; + + /* * Give parallel-aware nodes a chance to initialize their shared data. * This also initializes the elements of instrumentation->ps_instrument, * if it exists. @@ -488,7 +578,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers) } /* - * Copy instrumentation information about this node and its descendents from + * Copy instrumentation information about this node and its descendants from * dynamic shared memory. */ static bool @@ -500,8 +590,9 @@ ExecParallelRetrieveInstrumentation(PlanState *planstate, int n; int ibytes; int plan_node_id = planstate->plan->plan_node_id; + MemoryContext oldcontext; - /* Find the instumentation for this node. */ + /* Find the instrumentation for this node. */ for (i = 0; i < instrumentation->num_plan_nodes; ++i) if (instrumentation->plan_node_id[i] == plan_node_id) break; @@ -514,10 +605,19 @@ ExecParallelRetrieveInstrumentation(PlanState *planstate, for (n = 0; n < instrumentation->num_workers; ++n) InstrAggNode(planstate->instrument, &instrument[n]); - /* Also store the per-worker detail. */ + /* + * Also store the per-worker detail. + * + * Worker instrumentation should be allocated in the same context as the + * regular instrumentation information, which is the per-query context. + * Switch into per-query memory context. + */ + oldcontext = MemoryContextSwitchTo(planstate->state->es_query_cxt); ibytes = mul_size(instrumentation->num_workers, sizeof(Instrumentation)); planstate->worker_instrument = palloc(ibytes + offsetof(WorkerInstrumentation, instrument)); + MemoryContextSwitchTo(oldcontext); + planstate->worker_instrument->num_workers = instrumentation->num_workers; memcpy(&planstate->worker_instrument->instrument, instrument, ibytes); @@ -553,7 +653,7 @@ ExecParallelFinish(ParallelExecutorInfo *pei) } /* - * Clean up whatever ParallelExecutreInfo resources still exist after + * Clean up whatever ParallelExecutorInfo resources still exist after * ExecParallelFinish. We separate these routines because someone might * want to examine the contents of the DSM after ExecParallelFinish and * before calling this routine. @@ -561,6 +661,11 @@ ExecParallelFinish(ParallelExecutorInfo *pei) void ExecParallelCleanup(ParallelExecutorInfo *pei) { + if (pei->area != NULL) + { + dsa_detach(pei->area); + pei->area = NULL; + } if (pei->pcxt != NULL) { DestroyParallelContext(pei->pcxt); @@ -597,6 +702,10 @@ ExecParallelGetQueryDesc(shm_toc *toc, DestReceiver *receiver, char *paramspace; PlannedStmt *pstmt; ParamListInfo paramLI; + char *queryString; + + /* Get the query string from shared memory */ + queryString = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT); /* Reconstruct leader-supplied PlannedStmt. */ pstmtspace = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT); @@ -615,13 +724,13 @@ ExecParallelGetQueryDesc(shm_toc *toc, DestReceiver *receiver, * revising this someday. */ return CreateQueryDesc(pstmt, - "<parallel query>", + queryString, GetActiveSnapshot(), InvalidSnapshot, - receiver, paramLI, instrument_options); + receiver, paramLI, NULL, instrument_options); } /* - * Copy instrumentation information from this node and its descendents into + * Copy instrumentation information from this node and its descendants into * dynamic shared memory, so that the parallel leader can retrieve it. */ static bool @@ -661,7 +770,7 @@ ExecParallelReportInstrumentation(PlanState *planstate, } /* - * Initialize the PlanState and its descendents with the information + * Initialize the PlanState and its descendants with the information * retrieved from shared memory. This has to be done once the PlanState * is allocated and initialized by executor; that is, after ExecutorStart(). */ @@ -679,6 +788,12 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc) case T_SeqScanState: ExecSeqScanInitializeWorker((SeqScanState *) planstate, toc); break; + case T_IndexScanState: + ExecIndexScanInitializeWorker((IndexScanState *) planstate, toc); + break; + case T_IndexOnlyScanState: + ExecIndexOnlyScanInitializeWorker((IndexOnlyScanState *) planstate, toc); + break; case T_ForeignScanState: ExecForeignScanInitializeWorker((ForeignScanState *) planstate, toc); @@ -687,6 +802,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc) ExecCustomScanInitializeWorker((CustomScanState *) planstate, toc); break; + case T_BitmapHeapScanState: + ExecBitmapHeapInitializeWorker( + (BitmapHeapScanState *) planstate, toc); + break; default: break; } @@ -698,10 +817,11 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc) /* * Main entrypoint for parallel query worker processes. * - * We reach this function from ParallelMain, so the setup necessary to create - * a sensible parallel environment has already been done; ParallelMain worries - * about stuff like the transaction state, combo CID mappings, and GUC values, - * so we don't need to deal with any of that here. + * We reach this function from ParallelWorkerMain, so the setup necessary to + * create a sensible parallel environment has already been done; + * ParallelWorkerMain worries about stuff like the transaction state, combo + * CID mappings, and GUC values, so we don't need to deal with any of that + * here. * * Our job is to deal with concerns specific to the executor. The parallel * group leader will have stored a serialized PlannedStmt, and it's our job @@ -710,7 +830,7 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc) * to do this are also stored in the dsm_segment and can be accessed through * the shm_toc. */ -static void +void ParallelQueryMain(dsm_segment *seg, shm_toc *toc) { BufferUsage *buffer_usage; @@ -718,6 +838,8 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc) QueryDesc *queryDesc; SharedExecutorInstrumentation *instrumentation; int instrument_options = 0; + void *area_space; + dsa_area *area; /* Set up DestReceiver, SharedExecutorInstrumentation, and QueryDesc. */ receiver = ExecParallelGetReceiver(seg, toc); @@ -726,13 +848,30 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc) instrument_options = instrumentation->instrument_options; queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options); + /* Setting debug_query_string for individual workers */ + debug_query_string = queryDesc->sourceText; + + /* Report workers' query for monitoring purposes */ + pgstat_report_activity(STATE_RUNNING, debug_query_string); + /* Prepare to track buffer usage during query execution. */ InstrStartParallelQuery(); - /* Start up the executor, have it run the plan, and then shut it down. */ + /* Attach to the dynamic shared memory area. */ + area_space = shm_toc_lookup(toc, PARALLEL_KEY_DSA); + area = dsa_attach_in_place(area_space, seg); + + /* Start up the executor */ ExecutorStart(queryDesc, 0); + + /* Special executor initialization steps for parallel workers */ + queryDesc->planstate->state->es_query_dsa = area; ExecParallelInitializeWorker(queryDesc->planstate, toc); - ExecutorRun(queryDesc, ForwardScanDirection, 0L); + + /* Run the plan */ + ExecutorRun(queryDesc, ForwardScanDirection, 0L, true); + + /* Shut down the executor */ ExecutorFinish(queryDesc); /* Report buffer usage during parallel execution. */ @@ -748,6 +887,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc) ExecutorEnd(queryDesc); /* Cleanup. */ + dsa_detach(area); FreeQueryDesc(queryDesc); (*receiver->rDestroy) (receiver); } diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c index fa7bdfc923..f2d9ccb130 100644 --- a/src/backend/executor/execProcnode.c +++ b/src/backend/executor/execProcnode.c @@ -8,7 +8,7 @@ * processing. * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -89,6 +89,8 @@ #include "executor/nodeCustom.h" #include "executor/nodeForeignscan.h" #include "executor/nodeFunctionscan.h" +#include "executor/nodeGather.h" +#include "executor/nodeGatherMerge.h" #include "executor/nodeGroup.h" #include "executor/nodeHash.h" #include "executor/nodeHashjoin.h" @@ -100,8 +102,9 @@ #include "executor/nodeMergeAppend.h" #include "executor/nodeMergejoin.h" #include "executor/nodeModifyTable.h" +#include "executor/nodeNamedtuplestorescan.h" #include "executor/nodeNestloop.h" -#include "executor/nodeGather.h" +#include "executor/nodeProjectSet.h" #include "executor/nodeRecursiveunion.h" #include "executor/nodeResult.h" #include "executor/nodeSamplescan.h" @@ -110,6 +113,7 @@ #include "executor/nodeSort.h" #include "executor/nodeSubplan.h" #include "executor/nodeSubqueryscan.h" +#include "executor/nodeTableFuncscan.h" #include "executor/nodeTidscan.h" #include "executor/nodeUnique.h" #include "executor/nodeValuesscan.h" @@ -158,6 +162,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags) estate, eflags); break; + case T_ProjectSet: + result = (PlanState *) ExecInitProjectSet((ProjectSet *) node, + estate, eflags); + break; + case T_ModifyTable: result = (PlanState *) ExecInitModifyTable((ModifyTable *) node, estate, eflags); @@ -236,6 +245,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags) estate, eflags); break; + case T_TableFuncScan: + result = (PlanState *) ExecInitTableFuncScan((TableFuncScan *) node, + estate, eflags); + break; + case T_ValuesScan: result = (PlanState *) ExecInitValuesScan((ValuesScan *) node, estate, eflags); @@ -246,6 +260,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags) estate, eflags); break; + case T_NamedTuplestoreScan: + result = (PlanState *) ExecInitNamedTuplestoreScan((NamedTuplestoreScan *) node, + estate, eflags); + break; + case T_WorkTableScan: result = (PlanState *) ExecInitWorkTableScan((WorkTableScan *) node, estate, eflags); @@ -317,6 +336,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags) estate, eflags); break; + case T_GatherMerge: + result = (PlanState *) ExecInitGatherMerge((GatherMerge *) node, + estate, eflags); + break; + case T_Hash: result = (PlanState *) ExecInitHash((Hash *) node, estate, eflags); @@ -492,6 +516,10 @@ ExecProcNode(PlanState *node) result = ExecResult((ResultState *) node); break; + case T_ProjectSetState: + result = ExecProjectSet((ProjectSetState *) node); + break; + case T_ModifyTableState: result = ExecModifyTable((ModifyTableState *) node); break; @@ -549,6 +577,10 @@ ExecProcNode(PlanState *node) result = ExecFunctionScan((FunctionScanState *) node); break; + case T_TableFuncScanState: + result = ExecTableFuncScan((TableFuncScanState *) node); + break; + case T_ValuesScanState: result = ExecValuesScan((ValuesScanState *) node); break; @@ -557,6 +589,10 @@ ExecProcNode(PlanState *node) result = ExecCteScan((CteScanState *) node); break; + case T_NamedTuplestoreScanState: + result = ExecNamedTuplestoreScan((NamedTuplestoreScanState *) node); + break; + case T_WorkTableScanState: result = ExecWorkTableScan((WorkTableScanState *) node); break; @@ -615,6 +651,10 @@ ExecProcNode(PlanState *node) result = ExecGather((GatherState *) node); break; + case T_GatherMergeState: + result = ExecGatherMerge((GatherMergeState *) node); + break; + case T_HashState: result = ExecHash((HashState *) node); break; @@ -745,6 +785,10 @@ ExecEndNode(PlanState *node) ExecEndResult((ResultState *) node); break; + case T_ProjectSetState: + ExecEndProjectSet((ProjectSetState *) node); + break; + case T_ModifyTableState: ExecEndModifyTable((ModifyTableState *) node); break; @@ -784,6 +828,10 @@ ExecEndNode(PlanState *node) ExecEndGather((GatherState *) node); break; + case T_GatherMergeState: + ExecEndGatherMerge((GatherMergeState *) node); + break; + case T_IndexScanState: ExecEndIndexScan((IndexScanState *) node); break; @@ -812,6 +860,10 @@ ExecEndNode(PlanState *node) ExecEndFunctionScan((FunctionScanState *) node); break; + case T_TableFuncScanState: + ExecEndTableFuncScan((TableFuncScanState *) node); + break; + case T_ValuesScanState: ExecEndValuesScan((ValuesScanState *) node); break; @@ -820,6 +872,10 @@ ExecEndNode(PlanState *node) ExecEndCteScan((CteScanState *) node); break; + case T_NamedTuplestoreScanState: + ExecEndNamedTuplestoreScan((NamedTuplestoreScanState *) node); + break; + case T_WorkTableScanState: ExecEndWorkTableScan((WorkTableScanState *) node); break; @@ -923,14 +979,25 @@ ExecShutdownNode(PlanState *node) if (node == NULL) return false; + planstate_tree_walker(node, ExecShutdownNode, NULL); + switch (nodeTag(node)) { case T_GatherState: ExecShutdownGather((GatherState *) node); break; + case T_ForeignScanState: + ExecShutdownForeignScan((ForeignScanState *) node); + break; + case T_CustomScanState: + ExecShutdownCustomScan((CustomScanState *) node); + break; + case T_GatherMergeState: + ExecShutdownGatherMerge((GatherMergeState *) node); + break; default: break; } - return planstate_tree_walker(node, ExecShutdownNode, NULL); + return false; } diff --git a/src/backend/executor/execQual.c b/src/backend/executor/execQual.c deleted file mode 100644 index 98b85af92f..0000000000 --- a/src/backend/executor/execQual.c +++ /dev/null @@ -1,5656 +0,0 @@ -/*------------------------------------------------------------------------- - * - * execQual.c - * Routines to evaluate qualification and targetlist expressions - * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group - * Portions Copyright (c) 1994, Regents of the University of California - * - * - * IDENTIFICATION - * src/backend/executor/execQual.c - * - *------------------------------------------------------------------------- - */ -/* - * INTERFACE ROUTINES - * ExecEvalExpr - (now a macro) evaluate an expression, return a datum - * ExecEvalExprSwitchContext - same, but switch into eval memory context - * ExecQual - return true/false if qualification is satisfied - * ExecProject - form a new tuple by projecting the given tuple - * - * NOTES - * The more heavily used ExecEvalExpr routines, such as ExecEvalScalarVar, - * are hotspots. Making these faster will speed up the entire system. - * - * ExecProject() is used to make tuple projections. Rather then - * trying to speed it up, the execution plan should be pre-processed - * to facilitate attribute sharing between nodes wherever possible, - * instead of doing needless copying. -cim 5/31/91 - * - * During expression evaluation, we check_stack_depth only in - * ExecMakeFunctionResult (and substitute routines) rather than at every - * single node. This is a compromise that trades off precision of the - * stack limit setting to gain speed. - */ - -#include "postgres.h" - -#include "access/htup_details.h" -#include "access/nbtree.h" -#include "access/tupconvert.h" -#include "catalog/objectaccess.h" -#include "catalog/pg_type.h" -#include "executor/execdebug.h" -#include "executor/nodeSubplan.h" -#include "funcapi.h" -#include "miscadmin.h" -#include "nodes/makefuncs.h" -#include "nodes/nodeFuncs.h" -#include "optimizer/planner.h" -#include "parser/parse_coerce.h" -#include "parser/parsetree.h" -#include "pgstat.h" -#include "utils/acl.h" -#include "utils/builtins.h" -#include "utils/lsyscache.h" -#include "utils/memutils.h" -#include "utils/typcache.h" -#include "utils/xml.h" - - -/* static function decls */ -static Datum ExecEvalArrayRef(ArrayRefExprState *astate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static bool isAssignmentIndirectionExpr(ExprState *exprstate); -static Datum ExecEvalAggref(AggrefExprState *aggref, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalWindowFunc(WindowFuncExprState *wfunc, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalScalarVarFast(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalWholeRowVar(WholeRowVarExprState *wrvstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalWholeRowFast(WholeRowVarExprState *wrvstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalWholeRowSlow(WholeRowVarExprState *wrvstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalConst(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalParamExec(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalParamExtern(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static void init_fcache(Oid foid, Oid input_collation, FuncExprState *fcache, - MemoryContext fcacheCxt, bool needDescForSets); -static void ShutdownFuncExpr(Datum arg); -static TupleDesc get_cached_rowtype(Oid type_id, int32 typmod, - TupleDesc *cache_field, ExprContext *econtext); -static void ShutdownTupleDescRef(Datum arg); -static ExprDoneCond ExecEvalFuncArgs(FunctionCallInfo fcinfo, - List *argList, ExprContext *econtext); -static void ExecPrepareTuplestoreResult(FuncExprState *fcache, - ExprContext *econtext, - Tuplestorestate *resultStore, - TupleDesc resultDesc); -static void tupledesc_match(TupleDesc dst_tupdesc, TupleDesc src_tupdesc); -static Datum ExecMakeFunctionResult(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone); -static Datum ExecMakeFunctionResultNoSets(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalFunc(FuncExprState *fcache, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalOper(FuncExprState *fcache, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalDistinct(FuncExprState *fcache, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalScalarArrayOp(ScalarArrayOpExprState *sstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalNot(BoolExprState *notclause, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalOr(BoolExprState *orExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalAnd(BoolExprState *andExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalConvertRowtype(ConvertRowtypeExprState *cstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCase(CaseExprState *caseExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCaseTestExpr(ExprState *exprstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalArray(ArrayExprState *astate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalRow(RowExprState *rstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalRowCompare(RowCompareExprState *rstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCoalesce(CoalesceExprState *coalesceExpr, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalMinMax(MinMaxExprState *minmaxExpr, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalXml(XmlExprState *xmlExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalNullIf(FuncExprState *nullIfExpr, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalNullTest(NullTestState *nstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalBooleanTest(GenericExprState *bstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCoerceToDomain(CoerceToDomainState *cstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCoerceToDomainValue(ExprState *exprstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalFieldSelect(FieldSelectState *fstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalFieldStore(FieldStoreState *fstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalRelabelType(GenericExprState *exprstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCoerceViaIO(CoerceViaIOState *iostate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalArrayCoerceExpr(ArrayCoerceExprState *astate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalCurrentOfExpr(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); -static Datum ExecEvalGroupingFuncExpr(GroupingFuncExprState *gstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone); - - -/* ---------------------------------------------------------------- - * ExecEvalExpr routines - * - * Recursively evaluate a targetlist or qualification expression. - * - * Each of the following routines having the signature - * Datum ExecEvalFoo(ExprState *expression, - * ExprContext *econtext, - * bool *isNull, - * ExprDoneCond *isDone); - * is responsible for evaluating one type or subtype of ExprState node. - * They are normally called via the ExecEvalExpr macro, which makes use of - * the function pointer set up when the ExprState node was built by - * ExecInitExpr. (In some cases, we change this pointer later to avoid - * re-executing one-time overhead.) - * - * Note: for notational simplicity we declare these functions as taking the - * specific type of ExprState that they work on. This requires casting when - * assigning the function pointer in ExecInitExpr. Be careful that the - * function signature is declared correctly, because the cast suppresses - * automatic checking! - * - * - * All these functions share this calling convention: - * - * Inputs: - * expression: the expression state tree to evaluate - * econtext: evaluation context information - * - * Outputs: - * return value: Datum value of result - * *isNull: set to TRUE if result is NULL (actual return value is - * meaningless if so); set to FALSE if non-null result - * *isDone: set to indicator of set-result status - * - * A caller that can only accept a singleton (non-set) result should pass - * NULL for isDone; if the expression computes a set result then an error - * will be reported via ereport. If the caller does pass an isDone pointer - * then *isDone is set to one of these three states: - * ExprSingleResult singleton result (not a set) - * ExprMultipleResult return value is one element of a set - * ExprEndResult there are no more elements in the set - * When ExprMultipleResult is returned, the caller should invoke - * ExecEvalExpr() repeatedly until ExprEndResult is returned. ExprEndResult - * is returned after the last real set element. For convenience isNull will - * always be set TRUE when ExprEndResult is returned, but this should not be - * taken as indicating a NULL element of the set. Note that these return - * conventions allow us to distinguish among a singleton NULL, a NULL element - * of a set, and an empty set. - * - * The caller should already have switched into the temporary memory - * context econtext->ecxt_per_tuple_memory. The convenience entry point - * ExecEvalExprSwitchContext() is provided for callers who don't prefer to - * do the switch in an outer loop. We do not do the switch in these routines - * because it'd be a waste of cycles during nested expression evaluation. - * ---------------------------------------------------------------- - */ - - -/*---------- - * ExecEvalArrayRef - * - * This function takes an ArrayRef and returns the extracted Datum - * if it's a simple reference, or the modified array value if it's - * an array assignment (i.e., array element or slice insertion). - * - * NOTE: if we get a NULL result from a subscript expression, we return NULL - * when it's an array reference, or raise an error when it's an assignment. - *---------- - */ -static Datum -ExecEvalArrayRef(ArrayRefExprState *astate, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - ArrayRef *arrayRef = (ArrayRef *) astate->xprstate.expr; - Datum array_source; - bool isAssignment = (arrayRef->refassgnexpr != NULL); - bool eisnull; - ListCell *l; - int i = 0, - j = 0; - IntArray upper, - lower; - bool upperProvided[MAXDIM], - lowerProvided[MAXDIM]; - int *lIndex; - - array_source = ExecEvalExpr(astate->refexpr, - econtext, - isNull, - isDone); - - /* - * If refexpr yields NULL, and it's a fetch, then result is NULL. In the - * assignment case, we'll cons up something below. - */ - if (*isNull) - { - if (isDone && *isDone == ExprEndResult) - return (Datum) NULL; /* end of set result */ - if (!isAssignment) - return (Datum) NULL; - } - - foreach(l, astate->refupperindexpr) - { - ExprState *eltstate = (ExprState *) lfirst(l); - - if (i >= MAXDIM) - ereport(ERROR, - (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED), - errmsg("number of array dimensions (%d) exceeds the maximum allowed (%d)", - i + 1, MAXDIM))); - - if (eltstate == NULL) - { - /* Slice bound is omitted, so use array's upper bound */ - Assert(astate->reflowerindexpr != NIL); - upperProvided[i++] = false; - continue; - } - upperProvided[i] = true; - - upper.indx[i++] = DatumGetInt32(ExecEvalExpr(eltstate, - econtext, - &eisnull, - NULL)); - /* If any index expr yields NULL, result is NULL or error */ - if (eisnull) - { - if (isAssignment) - ereport(ERROR, - (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), - errmsg("array subscript in assignment must not be null"))); - *isNull = true; - return (Datum) NULL; - } - } - - if (astate->reflowerindexpr != NIL) - { - foreach(l, astate->reflowerindexpr) - { - ExprState *eltstate = (ExprState *) lfirst(l); - - if (j >= MAXDIM) - ereport(ERROR, - (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED), - errmsg("number of array dimensions (%d) exceeds the maximum allowed (%d)", - j + 1, MAXDIM))); - - if (eltstate == NULL) - { - /* Slice bound is omitted, so use array's lower bound */ - lowerProvided[j++] = false; - continue; - } - lowerProvided[j] = true; - - lower.indx[j++] = DatumGetInt32(ExecEvalExpr(eltstate, - econtext, - &eisnull, - NULL)); - /* If any index expr yields NULL, result is NULL or error */ - if (eisnull) - { - if (isAssignment) - ereport(ERROR, - (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), - errmsg("array subscript in assignment must not be null"))); - *isNull = true; - return (Datum) NULL; - } - } - /* this can't happen unless parser messed up */ - if (i != j) - elog(ERROR, "upper and lower index lists are not same length"); - lIndex = lower.indx; - } - else - lIndex = NULL; - - if (isAssignment) - { - Datum sourceData; - Datum save_datum; - bool save_isNull; - - /* - * We might have a nested-assignment situation, in which the - * refassgnexpr is itself a FieldStore or ArrayRef that needs to - * obtain and modify the previous value of the array element or slice - * being replaced. If so, we have to extract that value from the - * array and pass it down via the econtext's caseValue. It's safe to - * reuse the CASE mechanism because there cannot be a CASE between - * here and where the value would be needed, and an array assignment - * can't be within a CASE either. (So saving and restoring the - * caseValue is just paranoia, but let's do it anyway.) - * - * Since fetching the old element might be a nontrivial expense, do it - * only if the argument appears to actually need it. - */ - save_datum = econtext->caseValue_datum; - save_isNull = econtext->caseValue_isNull; - - if (isAssignmentIndirectionExpr(astate->refassgnexpr)) - { - if (*isNull) - { - /* whole array is null, so any element or slice is too */ - econtext->caseValue_datum = (Datum) 0; - econtext->caseValue_isNull = true; - } - else if (lIndex == NULL) - { - econtext->caseValue_datum = - array_get_element(array_source, i, - upper.indx, - astate->refattrlength, - astate->refelemlength, - astate->refelembyval, - astate->refelemalign, - &econtext->caseValue_isNull); - } - else - { - econtext->caseValue_datum = - array_get_slice(array_source, i, - upper.indx, lower.indx, - upperProvided, lowerProvided, - astate->refattrlength, - astate->refelemlength, - astate->refelembyval, - astate->refelemalign); - econtext->caseValue_isNull = false; - } - } - else - { - /* argument shouldn't need caseValue, but for safety set it null */ - econtext->caseValue_datum = (Datum) 0; - econtext->caseValue_isNull = true; - } - - /* - * Evaluate the value to be assigned into the array. - */ - sourceData = ExecEvalExpr(astate->refassgnexpr, - econtext, - &eisnull, - NULL); - - econtext->caseValue_datum = save_datum; - econtext->caseValue_isNull = save_isNull; - - /* - * For an assignment to a fixed-length array type, both the original - * array and the value to be assigned into it must be non-NULL, else - * we punt and return the original array. - */ - if (astate->refattrlength > 0) /* fixed-length array? */ - if (eisnull || *isNull) - return array_source; - - /* - * For assignment to varlena arrays, we handle a NULL original array - * by substituting an empty (zero-dimensional) array; insertion of the - * new element will result in a singleton array value. It does not - * matter whether the new element is NULL. - */ - if (*isNull) - { - array_source = PointerGetDatum(construct_empty_array(arrayRef->refelemtype)); - *isNull = false; - } - - if (lIndex == NULL) - return array_set_element(array_source, i, - upper.indx, - sourceData, - eisnull, - astate->refattrlength, - astate->refelemlength, - astate->refelembyval, - astate->refelemalign); - else - return array_set_slice(array_source, i, - upper.indx, lower.indx, - upperProvided, lowerProvided, - sourceData, - eisnull, - astate->refattrlength, - astate->refelemlength, - astate->refelembyval, - astate->refelemalign); - } - - if (lIndex == NULL) - return array_get_element(array_source, i, - upper.indx, - astate->refattrlength, - astate->refelemlength, - astate->refelembyval, - astate->refelemalign, - isNull); - else - return array_get_slice(array_source, i, - upper.indx, lower.indx, - upperProvided, lowerProvided, - astate->refattrlength, - astate->refelemlength, - astate->refelembyval, - astate->refelemalign); -} - -/* - * Helper for ExecEvalArrayRef: is expr a nested FieldStore or ArrayRef - * that might need the old element value passed down? - * - * (We could use this in ExecEvalFieldStore too, but in that case passing - * the old value is so cheap there's no need.) - */ -static bool -isAssignmentIndirectionExpr(ExprState *exprstate) -{ - if (exprstate == NULL) - return false; /* just paranoia */ - if (IsA(exprstate, FieldStoreState)) - { - FieldStore *fstore = (FieldStore *) exprstate->expr; - - if (fstore->arg && IsA(fstore->arg, CaseTestExpr)) - return true; - } - else if (IsA(exprstate, ArrayRefExprState)) - { - ArrayRef *arrayRef = (ArrayRef *) exprstate->expr; - - if (arrayRef->refexpr && IsA(arrayRef->refexpr, CaseTestExpr)) - return true; - } - return false; -} - -/* ---------------------------------------------------------------- - * ExecEvalAggref - * - * Returns a Datum whose value is the value of the precomputed - * aggregate found in the given expression context. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalAggref(AggrefExprState *aggref, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - if (isDone) - *isDone = ExprSingleResult; - - if (econtext->ecxt_aggvalues == NULL) /* safety check */ - elog(ERROR, "no aggregates in this expression context"); - - *isNull = econtext->ecxt_aggnulls[aggref->aggno]; - return econtext->ecxt_aggvalues[aggref->aggno]; -} - -/* ---------------------------------------------------------------- - * ExecEvalWindowFunc - * - * Returns a Datum whose value is the value of the precomputed - * window function found in the given expression context. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalWindowFunc(WindowFuncExprState *wfunc, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - if (isDone) - *isDone = ExprSingleResult; - - if (econtext->ecxt_aggvalues == NULL) /* safety check */ - elog(ERROR, "no window functions in this expression context"); - - *isNull = econtext->ecxt_aggnulls[wfunc->wfuncno]; - return econtext->ecxt_aggvalues[wfunc->wfuncno]; -} - -/* ---------------------------------------------------------------- - * ExecEvalScalarVar - * - * Returns a Datum whose value is the value of a scalar (not whole-row) - * range variable with respect to given expression context. - * - * Note: ExecEvalScalarVar is executed only the first time through in a given - * plan; it changes the ExprState's function pointer to pass control directly - * to ExecEvalScalarVarFast after making one-time checks. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Var *variable = (Var *) exprstate->expr; - TupleTableSlot *slot; - AttrNumber attnum; - - if (isDone) - *isDone = ExprSingleResult; - - /* Get the input slot and attribute number we want */ - switch (variable->varno) - { - case INNER_VAR: /* get the tuple from the inner node */ - slot = econtext->ecxt_innertuple; - break; - - case OUTER_VAR: /* get the tuple from the outer node */ - slot = econtext->ecxt_outertuple; - break; - - /* INDEX_VAR is handled by default case */ - - default: /* get the tuple from the relation being - * scanned */ - slot = econtext->ecxt_scantuple; - break; - } - - attnum = variable->varattno; - - /* This was checked by ExecInitExpr */ - Assert(attnum != InvalidAttrNumber); - - /* - * If it's a user attribute, check validity (bogus system attnums will be - * caught inside slot_getattr). What we have to check for here is the - * possibility of an attribute having been changed in type since the plan - * tree was created. Ideally the plan will get invalidated and not - * re-used, but just in case, we keep these defenses. Fortunately it's - * sufficient to check once on the first time through. - * - * Note: we allow a reference to a dropped attribute. slot_getattr will - * force a NULL result in such cases. - * - * Note: ideally we'd check typmod as well as typid, but that seems - * impractical at the moment: in many cases the tupdesc will have been - * generated by ExecTypeFromTL(), and that can't guarantee to generate an - * accurate typmod in all cases, because some expression node types don't - * carry typmod. - */ - if (attnum > 0) - { - TupleDesc slot_tupdesc = slot->tts_tupleDescriptor; - Form_pg_attribute attr; - - if (attnum > slot_tupdesc->natts) /* should never happen */ - elog(ERROR, "attribute number %d exceeds number of columns %d", - attnum, slot_tupdesc->natts); - - attr = slot_tupdesc->attrs[attnum - 1]; - - /* can't check type if dropped, since atttypid is probably 0 */ - if (!attr->attisdropped) - { - if (variable->vartype != attr->atttypid) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("attribute %d has wrong type", attnum), - errdetail("Table has type %s, but query expects %s.", - format_type_be(attr->atttypid), - format_type_be(variable->vartype)))); - } - } - - /* Skip the checking on future executions of node */ - exprstate->evalfunc = ExecEvalScalarVarFast; - - /* Fetch the value from the slot */ - return slot_getattr(slot, attnum, isNull); -} - -/* ---------------------------------------------------------------- - * ExecEvalScalarVarFast - * - * Returns a Datum for a scalar variable. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalScalarVarFast(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Var *variable = (Var *) exprstate->expr; - TupleTableSlot *slot; - AttrNumber attnum; - - if (isDone) - *isDone = ExprSingleResult; - - /* Get the input slot and attribute number we want */ - switch (variable->varno) - { - case INNER_VAR: /* get the tuple from the inner node */ - slot = econtext->ecxt_innertuple; - break; - - case OUTER_VAR: /* get the tuple from the outer node */ - slot = econtext->ecxt_outertuple; - break; - - /* INDEX_VAR is handled by default case */ - - default: /* get the tuple from the relation being - * scanned */ - slot = econtext->ecxt_scantuple; - break; - } - - attnum = variable->varattno; - - /* Fetch the value from the slot */ - return slot_getattr(slot, attnum, isNull); -} - -/* ---------------------------------------------------------------- - * ExecEvalWholeRowVar - * - * Returns a Datum whose value is the value of a whole-row range - * variable with respect to given expression context. - * - * Note: ExecEvalWholeRowVar is executed only the first time through in a - * given plan; it changes the ExprState's function pointer to pass control - * directly to ExecEvalWholeRowFast or ExecEvalWholeRowSlow after making - * one-time checks. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalWholeRowVar(WholeRowVarExprState *wrvstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Var *variable = (Var *) wrvstate->xprstate.expr; - TupleTableSlot *slot; - TupleDesc output_tupdesc; - MemoryContext oldcontext; - bool needslow = false; - - if (isDone) - *isDone = ExprSingleResult; - - /* This was checked by ExecInitExpr */ - Assert(variable->varattno == InvalidAttrNumber); - - /* Get the input slot we want */ - switch (variable->varno) - { - case INNER_VAR: /* get the tuple from the inner node */ - slot = econtext->ecxt_innertuple; - break; - - case OUTER_VAR: /* get the tuple from the outer node */ - slot = econtext->ecxt_outertuple; - break; - - /* INDEX_VAR is handled by default case */ - - default: /* get the tuple from the relation being - * scanned */ - slot = econtext->ecxt_scantuple; - break; - } - - /* - * If the input tuple came from a subquery, it might contain "resjunk" - * columns (such as GROUP BY or ORDER BY columns), which we don't want to - * keep in the whole-row result. We can get rid of such columns by - * passing the tuple through a JunkFilter --- but to make one, we have to - * lay our hands on the subquery's targetlist. Fortunately, there are not - * very many cases where this can happen, and we can identify all of them - * by examining our parent PlanState. We assume this is not an issue in - * standalone expressions that don't have parent plans. (Whole-row Vars - * can occur in such expressions, but they will always be referencing - * table rows.) - */ - if (wrvstate->parent) - { - PlanState *subplan = NULL; - - switch (nodeTag(wrvstate->parent)) - { - case T_SubqueryScanState: - subplan = ((SubqueryScanState *) wrvstate->parent)->subplan; - break; - case T_CteScanState: - subplan = ((CteScanState *) wrvstate->parent)->cteplanstate; - break; - default: - break; - } - - if (subplan) - { - bool junk_filter_needed = false; - ListCell *tlist; - - /* Detect whether subplan tlist actually has any junk columns */ - foreach(tlist, subplan->plan->targetlist) - { - TargetEntry *tle = (TargetEntry *) lfirst(tlist); - - if (tle->resjunk) - { - junk_filter_needed = true; - break; - } - } - - /* If so, build the junkfilter in the query memory context */ - if (junk_filter_needed) - { - oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - wrvstate->wrv_junkFilter = - ExecInitJunkFilter(subplan->plan->targetlist, - ExecGetResultType(subplan)->tdhasoid, - ExecInitExtraTupleSlot(wrvstate->parent->state)); - MemoryContextSwitchTo(oldcontext); - } - } - } - - /* Apply the junkfilter if any */ - if (wrvstate->wrv_junkFilter != NULL) - slot = ExecFilterJunk(wrvstate->wrv_junkFilter, slot); - - /* - * If the Var identifies a named composite type, we must check that the - * actual tuple type is compatible with it. - */ - if (variable->vartype != RECORDOID) - { - TupleDesc var_tupdesc; - TupleDesc slot_tupdesc; - int i; - - /* - * We really only care about numbers of attributes and data types. - * Also, we can ignore type mismatch on columns that are dropped in - * the destination type, so long as (1) the physical storage matches - * or (2) the actual column value is NULL. Case (1) is helpful in - * some cases involving out-of-date cached plans, while case (2) is - * expected behavior in situations such as an INSERT into a table with - * dropped columns (the planner typically generates an INT4 NULL - * regardless of the dropped column type). If we find a dropped - * column and cannot verify that case (1) holds, we have to use - * ExecEvalWholeRowSlow to check (2) for each row. - */ - var_tupdesc = lookup_rowtype_tupdesc(variable->vartype, -1); - - slot_tupdesc = slot->tts_tupleDescriptor; - - if (var_tupdesc->natts != slot_tupdesc->natts) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("table row type and query-specified row type do not match"), - errdetail_plural("Table row contains %d attribute, but query expects %d.", - "Table row contains %d attributes, but query expects %d.", - slot_tupdesc->natts, - slot_tupdesc->natts, - var_tupdesc->natts))); - - for (i = 0; i < var_tupdesc->natts; i++) - { - Form_pg_attribute vattr = var_tupdesc->attrs[i]; - Form_pg_attribute sattr = slot_tupdesc->attrs[i]; - - if (vattr->atttypid == sattr->atttypid) - continue; /* no worries */ - if (!vattr->attisdropped) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("table row type and query-specified row type do not match"), - errdetail("Table has type %s at ordinal position %d, but query expects %s.", - format_type_be(sattr->atttypid), - i + 1, - format_type_be(vattr->atttypid)))); - - if (vattr->attlen != sattr->attlen || - vattr->attalign != sattr->attalign) - needslow = true; /* need runtime check for null */ - } - - /* - * Use the variable's declared rowtype as the descriptor for the - * output values, modulo possibly assigning new column names below. In - * particular, we *must* absorb any attisdropped markings. - */ - oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - output_tupdesc = CreateTupleDescCopy(var_tupdesc); - MemoryContextSwitchTo(oldcontext); - - ReleaseTupleDesc(var_tupdesc); - } - else - { - /* - * In the RECORD case, we use the input slot's rowtype as the - * descriptor for the output values, modulo possibly assigning new - * column names below. - */ - oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - output_tupdesc = CreateTupleDescCopy(slot->tts_tupleDescriptor); - MemoryContextSwitchTo(oldcontext); - } - - /* - * Construct a tuple descriptor for the composite values we'll produce, - * and make sure its record type is "blessed". The main reason to do this - * is to be sure that operations such as row_to_json() will see the - * desired column names when they look up the descriptor from the type - * information embedded in the composite values. - * - * We already got the correct physical datatype info above, but now we - * should try to find the source RTE and adopt its column aliases, in case - * they are different from the original rowtype's names. For example, in - * "SELECT foo(t) FROM tab t(x,y)", the first two columns in the composite - * output should be named "x" and "y" regardless of tab's column names. - * - * If we can't locate the RTE, assume the column names we've got are OK. - * (As of this writing, the only cases where we can't locate the RTE are - * in execution of trigger WHEN clauses, and then the Var will have the - * trigger's relation's rowtype, so its names are fine.) Also, if the - * creator of the RTE didn't bother to fill in an eref field, assume our - * column names are OK. (This happens in COPY, and perhaps other places.) - */ - if (econtext->ecxt_estate && - variable->varno <= list_length(econtext->ecxt_estate->es_range_table)) - { - RangeTblEntry *rte = rt_fetch(variable->varno, - econtext->ecxt_estate->es_range_table); - - if (rte->eref) - ExecTypeSetColNames(output_tupdesc, rte->eref->colnames); - } - - /* Bless the tupdesc if needed, and save it in the execution state */ - wrvstate->wrv_tupdesc = BlessTupleDesc(output_tupdesc); - - /* Skip all the above on future executions of node */ - if (needslow) - wrvstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalWholeRowSlow; - else - wrvstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalWholeRowFast; - - /* Fetch the value */ - return (*wrvstate->xprstate.evalfunc) ((ExprState *) wrvstate, econtext, - isNull, isDone); -} - -/* ---------------------------------------------------------------- - * ExecEvalWholeRowFast - * - * Returns a Datum for a whole-row variable. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalWholeRowFast(WholeRowVarExprState *wrvstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Var *variable = (Var *) wrvstate->xprstate.expr; - TupleTableSlot *slot; - HeapTupleHeader dtuple; - - if (isDone) - *isDone = ExprSingleResult; - *isNull = false; - - /* Get the input slot we want */ - switch (variable->varno) - { - case INNER_VAR: /* get the tuple from the inner node */ - slot = econtext->ecxt_innertuple; - break; - - case OUTER_VAR: /* get the tuple from the outer node */ - slot = econtext->ecxt_outertuple; - break; - - /* INDEX_VAR is handled by default case */ - - default: /* get the tuple from the relation being - * scanned */ - slot = econtext->ecxt_scantuple; - break; - } - - /* Apply the junkfilter if any */ - if (wrvstate->wrv_junkFilter != NULL) - slot = ExecFilterJunk(wrvstate->wrv_junkFilter, slot); - - /* - * Copy the slot tuple and make sure any toasted fields get detoasted. - */ - dtuple = DatumGetHeapTupleHeader(ExecFetchSlotTupleDatum(slot)); - - /* - * Label the datum with the composite type info we identified before. - */ - HeapTupleHeaderSetTypeId(dtuple, wrvstate->wrv_tupdesc->tdtypeid); - HeapTupleHeaderSetTypMod(dtuple, wrvstate->wrv_tupdesc->tdtypmod); - - return PointerGetDatum(dtuple); -} - -/* ---------------------------------------------------------------- - * ExecEvalWholeRowSlow - * - * Returns a Datum for a whole-row variable, in the "slow" case where - * we can't just copy the subplan's output. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalWholeRowSlow(WholeRowVarExprState *wrvstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Var *variable = (Var *) wrvstate->xprstate.expr; - TupleTableSlot *slot; - HeapTuple tuple; - TupleDesc tupleDesc; - TupleDesc var_tupdesc; - HeapTupleHeader dtuple; - int i; - - if (isDone) - *isDone = ExprSingleResult; - *isNull = false; - - /* Get the input slot we want */ - switch (variable->varno) - { - case INNER_VAR: /* get the tuple from the inner node */ - slot = econtext->ecxt_innertuple; - break; - - case OUTER_VAR: /* get the tuple from the outer node */ - slot = econtext->ecxt_outertuple; - break; - - /* INDEX_VAR is handled by default case */ - - default: /* get the tuple from the relation being - * scanned */ - slot = econtext->ecxt_scantuple; - break; - } - - /* Apply the junkfilter if any */ - if (wrvstate->wrv_junkFilter != NULL) - slot = ExecFilterJunk(wrvstate->wrv_junkFilter, slot); - - tuple = ExecFetchSlotTuple(slot); - tupleDesc = slot->tts_tupleDescriptor; - - /* wrv_tupdesc is a good enough representation of the Var's rowtype */ - Assert(variable->vartype != RECORDOID); - var_tupdesc = wrvstate->wrv_tupdesc; - - /* Check to see if any dropped attributes are non-null */ - for (i = 0; i < var_tupdesc->natts; i++) - { - Form_pg_attribute vattr = var_tupdesc->attrs[i]; - Form_pg_attribute sattr = tupleDesc->attrs[i]; - - if (!vattr->attisdropped) - continue; /* already checked non-dropped cols */ - if (heap_attisnull(tuple, i + 1)) - continue; /* null is always okay */ - if (vattr->attlen != sattr->attlen || - vattr->attalign != sattr->attalign) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("table row type and query-specified row type do not match"), - errdetail("Physical storage mismatch on dropped attribute at ordinal position %d.", - i + 1))); - } - - /* - * Copy the slot tuple and make sure any toasted fields get detoasted. - */ - dtuple = DatumGetHeapTupleHeader(ExecFetchSlotTupleDatum(slot)); - - /* - * Label the datum with the composite type info we identified before. - */ - HeapTupleHeaderSetTypeId(dtuple, wrvstate->wrv_tupdesc->tdtypeid); - HeapTupleHeaderSetTypMod(dtuple, wrvstate->wrv_tupdesc->tdtypmod); - - return PointerGetDatum(dtuple); -} - -/* ---------------------------------------------------------------- - * ExecEvalConst - * - * Returns the value of a constant. - * - * Note that for pass-by-ref datatypes, we return a pointer to the - * actual constant node. This is one of the reasons why functions - * must treat their input arguments as read-only. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalConst(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Const *con = (Const *) exprstate->expr; - - if (isDone) - *isDone = ExprSingleResult; - - *isNull = con->constisnull; - return con->constvalue; -} - -/* ---------------------------------------------------------------- - * ExecEvalParamExec - * - * Returns the value of a PARAM_EXEC parameter. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalParamExec(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Param *expression = (Param *) exprstate->expr; - int thisParamId = expression->paramid; - ParamExecData *prm; - - if (isDone) - *isDone = ExprSingleResult; - - /* - * PARAM_EXEC params (internal executor parameters) are stored in the - * ecxt_param_exec_vals array, and can be accessed by array index. - */ - prm = &(econtext->ecxt_param_exec_vals[thisParamId]); - if (prm->execPlan != NULL) - { - /* Parameter not evaluated yet, so go do it */ - ExecSetParamPlan(prm->execPlan, econtext); - /* ExecSetParamPlan should have processed this param... */ - Assert(prm->execPlan == NULL); - } - *isNull = prm->isnull; - return prm->value; -} - -/* ---------------------------------------------------------------- - * ExecEvalParamExtern - * - * Returns the value of a PARAM_EXTERN parameter. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalParamExtern(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Param *expression = (Param *) exprstate->expr; - int thisParamId = expression->paramid; - ParamListInfo paramInfo = econtext->ecxt_param_list_info; - - if (isDone) - *isDone = ExprSingleResult; - - /* - * PARAM_EXTERN parameters must be sought in ecxt_param_list_info. - */ - if (paramInfo && - thisParamId > 0 && thisParamId <= paramInfo->numParams) - { - ParamExternData *prm = ¶mInfo->params[thisParamId - 1]; - - /* give hook a chance in case parameter is dynamic */ - if (!OidIsValid(prm->ptype) && paramInfo->paramFetch != NULL) - (*paramInfo->paramFetch) (paramInfo, thisParamId); - - if (OidIsValid(prm->ptype)) - { - /* safety check in case hook did something unexpected */ - if (prm->ptype != expression->paramtype) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("type of parameter %d (%s) does not match that when preparing the plan (%s)", - thisParamId, - format_type_be(prm->ptype), - format_type_be(expression->paramtype)))); - - *isNull = prm->isnull; - return prm->value; - } - } - - ereport(ERROR, - (errcode(ERRCODE_UNDEFINED_OBJECT), - errmsg("no value found for parameter %d", thisParamId))); - return (Datum) 0; /* keep compiler quiet */ -} - - -/* ---------------------------------------------------------------- - * ExecEvalOper / ExecEvalFunc support routines - * ---------------------------------------------------------------- - */ - -/* - * GetAttributeByName - * GetAttributeByNum - * - * These functions return the value of the requested attribute - * out of the given tuple Datum. - * C functions which take a tuple as an argument are expected - * to use these. Ex: overpaid(EMP) might call GetAttributeByNum(). - * Note: these are actually rather slow because they do a typcache - * lookup on each call. - */ -Datum -GetAttributeByNum(HeapTupleHeader tuple, - AttrNumber attrno, - bool *isNull) -{ - Datum result; - Oid tupType; - int32 tupTypmod; - TupleDesc tupDesc; - HeapTupleData tmptup; - - if (!AttributeNumberIsValid(attrno)) - elog(ERROR, "invalid attribute number %d", attrno); - - if (isNull == NULL) - elog(ERROR, "a NULL isNull pointer was passed"); - - if (tuple == NULL) - { - /* Kinda bogus but compatible with old behavior... */ - *isNull = true; - return (Datum) 0; - } - - tupType = HeapTupleHeaderGetTypeId(tuple); - tupTypmod = HeapTupleHeaderGetTypMod(tuple); - tupDesc = lookup_rowtype_tupdesc(tupType, tupTypmod); - - /* - * heap_getattr needs a HeapTuple not a bare HeapTupleHeader. We set all - * the fields in the struct just in case user tries to inspect system - * columns. - */ - tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); - ItemPointerSetInvalid(&(tmptup.t_self)); - tmptup.t_tableOid = InvalidOid; -#ifdef PGXC - tmptup.t_xc_node_id = 0; -#endif - tmptup.t_data = tuple; - - result = heap_getattr(&tmptup, - attrno, - tupDesc, - isNull); - - ReleaseTupleDesc(tupDesc); - - return result; -} - -Datum -GetAttributeByName(HeapTupleHeader tuple, const char *attname, bool *isNull) -{ - AttrNumber attrno; - Datum result; - Oid tupType; - int32 tupTypmod; - TupleDesc tupDesc; - HeapTupleData tmptup; - int i; - - if (attname == NULL) - elog(ERROR, "invalid attribute name"); - - if (isNull == NULL) - elog(ERROR, "a NULL isNull pointer was passed"); - - if (tuple == NULL) - { - /* Kinda bogus but compatible with old behavior... */ - *isNull = true; - return (Datum) 0; - } - - tupType = HeapTupleHeaderGetTypeId(tuple); - tupTypmod = HeapTupleHeaderGetTypMod(tuple); - tupDesc = lookup_rowtype_tupdesc(tupType, tupTypmod); - - attrno = InvalidAttrNumber; - for (i = 0; i < tupDesc->natts; i++) - { - if (namestrcmp(&(tupDesc->attrs[i]->attname), attname) == 0) - { - attrno = tupDesc->attrs[i]->attnum; - break; - } - } - - if (attrno == InvalidAttrNumber) - elog(ERROR, "attribute \"%s\" does not exist", attname); - - /* - * heap_getattr needs a HeapTuple not a bare HeapTupleHeader. We set all - * the fields in the struct just in case user tries to inspect system - * columns. - */ - tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); - ItemPointerSetInvalid(&(tmptup.t_self)); - tmptup.t_tableOid = InvalidOid; -#ifdef PGXC - tmptup.t_xc_node_id = 0; -#endif - tmptup.t_data = tuple; - - result = heap_getattr(&tmptup, - attrno, - tupDesc, - isNull); - - ReleaseTupleDesc(tupDesc); - - return result; -} - -/* - * init_fcache - initialize a FuncExprState node during first use - */ -static void -init_fcache(Oid foid, Oid input_collation, FuncExprState *fcache, - MemoryContext fcacheCxt, bool needDescForSets) -{ - AclResult aclresult; - - /* Check permission to call function */ - aclresult = pg_proc_aclcheck(foid, GetUserId(), ACL_EXECUTE); - if (aclresult != ACLCHECK_OK) - aclcheck_error(aclresult, ACL_KIND_PROC, get_func_name(foid)); - InvokeFunctionExecuteHook(foid); - - /* - * Safety check on nargs. Under normal circumstances this should never - * fail, as parser should check sooner. But possibly it might fail if - * server has been compiled with FUNC_MAX_ARGS smaller than some functions - * declared in pg_proc? - */ - if (list_length(fcache->args) > FUNC_MAX_ARGS) - ereport(ERROR, - (errcode(ERRCODE_TOO_MANY_ARGUMENTS), - errmsg_plural("cannot pass more than %d argument to a function", - "cannot pass more than %d arguments to a function", - FUNC_MAX_ARGS, - FUNC_MAX_ARGS))); - - /* Set up the primary fmgr lookup information */ - fmgr_info_cxt(foid, &(fcache->func), fcacheCxt); - fmgr_info_set_expr((Node *) fcache->xprstate.expr, &(fcache->func)); - - /* Initialize the function call parameter struct as well */ - InitFunctionCallInfoData(fcache->fcinfo_data, &(fcache->func), - list_length(fcache->args), - input_collation, NULL, NULL); - - /* If function returns set, prepare expected tuple descriptor */ - if (fcache->func.fn_retset && needDescForSets) - { - TypeFuncClass functypclass; - Oid funcrettype; - TupleDesc tupdesc; - MemoryContext oldcontext; - - functypclass = get_expr_result_type(fcache->func.fn_expr, - &funcrettype, - &tupdesc); - - /* Must save tupdesc in fcache's context */ - oldcontext = MemoryContextSwitchTo(fcacheCxt); - - if (functypclass == TYPEFUNC_COMPOSITE) - { - /* Composite data type, e.g. a table's row type */ - Assert(tupdesc); - /* Must copy it out of typcache for safety */ - fcache->funcResultDesc = CreateTupleDescCopy(tupdesc); - fcache->funcReturnsTuple = true; - } - else if (functypclass == TYPEFUNC_SCALAR) - { - /* Base data type, i.e. scalar */ - tupdesc = CreateTemplateTupleDesc(1, false); - TupleDescInitEntry(tupdesc, - (AttrNumber) 1, - NULL, - funcrettype, - -1, - 0); - fcache->funcResultDesc = tupdesc; - fcache->funcReturnsTuple = false; - } - else if (functypclass == TYPEFUNC_RECORD) - { - /* This will work if function doesn't need an expectedDesc */ - fcache->funcResultDesc = NULL; - fcache->funcReturnsTuple = true; - } - else - { - /* Else, we will fail if function needs an expectedDesc */ - fcache->funcResultDesc = NULL; - } - - MemoryContextSwitchTo(oldcontext); - } - else - fcache->funcResultDesc = NULL; - - /* Initialize additional state */ - fcache->funcResultStore = NULL; - fcache->funcResultSlot = NULL; - fcache->setArgsValid = false; - fcache->shutdown_reg = false; -} - -/* - * callback function in case a FuncExpr returning a set needs to be shut down - * before it has been run to completion - */ -static void -ShutdownFuncExpr(Datum arg) -{ - FuncExprState *fcache = (FuncExprState *) DatumGetPointer(arg); - - /* If we have a slot, make sure it's let go of any tuplestore pointer */ - if (fcache->funcResultSlot) - ExecClearTuple(fcache->funcResultSlot); - - /* Release any open tuplestore */ - if (fcache->funcResultStore) - tuplestore_end(fcache->funcResultStore); - fcache->funcResultStore = NULL; - - /* Clear any active set-argument state */ - fcache->setArgsValid = false; - - /* execUtils will deregister the callback... */ - fcache->shutdown_reg = false; -} - -/* - * get_cached_rowtype: utility function to lookup a rowtype tupdesc - * - * type_id, typmod: identity of the rowtype - * cache_field: where to cache the TupleDesc pointer in expression state node - * (field must be initialized to NULL) - * econtext: expression context we are executing in - * - * NOTE: because the shutdown callback will be called during plan rescan, - * must be prepared to re-do this during any node execution; cannot call - * just once during expression initialization - */ -static TupleDesc -get_cached_rowtype(Oid type_id, int32 typmod, - TupleDesc *cache_field, ExprContext *econtext) -{ - TupleDesc tupDesc = *cache_field; - - /* Do lookup if no cached value or if requested type changed */ - if (tupDesc == NULL || - type_id != tupDesc->tdtypeid || - typmod != tupDesc->tdtypmod) - { - tupDesc = lookup_rowtype_tupdesc(type_id, typmod); - - if (*cache_field) - { - /* Release old tupdesc; but callback is already registered */ - ReleaseTupleDesc(*cache_field); - } - else - { - /* Need to register shutdown callback to release tupdesc */ - RegisterExprContextCallback(econtext, - ShutdownTupleDescRef, - PointerGetDatum(cache_field)); - } - *cache_field = tupDesc; - } - return tupDesc; -} - -/* - * Callback function to release a tupdesc refcount at expression tree shutdown - */ -static void -ShutdownTupleDescRef(Datum arg) -{ - TupleDesc *cache_field = (TupleDesc *) DatumGetPointer(arg); - - if (*cache_field) - ReleaseTupleDesc(*cache_field); - *cache_field = NULL; -} - -/* - * Evaluate arguments for a function. - */ -static ExprDoneCond -ExecEvalFuncArgs(FunctionCallInfo fcinfo, - List *argList, - ExprContext *econtext) -{ - ExprDoneCond argIsDone; - int i; - ListCell *arg; - - argIsDone = ExprSingleResult; /* default assumption */ - - i = 0; - foreach(arg, argList) - { - ExprState *argstate = (ExprState *) lfirst(arg); - ExprDoneCond thisArgIsDone; - - fcinfo->arg[i] = ExecEvalExpr(argstate, - econtext, - &fcinfo->argnull[i], - &thisArgIsDone); - - if (thisArgIsDone != ExprSingleResult) - { - /* - * We allow only one argument to have a set value; we'd need much - * more complexity to keep track of multiple set arguments (cf. - * ExecTargetList) and it doesn't seem worth it. - */ - if (argIsDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("functions and operators can take at most one set argument"))); - argIsDone = thisArgIsDone; - } - i++; - } - - Assert(i == fcinfo->nargs); - - return argIsDone; -} - -/* - * ExecPrepareTuplestoreResult - * - * Subroutine for ExecMakeFunctionResult: prepare to extract rows from a - * tuplestore function result. We must set up a funcResultSlot (unless - * already done in a previous call cycle) and verify that the function - * returned the expected tuple descriptor. - */ -static void -ExecPrepareTuplestoreResult(FuncExprState *fcache, - ExprContext *econtext, - Tuplestorestate *resultStore, - TupleDesc resultDesc) -{ - fcache->funcResultStore = resultStore; - - if (fcache->funcResultSlot == NULL) - { - /* Create a slot so we can read data out of the tuplestore */ - TupleDesc slotDesc; - MemoryContext oldcontext; - - oldcontext = MemoryContextSwitchTo(fcache->func.fn_mcxt); - - /* - * If we were not able to determine the result rowtype from context, - * and the function didn't return a tupdesc, we have to fail. - */ - if (fcache->funcResultDesc) - slotDesc = fcache->funcResultDesc; - else if (resultDesc) - { - /* don't assume resultDesc is long-lived */ - slotDesc = CreateTupleDescCopy(resultDesc); - } - else - { - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning setof record called in " - "context that cannot accept type record"))); - slotDesc = NULL; /* keep compiler quiet */ - } - - fcache->funcResultSlot = MakeSingleTupleTableSlot(slotDesc); - MemoryContextSwitchTo(oldcontext); - } - - /* - * If function provided a tupdesc, cross-check it. We only really need to - * do this for functions returning RECORD, but might as well do it always. - */ - if (resultDesc) - { - if (fcache->funcResultDesc) - tupledesc_match(fcache->funcResultDesc, resultDesc); - - /* - * If it is a dynamically-allocated TupleDesc, free it: it is - * typically allocated in a per-query context, so we must avoid - * leaking it across multiple usages. - */ - if (resultDesc->tdrefcount == -1) - FreeTupleDesc(resultDesc); - } - - /* Register cleanup callback if we didn't already */ - if (!fcache->shutdown_reg) - { - RegisterExprContextCallback(econtext, - ShutdownFuncExpr, - PointerGetDatum(fcache)); - fcache->shutdown_reg = true; - } -} - -/* - * Check that function result tuple type (src_tupdesc) matches or can - * be considered to match what the query expects (dst_tupdesc). If - * they don't match, ereport. - * - * We really only care about number of attributes and data type. - * Also, we can ignore type mismatch on columns that are dropped in the - * destination type, so long as the physical storage matches. This is - * helpful in some cases involving out-of-date cached plans. - */ -static void -tupledesc_match(TupleDesc dst_tupdesc, TupleDesc src_tupdesc) -{ - int i; - - if (dst_tupdesc->natts != src_tupdesc->natts) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("function return row and query-specified return row do not match"), - errdetail_plural("Returned row contains %d attribute, but query expects %d.", - "Returned row contains %d attributes, but query expects %d.", - src_tupdesc->natts, - src_tupdesc->natts, dst_tupdesc->natts))); - - for (i = 0; i < dst_tupdesc->natts; i++) - { - Form_pg_attribute dattr = dst_tupdesc->attrs[i]; - Form_pg_attribute sattr = src_tupdesc->attrs[i]; - - if (IsBinaryCoercible(sattr->atttypid, dattr->atttypid)) - continue; /* no worries */ - if (!dattr->attisdropped) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("function return row and query-specified return row do not match"), - errdetail("Returned type %s at ordinal position %d, but query expects %s.", - format_type_be(sattr->atttypid), - i + 1, - format_type_be(dattr->atttypid)))); - - if (dattr->attlen != sattr->attlen || - dattr->attalign != sattr->attalign) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("function return row and query-specified return row do not match"), - errdetail("Physical storage mismatch on dropped attribute at ordinal position %d.", - i + 1))); - } -} - -/* - * ExecMakeFunctionResult - * - * Evaluate the arguments to a function and then the function itself. - * init_fcache is presumed already run on the FuncExprState. - * - * This function handles the most general case, wherein the function or - * one of its arguments can return a set. - */ -static Datum -ExecMakeFunctionResult(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - List *arguments; - Datum result; - FunctionCallInfo fcinfo; - PgStat_FunctionCallUsage fcusage; - ReturnSetInfo rsinfo; /* for functions returning sets */ - ExprDoneCond argDone; - bool hasSetArg; - int i; - -restart: - - /* Guard against stack overflow due to overly complex expressions */ - check_stack_depth(); - - /* - * If a previous call of the function returned a set result in the form of - * a tuplestore, continue reading rows from the tuplestore until it's - * empty. - */ - if (fcache->funcResultStore) - { - Assert(isDone); /* it was provided before ... */ - if (tuplestore_gettupleslot(fcache->funcResultStore, true, false, - fcache->funcResultSlot)) - { - *isDone = ExprMultipleResult; - if (fcache->funcReturnsTuple) - { - /* We must return the whole tuple as a Datum. */ - *isNull = false; - return ExecFetchSlotTupleDatum(fcache->funcResultSlot); - } - else - { - /* Extract the first column and return it as a scalar. */ - return slot_getattr(fcache->funcResultSlot, 1, isNull); - } - } - /* Exhausted the tuplestore, so clean up */ - tuplestore_end(fcache->funcResultStore); - fcache->funcResultStore = NULL; - /* We are done unless there was a set-valued argument */ - if (!fcache->setHasSetArg) - { - *isDone = ExprEndResult; - *isNull = true; - return (Datum) 0; - } - /* If there was, continue evaluating the argument values */ - Assert(!fcache->setArgsValid); - } - - /* - * arguments is a list of expressions to evaluate before passing to the - * function manager. We skip the evaluation if it was already done in the - * previous call (ie, we are continuing the evaluation of a set-valued - * function). Otherwise, collect the current argument values into fcinfo. - */ - fcinfo = &fcache->fcinfo_data; - arguments = fcache->args; - if (!fcache->setArgsValid) - { - argDone = ExecEvalFuncArgs(fcinfo, arguments, econtext); - if (argDone == ExprEndResult) - { - /* input is an empty set, so return an empty set. */ - *isNull = true; - if (isDone) - *isDone = ExprEndResult; - else - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("set-valued function called in context that cannot accept a set"))); - return (Datum) 0; - } - hasSetArg = (argDone != ExprSingleResult); - } - else - { - /* Re-use callinfo from previous evaluation */ - hasSetArg = fcache->setHasSetArg; - /* Reset flag (we may set it again below) */ - fcache->setArgsValid = false; - } - - /* - * Now call the function, passing the evaluated parameter values. - */ - if (fcache->func.fn_retset || hasSetArg) - { - /* - * We need to return a set result. Complain if caller not ready to - * accept one. - */ - if (isDone == NULL) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("set-valued function called in context that cannot accept a set"))); - - /* - * Prepare a resultinfo node for communication. If the function - * doesn't itself return set, we don't pass the resultinfo to the - * function, but we need to fill it in anyway for internal use. - */ - if (fcache->func.fn_retset) - fcinfo->resultinfo = (Node *) &rsinfo; - rsinfo.type = T_ReturnSetInfo; - rsinfo.econtext = econtext; - rsinfo.expectedDesc = fcache->funcResultDesc; - rsinfo.allowedModes = (int) (SFRM_ValuePerCall | SFRM_Materialize); - /* note we do not set SFRM_Materialize_Random or _Preferred */ - rsinfo.returnMode = SFRM_ValuePerCall; - /* isDone is filled below */ - rsinfo.setResult = NULL; - rsinfo.setDesc = NULL; - - /* - * This loop handles the situation where we have both a set argument - * and a set-valued function. Once we have exhausted the function's - * value(s) for a particular argument value, we have to get the next - * argument value and start the function over again. We might have to - * do it more than once, if the function produces an empty result set - * for a particular input value. - */ - for (;;) - { - /* - * If function is strict, and there are any NULL arguments, skip - * calling the function (at least for this set of args). - */ - bool callit = true; - - if (fcache->func.fn_strict) - { - for (i = 0; i < fcinfo->nargs; i++) - { - if (fcinfo->argnull[i]) - { - callit = false; - break; - } - } - } - - if (callit) - { - pgstat_init_function_usage(fcinfo, &fcusage); - - fcinfo->isnull = false; - rsinfo.isDone = ExprSingleResult; - result = FunctionCallInvoke(fcinfo); - *isNull = fcinfo->isnull; - *isDone = rsinfo.isDone; - - pgstat_end_function_usage(&fcusage, - rsinfo.isDone != ExprMultipleResult); - } - else if (fcache->func.fn_retset) - { - /* for a strict SRF, result for NULL is an empty set */ - result = (Datum) 0; - *isNull = true; - *isDone = ExprEndResult; - } - else - { - /* for a strict non-SRF, result for NULL is a NULL */ - result = (Datum) 0; - *isNull = true; - *isDone = ExprSingleResult; - } - - /* Which protocol does function want to use? */ - if (rsinfo.returnMode == SFRM_ValuePerCall) - { - if (*isDone != ExprEndResult) - { - /* - * Got a result from current argument. If function itself - * returns set, save the current argument values to re-use - * on the next call. - */ - if (fcache->func.fn_retset && - *isDone == ExprMultipleResult) - { - fcache->setHasSetArg = hasSetArg; - fcache->setArgsValid = true; - /* Register cleanup callback if we didn't already */ - if (!fcache->shutdown_reg) - { - RegisterExprContextCallback(econtext, - ShutdownFuncExpr, - PointerGetDatum(fcache)); - fcache->shutdown_reg = true; - } - } - - /* - * Make sure we say we are returning a set, even if the - * function itself doesn't return sets. - */ - if (hasSetArg) - *isDone = ExprMultipleResult; - break; - } - } - else if (rsinfo.returnMode == SFRM_Materialize) - { - /* check we're on the same page as the function author */ - if (rsinfo.isDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), - errmsg("table-function protocol for materialize mode was not followed"))); - if (rsinfo.setResult != NULL) - { - /* prepare to return values from the tuplestore */ - ExecPrepareTuplestoreResult(fcache, econtext, - rsinfo.setResult, - rsinfo.setDesc); - /* remember whether we had set arguments */ - fcache->setHasSetArg = hasSetArg; - /* loop back to top to start returning from tuplestore */ - goto restart; - } - /* if setResult was left null, treat it as empty set */ - *isDone = ExprEndResult; - *isNull = true; - result = (Datum) 0; - } - else - ereport(ERROR, - (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), - errmsg("unrecognized table-function returnMode: %d", - (int) rsinfo.returnMode))); - - /* Else, done with this argument */ - if (!hasSetArg) - break; /* input not a set, so done */ - - /* Re-eval args to get the next element of the input set */ - argDone = ExecEvalFuncArgs(fcinfo, arguments, econtext); - - if (argDone != ExprMultipleResult) - { - /* End of argument set, so we're done. */ - *isNull = true; - *isDone = ExprEndResult; - result = (Datum) 0; - break; - } - - /* - * If we reach here, loop around to run the function on the new - * argument. - */ - } - } - else - { - /* - * Non-set case: much easier. - * - * In common cases, this code path is unreachable because we'd have - * selected ExecMakeFunctionResultNoSets instead. However, it's - * possible to get here if an argument sometimes produces set results - * and sometimes scalar results. For example, a CASE expression might - * call a set-returning function in only some of its arms. - */ - if (isDone) - *isDone = ExprSingleResult; - - /* - * If function is strict, and there are any NULL arguments, skip - * calling the function and return NULL. - */ - if (fcache->func.fn_strict) - { - for (i = 0; i < fcinfo->nargs; i++) - { - if (fcinfo->argnull[i]) - { - *isNull = true; - return (Datum) 0; - } - } - } - - pgstat_init_function_usage(fcinfo, &fcusage); - - fcinfo->isnull = false; - result = FunctionCallInvoke(fcinfo); - *isNull = fcinfo->isnull; - - pgstat_end_function_usage(&fcusage, true); - } - - return result; -} - -/* - * ExecMakeFunctionResultNoSets - * - * Simplified version of ExecMakeFunctionResult that can only handle - * non-set cases. Hand-tuned for speed. - */ -static Datum -ExecMakeFunctionResultNoSets(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - ListCell *arg; - Datum result; - FunctionCallInfo fcinfo; - PgStat_FunctionCallUsage fcusage; - int i; - - /* Guard against stack overflow due to overly complex expressions */ - check_stack_depth(); - - if (isDone) - *isDone = ExprSingleResult; - - /* inlined, simplified version of ExecEvalFuncArgs */ - fcinfo = &fcache->fcinfo_data; - i = 0; - foreach(arg, fcache->args) - { - ExprState *argstate = (ExprState *) lfirst(arg); - - fcinfo->arg[i] = ExecEvalExpr(argstate, - econtext, - &fcinfo->argnull[i], - NULL); - i++; - } - - /* - * If function is strict, and there are any NULL arguments, skip calling - * the function and return NULL. - */ - if (fcache->func.fn_strict) - { - while (--i >= 0) - { - if (fcinfo->argnull[i]) - { - *isNull = true; - return (Datum) 0; - } - } - } - - pgstat_init_function_usage(fcinfo, &fcusage); - - fcinfo->isnull = false; - result = FunctionCallInvoke(fcinfo); - *isNull = fcinfo->isnull; - - pgstat_end_function_usage(&fcusage, true); - - return result; -} - - -/* - * ExecMakeTableFunctionResult - * - * Evaluate a table function, producing a materialized result in a Tuplestore - * object. - */ -Tuplestorestate * -ExecMakeTableFunctionResult(ExprState *funcexpr, - ExprContext *econtext, - MemoryContext argContext, - TupleDesc expectedDesc, - bool randomAccess) -{ - Tuplestorestate *tupstore = NULL; - TupleDesc tupdesc = NULL; - Oid funcrettype; - bool returnsTuple; - bool returnsSet = false; - FunctionCallInfoData fcinfo; - PgStat_FunctionCallUsage fcusage; - ReturnSetInfo rsinfo; - HeapTupleData tmptup; - MemoryContext callerContext; - MemoryContext oldcontext; - bool direct_function_call; - bool first_time = true; - - callerContext = CurrentMemoryContext; - - funcrettype = exprType((Node *) funcexpr->expr); - - returnsTuple = type_is_rowtype(funcrettype); - - /* - * Prepare a resultinfo node for communication. We always do this even if - * not expecting a set result, so that we can pass expectedDesc. In the - * generic-expression case, the expression doesn't actually get to see the - * resultinfo, but set it up anyway because we use some of the fields as - * our own state variables. - */ - rsinfo.type = T_ReturnSetInfo; - rsinfo.econtext = econtext; - rsinfo.expectedDesc = expectedDesc; - rsinfo.allowedModes = (int) (SFRM_ValuePerCall | SFRM_Materialize | SFRM_Materialize_Preferred); - if (randomAccess) - rsinfo.allowedModes |= (int) SFRM_Materialize_Random; - rsinfo.returnMode = SFRM_ValuePerCall; - /* isDone is filled below */ - rsinfo.setResult = NULL; - rsinfo.setDesc = NULL; - - /* - * Normally the passed expression tree will be a FuncExprState, since the - * grammar only allows a function call at the top level of a table - * function reference. However, if the function doesn't return set then - * the planner might have replaced the function call via constant-folding - * or inlining. So if we see any other kind of expression node, execute - * it via the general ExecEvalExpr() code; the only difference is that we - * don't get a chance to pass a special ReturnSetInfo to any functions - * buried in the expression. - */ - if (funcexpr && IsA(funcexpr, FuncExprState) && - IsA(funcexpr->expr, FuncExpr)) - { - FuncExprState *fcache = (FuncExprState *) funcexpr; - ExprDoneCond argDone; - - /* - * This path is similar to ExecMakeFunctionResult. - */ - direct_function_call = true; - - /* - * Initialize function cache if first time through - */ - if (fcache->func.fn_oid == InvalidOid) - { - FuncExpr *func = (FuncExpr *) fcache->xprstate.expr; - - init_fcache(func->funcid, func->inputcollid, fcache, - econtext->ecxt_per_query_memory, false); - } - returnsSet = fcache->func.fn_retset; - InitFunctionCallInfoData(fcinfo, &(fcache->func), - list_length(fcache->args), - fcache->fcinfo_data.fncollation, - NULL, (Node *) &rsinfo); - - /* - * Evaluate the function's argument list. - * - * We can't do this in the per-tuple context: the argument values - * would disappear when we reset that context in the inner loop. And - * the caller's CurrentMemoryContext is typically a query-lifespan - * context, so we don't want to leak memory there. We require the - * caller to pass a separate memory context that can be used for this, - * and can be reset each time through to avoid bloat. - */ - MemoryContextReset(argContext); - oldcontext = MemoryContextSwitchTo(argContext); - argDone = ExecEvalFuncArgs(&fcinfo, fcache->args, econtext); - MemoryContextSwitchTo(oldcontext); - - /* We don't allow sets in the arguments of the table function */ - if (argDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("set-valued function called in context that cannot accept a set"))); - - /* - * If function is strict, and there are any NULL arguments, skip - * calling the function and act like it returned NULL (or an empty - * set, in the returns-set case). - */ - if (fcache->func.fn_strict) - { - int i; - - for (i = 0; i < fcinfo.nargs; i++) - { - if (fcinfo.argnull[i]) - goto no_function_result; - } - } - } - else - { - /* Treat funcexpr as a generic expression */ - direct_function_call = false; - InitFunctionCallInfoData(fcinfo, NULL, 0, InvalidOid, NULL, NULL); - } - - /* - * Switch to short-lived context for calling the function or expression. - */ - MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); - - /* - * Loop to handle the ValuePerCall protocol (which is also the same - * behavior needed in the generic ExecEvalExpr path). - */ - for (;;) - { - Datum result; - - CHECK_FOR_INTERRUPTS(); - - /* - * reset per-tuple memory context before each call of the function or - * expression. This cleans up any local memory the function may leak - * when called. - */ - ResetExprContext(econtext); - - /* Call the function or expression one time */ - if (direct_function_call) - { - pgstat_init_function_usage(&fcinfo, &fcusage); - - fcinfo.isnull = false; - rsinfo.isDone = ExprSingleResult; - result = FunctionCallInvoke(&fcinfo); - - pgstat_end_function_usage(&fcusage, - rsinfo.isDone != ExprMultipleResult); - } - else - { - result = ExecEvalExpr(funcexpr, econtext, - &fcinfo.isnull, &rsinfo.isDone); - } - - /* Which protocol does function want to use? */ - if (rsinfo.returnMode == SFRM_ValuePerCall) - { - /* - * Check for end of result set. - */ - if (rsinfo.isDone == ExprEndResult) - break; - - /* - * If first time through, build tuplestore for result. For a - * scalar function result type, also make a suitable tupdesc. - */ - if (first_time) - { - oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - tupstore = tuplestore_begin_heap(randomAccess, false, work_mem); - rsinfo.setResult = tupstore; - if (!returnsTuple) - { - tupdesc = CreateTemplateTupleDesc(1, false); - TupleDescInitEntry(tupdesc, - (AttrNumber) 1, - "column", - funcrettype, - -1, - 0); - rsinfo.setDesc = tupdesc; - } - MemoryContextSwitchTo(oldcontext); - } - - /* - * Store current resultset item. - */ - if (returnsTuple) - { - if (!fcinfo.isnull) - { - HeapTupleHeader td = DatumGetHeapTupleHeader(result); - - if (tupdesc == NULL) - { - /* - * This is the first non-NULL result from the - * function. Use the type info embedded in the - * rowtype Datum to look up the needed tupdesc. Make - * a copy for the query. - */ - oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - tupdesc = lookup_rowtype_tupdesc_copy(HeapTupleHeaderGetTypeId(td), - HeapTupleHeaderGetTypMod(td)); - rsinfo.setDesc = tupdesc; - MemoryContextSwitchTo(oldcontext); - } - else - { - /* - * Verify all later returned rows have same subtype; - * necessary in case the type is RECORD. - */ - if (HeapTupleHeaderGetTypeId(td) != tupdesc->tdtypeid || - HeapTupleHeaderGetTypMod(td) != tupdesc->tdtypmod) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("rows returned by function are not all of the same row type"))); - } - - /* - * tuplestore_puttuple needs a HeapTuple not a bare - * HeapTupleHeader, but it doesn't need all the fields. - */ - tmptup.t_len = HeapTupleHeaderGetDatumLength(td); - tmptup.t_data = td; - - tuplestore_puttuple(tupstore, &tmptup); - } - else - { - /* - * NULL result from a tuple-returning function; expand it - * to a row of all nulls. We rely on the expectedDesc to - * form such rows. (Note: this would be problematic if - * tuplestore_putvalues saved the tdtypeid/tdtypmod from - * the provided descriptor, since that might not match - * what we get from the function itself. But it doesn't.) - */ - int natts = expectedDesc->natts; - bool *nullflags; - - nullflags = (bool *) palloc(natts * sizeof(bool)); - memset(nullflags, true, natts * sizeof(bool)); - tuplestore_putvalues(tupstore, expectedDesc, NULL, nullflags); - } - } - else - { - /* Scalar-type case: just store the function result */ - tuplestore_putvalues(tupstore, tupdesc, &result, &fcinfo.isnull); - } - - /* - * Are we done? - */ - if (rsinfo.isDone != ExprMultipleResult) - break; - } - else if (rsinfo.returnMode == SFRM_Materialize) - { - /* check we're on the same page as the function author */ - if (!first_time || rsinfo.isDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), - errmsg("table-function protocol for materialize mode was not followed"))); - /* Done evaluating the set result */ - break; - } - else - ereport(ERROR, - (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), - errmsg("unrecognized table-function returnMode: %d", - (int) rsinfo.returnMode))); - - first_time = false; - } - -no_function_result: - - /* - * If we got nothing from the function (ie, an empty-set or NULL result), - * we have to create the tuplestore to return, and if it's a - * non-set-returning function then insert a single all-nulls row. As - * above, we depend on the expectedDesc to manufacture the dummy row. - */ - if (rsinfo.setResult == NULL) - { - MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - tupstore = tuplestore_begin_heap(randomAccess, false, work_mem); - rsinfo.setResult = tupstore; - if (!returnsSet) - { - int natts = expectedDesc->natts; - bool *nullflags; - - MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); - nullflags = (bool *) palloc(natts * sizeof(bool)); - memset(nullflags, true, natts * sizeof(bool)); - tuplestore_putvalues(tupstore, expectedDesc, NULL, nullflags); - } - } - - /* - * If function provided a tupdesc, cross-check it. We only really need to - * do this for functions returning RECORD, but might as well do it always. - */ - if (rsinfo.setDesc) - { - tupledesc_match(expectedDesc, rsinfo.setDesc); - - /* - * If it is a dynamically-allocated TupleDesc, free it: it is - * typically allocated in a per-query context, so we must avoid - * leaking it across multiple usages. - */ - if (rsinfo.setDesc->tdrefcount == -1) - FreeTupleDesc(rsinfo.setDesc); - } - - MemoryContextSwitchTo(callerContext); - - /* All done, pass back the tuplestore */ - return rsinfo.setResult; -} - - -/* ---------------------------------------------------------------- - * ExecEvalFunc - * ExecEvalOper - * - * Evaluate the functional result of a list of arguments by calling the - * function manager. - * ---------------------------------------------------------------- - */ - -/* ---------------------------------------------------------------- - * ExecEvalFunc - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalFunc(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - /* This is called only the first time through */ - FuncExpr *func = (FuncExpr *) fcache->xprstate.expr; - - /* Initialize function lookup info */ - init_fcache(func->funcid, func->inputcollid, fcache, - econtext->ecxt_per_query_memory, true); - - /* - * We need to invoke ExecMakeFunctionResult if either the function itself - * or any of its input expressions can return a set. Otherwise, invoke - * ExecMakeFunctionResultNoSets. In either case, change the evalfunc - * pointer to go directly there on subsequent uses. - */ - if (fcache->func.fn_retset || expression_returns_set((Node *) func->args)) - { - fcache->xprstate.evalfunc = (ExprStateEvalFunc) ExecMakeFunctionResult; - return ExecMakeFunctionResult(fcache, econtext, isNull, isDone); - } - else - { - fcache->xprstate.evalfunc = (ExprStateEvalFunc) ExecMakeFunctionResultNoSets; - return ExecMakeFunctionResultNoSets(fcache, econtext, isNull, isDone); - } -} - -/* ---------------------------------------------------------------- - * ExecEvalOper - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalOper(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - /* This is called only the first time through */ - OpExpr *op = (OpExpr *) fcache->xprstate.expr; - - /* Initialize function lookup info */ - init_fcache(op->opfuncid, op->inputcollid, fcache, - econtext->ecxt_per_query_memory, true); - - /* - * We need to invoke ExecMakeFunctionResult if either the function itself - * or any of its input expressions can return a set. Otherwise, invoke - * ExecMakeFunctionResultNoSets. In either case, change the evalfunc - * pointer to go directly there on subsequent uses. - */ - if (fcache->func.fn_retset || expression_returns_set((Node *) op->args)) - { - fcache->xprstate.evalfunc = (ExprStateEvalFunc) ExecMakeFunctionResult; - return ExecMakeFunctionResult(fcache, econtext, isNull, isDone); - } - else - { - fcache->xprstate.evalfunc = (ExprStateEvalFunc) ExecMakeFunctionResultNoSets; - return ExecMakeFunctionResultNoSets(fcache, econtext, isNull, isDone); - } -} - -/* ---------------------------------------------------------------- - * ExecEvalDistinct - * - * IS DISTINCT FROM must evaluate arguments to determine whether - * they are NULL; if either is NULL then the result is already - * known. If neither is NULL, then proceed to evaluate the - * function. Note that this is *always* derived from the equals - * operator, but since we need special processing of the arguments - * we can not simply reuse ExecEvalOper() or ExecEvalFunc(). - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalDistinct(FuncExprState *fcache, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - Datum result; - FunctionCallInfo fcinfo; - ExprDoneCond argDone; - - /* Set default values for result flags: non-null, not a set result */ - *isNull = false; - if (isDone) - *isDone = ExprSingleResult; - - /* - * Initialize function cache if first time through - */ - if (fcache->func.fn_oid == InvalidOid) - { - DistinctExpr *op = (DistinctExpr *) fcache->xprstate.expr; - - init_fcache(op->opfuncid, op->inputcollid, fcache, - econtext->ecxt_per_query_memory, true); - Assert(!fcache->func.fn_retset); - } - - /* - * Evaluate arguments - */ - fcinfo = &fcache->fcinfo_data; - argDone = ExecEvalFuncArgs(fcinfo, fcache->args, econtext); - if (argDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("IS DISTINCT FROM does not support set arguments"))); - Assert(fcinfo->nargs == 2); - - if (fcinfo->argnull[0] && fcinfo->argnull[1]) - { - /* Both NULL? Then is not distinct... */ - result = BoolGetDatum(FALSE); - } - else if (fcinfo->argnull[0] || fcinfo->argnull[1]) - { - /* Only one is NULL? Then is distinct... */ - result = BoolGetDatum(TRUE); - } - else - { - fcinfo->isnull = false; - result = FunctionCallInvoke(fcinfo); - *isNull = fcinfo->isnull; - /* Must invert result of "=" */ - result = BoolGetDatum(!DatumGetBool(result)); - } - - return result; -} - -/* - * ExecEvalScalarArrayOp - * - * Evaluate "scalar op ANY/ALL (array)". The operator always yields boolean, - * and we combine the results across all array elements using OR and AND - * (for ANY and ALL respectively). Of course we short-circuit as soon as - * the result is known. - */ -static Datum -ExecEvalScalarArrayOp(ScalarArrayOpExprState *sstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ScalarArrayOpExpr *opexpr = (ScalarArrayOpExpr *) sstate->fxprstate.xprstate.expr; - bool useOr = opexpr->useOr; - ArrayType *arr; - int nitems; - Datum result; - bool resultnull; - FunctionCallInfo fcinfo; - ExprDoneCond argDone; - int i; - int16 typlen; - bool typbyval; - char typalign; - char *s; - bits8 *bitmap; - int bitmask; - - /* Set default values for result flags: non-null, not a set result */ - *isNull = false; - if (isDone) - *isDone = ExprSingleResult; - - /* - * Initialize function cache if first time through - */ - if (sstate->fxprstate.func.fn_oid == InvalidOid) - { - init_fcache(opexpr->opfuncid, opexpr->inputcollid, &sstate->fxprstate, - econtext->ecxt_per_query_memory, true); - Assert(!sstate->fxprstate.func.fn_retset); - } - - /* - * Evaluate arguments - */ - fcinfo = &sstate->fxprstate.fcinfo_data; - argDone = ExecEvalFuncArgs(fcinfo, sstate->fxprstate.args, econtext); - if (argDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("op ANY/ALL (array) does not support set arguments"))); - Assert(fcinfo->nargs == 2); - - /* - * If the array is NULL then we return NULL --- it's not very meaningful - * to do anything else, even if the operator isn't strict. - */ - if (fcinfo->argnull[1]) - { - *isNull = true; - return (Datum) 0; - } - /* Else okay to fetch and detoast the array */ - arr = DatumGetArrayTypeP(fcinfo->arg[1]); - - /* - * If the array is empty, we return either FALSE or TRUE per the useOr - * flag. This is correct even if the scalar is NULL; since we would - * evaluate the operator zero times, it matters not whether it would want - * to return NULL. - */ - nitems = ArrayGetNItems(ARR_NDIM(arr), ARR_DIMS(arr)); - if (nitems <= 0) - return BoolGetDatum(!useOr); - - /* - * If the scalar is NULL, and the function is strict, return NULL; no - * point in iterating the loop. - */ - if (fcinfo->argnull[0] && sstate->fxprstate.func.fn_strict) - { - *isNull = true; - return (Datum) 0; - } - - /* - * We arrange to look up info about the element type only once per series - * of calls, assuming the element type doesn't change underneath us. - */ - if (sstate->element_type != ARR_ELEMTYPE(arr)) - { - get_typlenbyvalalign(ARR_ELEMTYPE(arr), - &sstate->typlen, - &sstate->typbyval, - &sstate->typalign); - sstate->element_type = ARR_ELEMTYPE(arr); - } - typlen = sstate->typlen; - typbyval = sstate->typbyval; - typalign = sstate->typalign; - - result = BoolGetDatum(!useOr); - resultnull = false; - - /* Loop over the array elements */ - s = (char *) ARR_DATA_PTR(arr); - bitmap = ARR_NULLBITMAP(arr); - bitmask = 1; - - for (i = 0; i < nitems; i++) - { - Datum elt; - Datum thisresult; - - /* Get array element, checking for NULL */ - if (bitmap && (*bitmap & bitmask) == 0) - { - fcinfo->arg[1] = (Datum) 0; - fcinfo->argnull[1] = true; - } - else - { - elt = fetch_att(s, typbyval, typlen); - s = att_addlength_pointer(s, typlen, s); - s = (char *) att_align_nominal(s, typalign); - fcinfo->arg[1] = elt; - fcinfo->argnull[1] = false; - } - - /* Call comparison function */ - if (fcinfo->argnull[1] && sstate->fxprstate.func.fn_strict) - { - fcinfo->isnull = true; - thisresult = (Datum) 0; - } - else - { - fcinfo->isnull = false; - thisresult = FunctionCallInvoke(fcinfo); - } - - /* Combine results per OR or AND semantics */ - if (fcinfo->isnull) - resultnull = true; - else if (useOr) - { - if (DatumGetBool(thisresult)) - { - result = BoolGetDatum(true); - resultnull = false; - break; /* needn't look at any more elements */ - } - } - else - { - if (!DatumGetBool(thisresult)) - { - result = BoolGetDatum(false); - resultnull = false; - break; /* needn't look at any more elements */ - } - } - - /* advance bitmap pointer if any */ - if (bitmap) - { - bitmask <<= 1; - if (bitmask == 0x100) - { - bitmap++; - bitmask = 1; - } - } - } - - *isNull = resultnull; - return result; -} - -/* ---------------------------------------------------------------- - * ExecEvalNot - * ExecEvalOr - * ExecEvalAnd - * - * Evaluate boolean expressions, with appropriate short-circuiting. - * - * The query planner reformulates clause expressions in the - * qualification to conjunctive normal form. If we ever get - * an AND to evaluate, we can be sure that it's not a top-level - * clause in the qualification, but appears lower (as a function - * argument, for example), or in the target list. Not that you - * need to know this, mind you... - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalNot(BoolExprState *notclause, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ExprState *clause = linitial(notclause->args); - Datum expr_value; - - if (isDone) - *isDone = ExprSingleResult; - - expr_value = ExecEvalExpr(clause, econtext, isNull, NULL); - - /* - * if the expression evaluates to null, then we just cascade the null back - * to whoever called us. - */ - if (*isNull) - return expr_value; - - /* - * evaluation of 'not' is simple.. expr is false, then return 'true' and - * vice versa. - */ - return BoolGetDatum(!DatumGetBool(expr_value)); -} - -/* ---------------------------------------------------------------- - * ExecEvalOr - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalOr(BoolExprState *orExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - List *clauses = orExpr->args; - ListCell *clause; - bool AnyNull; - - if (isDone) - *isDone = ExprSingleResult; - - AnyNull = false; - - /* - * If any of the clauses is TRUE, the OR result is TRUE regardless of the - * states of the rest of the clauses, so we can stop evaluating and return - * TRUE immediately. If none are TRUE and one or more is NULL, we return - * NULL; otherwise we return FALSE. This makes sense when you interpret - * NULL as "don't know": if we have a TRUE then the OR is TRUE even if we - * aren't sure about some of the other inputs. If all the known inputs are - * FALSE, but we have one or more "don't knows", then we have to report - * that we "don't know" what the OR's result should be --- perhaps one of - * the "don't knows" would have been TRUE if we'd known its value. Only - * when all the inputs are known to be FALSE can we state confidently that - * the OR's result is FALSE. - */ - foreach(clause, clauses) - { - ExprState *clausestate = (ExprState *) lfirst(clause); - Datum clause_value; - - clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL); - - /* - * if we have a non-null true result, then return it. - */ - if (*isNull) - AnyNull = true; /* remember we got a null */ - else if (DatumGetBool(clause_value)) - return clause_value; - } - - /* AnyNull is true if at least one clause evaluated to NULL */ - *isNull = AnyNull; - return BoolGetDatum(false); -} - -/* ---------------------------------------------------------------- - * ExecEvalAnd - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalAnd(BoolExprState *andExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - List *clauses = andExpr->args; - ListCell *clause; - bool AnyNull; - - if (isDone) - *isDone = ExprSingleResult; - - AnyNull = false; - - /* - * If any of the clauses is FALSE, the AND result is FALSE regardless of - * the states of the rest of the clauses, so we can stop evaluating and - * return FALSE immediately. If none are FALSE and one or more is NULL, - * we return NULL; otherwise we return TRUE. This makes sense when you - * interpret NULL as "don't know", using the same sort of reasoning as for - * OR, above. - */ - - foreach(clause, clauses) - { - ExprState *clausestate = (ExprState *) lfirst(clause); - Datum clause_value; - - clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL); - - /* - * if we have a non-null false result, then return it. - */ - if (*isNull) - AnyNull = true; /* remember we got a null */ - else if (!DatumGetBool(clause_value)) - return clause_value; - } - - /* AnyNull is true if at least one clause evaluated to NULL */ - *isNull = AnyNull; - return BoolGetDatum(!AnyNull); -} - -/* ---------------------------------------------------------------- - * ExecEvalConvertRowtype - * - * Evaluate a rowtype coercion operation. This may require - * rearranging field positions. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalConvertRowtype(ConvertRowtypeExprState *cstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ConvertRowtypeExpr *convert = (ConvertRowtypeExpr *) cstate->xprstate.expr; - HeapTuple result; - Datum tupDatum; - HeapTupleHeader tuple; - HeapTupleData tmptup; - - tupDatum = ExecEvalExpr(cstate->arg, econtext, isNull, isDone); - - /* this test covers the isDone exception too: */ - if (*isNull) - return tupDatum; - - tuple = DatumGetHeapTupleHeader(tupDatum); - - /* Lookup tupdescs if first time through or after rescan */ - if (cstate->indesc == NULL) - { - get_cached_rowtype(exprType((Node *) convert->arg), -1, - &cstate->indesc, econtext); - cstate->initialized = false; - } - if (cstate->outdesc == NULL) - { - get_cached_rowtype(convert->resulttype, -1, - &cstate->outdesc, econtext); - cstate->initialized = false; - } - - /* - * We used to be able to assert that incoming tuples are marked with - * exactly the rowtype of cstate->indesc. However, now that - * ExecEvalWholeRowVar might change the tuples' marking to plain RECORD - * due to inserting aliases, we can only make this weak test: - */ - Assert(HeapTupleHeaderGetTypeId(tuple) == cstate->indesc->tdtypeid || - HeapTupleHeaderGetTypeId(tuple) == RECORDOID); - - /* if first time through, initialize conversion map */ - if (!cstate->initialized) - { - MemoryContext old_cxt; - - /* allocate map in long-lived memory context */ - old_cxt = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); - - /* prepare map from old to new attribute numbers */ - cstate->map = convert_tuples_by_name(cstate->indesc, - cstate->outdesc, - gettext_noop("could not convert row type")); - cstate->initialized = true; - - MemoryContextSwitchTo(old_cxt); - } - - /* - * No-op if no conversion needed (not clear this can happen here). - */ - if (cstate->map == NULL) - return tupDatum; - - /* - * do_convert_tuple needs a HeapTuple not a bare HeapTupleHeader. - */ - tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); - tmptup.t_data = tuple; - - result = do_convert_tuple(&tmptup, cstate->map); - - return HeapTupleGetDatum(result); -} - -/* ---------------------------------------------------------------- - * ExecEvalCase - * - * Evaluate a CASE clause. Will have boolean expressions - * inside the WHEN clauses, and will have expressions - * for results. - * - thomas 1998-11-09 - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalCase(CaseExprState *caseExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - List *clauses = caseExpr->args; - ListCell *clause; - Datum save_datum; - bool save_isNull; - - if (isDone) - *isDone = ExprSingleResult; - - /* - * If there's a test expression, we have to evaluate it and save the value - * where the CaseTestExpr placeholders can find it. We must save and - * restore prior setting of econtext's caseValue fields, in case this node - * is itself within a larger CASE. Furthermore, don't assign to the - * econtext fields until after returning from evaluation of the test - * expression. We used to pass &econtext->caseValue_isNull to the - * recursive call, but that leads to aliasing that variable within said - * call, which can (and did) produce bugs when the test expression itself - * contains a CASE. - * - * If there's no test expression, we don't actually need to save and - * restore these fields; but it's less code to just do so unconditionally. - */ - save_datum = econtext->caseValue_datum; - save_isNull = econtext->caseValue_isNull; - - if (caseExpr->arg) - { - bool arg_isNull; - - econtext->caseValue_datum = ExecEvalExpr(caseExpr->arg, - econtext, - &arg_isNull, - NULL); - econtext->caseValue_isNull = arg_isNull; - } - - /* - * we evaluate each of the WHEN clauses in turn, as soon as one is true we - * return the corresponding result. If none are true then we return the - * value of the default clause, or NULL if there is none. - */ - foreach(clause, clauses) - { - CaseWhenState *wclause = lfirst(clause); - Datum clause_value; - bool clause_isNull; - - clause_value = ExecEvalExpr(wclause->expr, - econtext, - &clause_isNull, - NULL); - - /* - * if we have a true test, then we return the result, since the case - * statement is satisfied. A NULL result from the test is not - * considered true. - */ - if (DatumGetBool(clause_value) && !clause_isNull) - { - econtext->caseValue_datum = save_datum; - econtext->caseValue_isNull = save_isNull; - return ExecEvalExpr(wclause->result, - econtext, - isNull, - isDone); - } - } - - econtext->caseValue_datum = save_datum; - econtext->caseValue_isNull = save_isNull; - - if (caseExpr->defresult) - { - return ExecEvalExpr(caseExpr->defresult, - econtext, - isNull, - isDone); - } - - *isNull = true; - return (Datum) 0; -} - -/* - * ExecEvalCaseTestExpr - * - * Return the value stored by CASE. - */ -static Datum -ExecEvalCaseTestExpr(ExprState *exprstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - if (isDone) - *isDone = ExprSingleResult; - *isNull = econtext->caseValue_isNull; - return econtext->caseValue_datum; -} - -/* - * ExecEvalGroupingFuncExpr - * - * Return a bitmask with a bit for each (unevaluated) argument expression - * (rightmost arg is least significant bit). - * - * A bit is set if the corresponding expression is NOT part of the set of - * grouping expressions in the current grouping set. - */ -static Datum -ExecEvalGroupingFuncExpr(GroupingFuncExprState *gstate, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - int result = 0; - int attnum = 0; - Bitmapset *grouped_cols = gstate->aggstate->grouped_cols; - ListCell *lc; - - if (isDone) - *isDone = ExprSingleResult; - - *isNull = false; - - foreach(lc, (gstate->clauses)) - { - attnum = lfirst_int(lc); - - result = result << 1; - - if (!bms_is_member(attnum, grouped_cols)) - result = result | 1; - } - - return (Datum) result; -} - -/* ---------------------------------------------------------------- - * ExecEvalArray - ARRAY[] expressions - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalArray(ArrayExprState *astate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ArrayExpr *arrayExpr = (ArrayExpr *) astate->xprstate.expr; - ArrayType *result; - ListCell *element; - Oid element_type = arrayExpr->element_typeid; - int ndims = 0; - int dims[MAXDIM]; - int lbs[MAXDIM]; - - /* Set default values for result flags: non-null, not a set result */ - *isNull = false; - if (isDone) - *isDone = ExprSingleResult; - - if (!arrayExpr->multidims) - { - /* Elements are presumably of scalar type */ - int nelems; - Datum *dvalues; - bool *dnulls; - int i = 0; - - ndims = 1; - nelems = list_length(astate->elements); - - /* Shouldn't happen here, but if length is 0, return empty array */ - if (nelems == 0) - return PointerGetDatum(construct_empty_array(element_type)); - - dvalues = (Datum *) palloc(nelems * sizeof(Datum)); - dnulls = (bool *) palloc(nelems * sizeof(bool)); - - /* loop through and build array of datums */ - foreach(element, astate->elements) - { - ExprState *e = (ExprState *) lfirst(element); - - dvalues[i] = ExecEvalExpr(e, econtext, &dnulls[i], NULL); - i++; - } - - /* setup for 1-D array of the given length */ - dims[0] = nelems; - lbs[0] = 1; - - result = construct_md_array(dvalues, dnulls, ndims, dims, lbs, - element_type, - astate->elemlength, - astate->elembyval, - astate->elemalign); - } - else - { - /* Must be nested array expressions */ - int nbytes = 0; - int nitems = 0; - int outer_nelems = 0; - int elem_ndims = 0; - int *elem_dims = NULL; - int *elem_lbs = NULL; - bool firstone = true; - bool havenulls = false; - bool haveempty = false; - char **subdata; - bits8 **subbitmaps; - int *subbytes; - int *subnitems; - int i; - int32 dataoffset; - char *dat; - int iitem; - - i = list_length(astate->elements); - subdata = (char **) palloc(i * sizeof(char *)); - subbitmaps = (bits8 **) palloc(i * sizeof(bits8 *)); - subbytes = (int *) palloc(i * sizeof(int)); - subnitems = (int *) palloc(i * sizeof(int)); - - /* loop through and get data area from each element */ - foreach(element, astate->elements) - { - ExprState *e = (ExprState *) lfirst(element); - bool eisnull; - Datum arraydatum; - ArrayType *array; - int this_ndims; - - arraydatum = ExecEvalExpr(e, econtext, &eisnull, NULL); - /* temporarily ignore null subarrays */ - if (eisnull) - { - haveempty = true; - continue; - } - - array = DatumGetArrayTypeP(arraydatum); - - /* run-time double-check on element type */ - if (element_type != ARR_ELEMTYPE(array)) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("cannot merge incompatible arrays"), - errdetail("Array with element type %s cannot be " - "included in ARRAY construct with element type %s.", - format_type_be(ARR_ELEMTYPE(array)), - format_type_be(element_type)))); - - this_ndims = ARR_NDIM(array); - /* temporarily ignore zero-dimensional subarrays */ - if (this_ndims <= 0) - { - haveempty = true; - continue; - } - - if (firstone) - { - /* Get sub-array details from first member */ - elem_ndims = this_ndims; - ndims = elem_ndims + 1; - if (ndims <= 0 || ndims > MAXDIM) - ereport(ERROR, - (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED), - errmsg("number of array dimensions (%d) exceeds " \ - "the maximum allowed (%d)", ndims, MAXDIM))); - - elem_dims = (int *) palloc(elem_ndims * sizeof(int)); - memcpy(elem_dims, ARR_DIMS(array), elem_ndims * sizeof(int)); - elem_lbs = (int *) palloc(elem_ndims * sizeof(int)); - memcpy(elem_lbs, ARR_LBOUND(array), elem_ndims * sizeof(int)); - - firstone = false; - } - else - { - /* Check other sub-arrays are compatible */ - if (elem_ndims != this_ndims || - memcmp(elem_dims, ARR_DIMS(array), - elem_ndims * sizeof(int)) != 0 || - memcmp(elem_lbs, ARR_LBOUND(array), - elem_ndims * sizeof(int)) != 0) - ereport(ERROR, - (errcode(ERRCODE_ARRAY_SUBSCRIPT_ERROR), - errmsg("multidimensional arrays must have array " - "expressions with matching dimensions"))); - } - - subdata[outer_nelems] = ARR_DATA_PTR(array); - subbitmaps[outer_nelems] = ARR_NULLBITMAP(array); - subbytes[outer_nelems] = ARR_SIZE(array) - ARR_DATA_OFFSET(array); - nbytes += subbytes[outer_nelems]; - subnitems[outer_nelems] = ArrayGetNItems(this_ndims, - ARR_DIMS(array)); - nitems += subnitems[outer_nelems]; - havenulls |= ARR_HASNULL(array); - outer_nelems++; - } - - /* - * If all items were null or empty arrays, return an empty array; - * otherwise, if some were and some weren't, raise error. (Note: we - * must special-case this somehow to avoid trying to generate a 1-D - * array formed from empty arrays. It's not ideal...) - */ - if (haveempty) - { - if (ndims == 0) /* didn't find any nonempty array */ - return PointerGetDatum(construct_empty_array(element_type)); - ereport(ERROR, - (errcode(ERRCODE_ARRAY_SUBSCRIPT_ERROR), - errmsg("multidimensional arrays must have array " - "expressions with matching dimensions"))); - } - - /* setup for multi-D array */ - dims[0] = outer_nelems; - lbs[0] = 1; - for (i = 1; i < ndims; i++) - { - dims[i] = elem_dims[i - 1]; - lbs[i] = elem_lbs[i - 1]; - } - - if (havenulls) - { - dataoffset = ARR_OVERHEAD_WITHNULLS(ndims, nitems); - nbytes += dataoffset; - } - else - { - dataoffset = 0; /* marker for no null bitmap */ - nbytes += ARR_OVERHEAD_NONULLS(ndims); - } - - result = (ArrayType *) palloc(nbytes); - SET_VARSIZE(result, nbytes); - result->ndim = ndims; - result->dataoffset = dataoffset; - result->elemtype = element_type; - memcpy(ARR_DIMS(result), dims, ndims * sizeof(int)); - memcpy(ARR_LBOUND(result), lbs, ndims * sizeof(int)); - - dat = ARR_DATA_PTR(result); - iitem = 0; - for (i = 0; i < outer_nelems; i++) - { - memcpy(dat, subdata[i], subbytes[i]); - dat += subbytes[i]; - if (havenulls) - array_bitmap_copy(ARR_NULLBITMAP(result), iitem, - subbitmaps[i], 0, - subnitems[i]); - iitem += subnitems[i]; - } - } - - return PointerGetDatum(result); -} - -/* ---------------------------------------------------------------- - * ExecEvalRow - ROW() expressions - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalRow(RowExprState *rstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - HeapTuple tuple; - Datum *values; - bool *isnull; - int natts; - ListCell *arg; - int i; - - /* Set default values for result flags: non-null, not a set result */ - *isNull = false; - if (isDone) - *isDone = ExprSingleResult; - - /* Allocate workspace */ - natts = rstate->tupdesc->natts; - values = (Datum *) palloc0(natts * sizeof(Datum)); - isnull = (bool *) palloc(natts * sizeof(bool)); - - /* preset to nulls in case rowtype has some later-added columns */ - memset(isnull, true, natts * sizeof(bool)); - - /* Evaluate field values */ - i = 0; - foreach(arg, rstate->args) - { - ExprState *e = (ExprState *) lfirst(arg); - - values[i] = ExecEvalExpr(e, econtext, &isnull[i], NULL); - i++; - } - - tuple = heap_form_tuple(rstate->tupdesc, values, isnull); - - pfree(values); - pfree(isnull); - - return HeapTupleGetDatum(tuple); -} - -/* ---------------------------------------------------------------- - * ExecEvalRowCompare - ROW() comparison-op ROW() - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalRowCompare(RowCompareExprState *rstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - bool result; - RowCompareType rctype = ((RowCompareExpr *) rstate->xprstate.expr)->rctype; - int32 cmpresult = 0; - ListCell *l; - ListCell *r; - int i; - - if (isDone) - *isDone = ExprSingleResult; - *isNull = true; /* until we get a result */ - - i = 0; - forboth(l, rstate->largs, r, rstate->rargs) - { - ExprState *le = (ExprState *) lfirst(l); - ExprState *re = (ExprState *) lfirst(r); - FunctionCallInfoData locfcinfo; - - InitFunctionCallInfoData(locfcinfo, &(rstate->funcs[i]), 2, - rstate->collations[i], - NULL, NULL); - locfcinfo.arg[0] = ExecEvalExpr(le, econtext, - &locfcinfo.argnull[0], NULL); - locfcinfo.arg[1] = ExecEvalExpr(re, econtext, - &locfcinfo.argnull[1], NULL); - if (rstate->funcs[i].fn_strict && - (locfcinfo.argnull[0] || locfcinfo.argnull[1])) - return (Datum) 0; /* force NULL result */ - locfcinfo.isnull = false; - cmpresult = DatumGetInt32(FunctionCallInvoke(&locfcinfo)); - if (locfcinfo.isnull) - return (Datum) 0; /* force NULL result */ - if (cmpresult != 0) - break; /* no need to compare remaining columns */ - i++; - } - - switch (rctype) - { - /* EQ and NE cases aren't allowed here */ - case ROWCOMPARE_LT: - result = (cmpresult < 0); - break; - case ROWCOMPARE_LE: - result = (cmpresult <= 0); - break; - case ROWCOMPARE_GE: - result = (cmpresult >= 0); - break; - case ROWCOMPARE_GT: - result = (cmpresult > 0); - break; - default: - elog(ERROR, "unrecognized RowCompareType: %d", (int) rctype); - result = 0; /* keep compiler quiet */ - break; - } - - *isNull = false; - return BoolGetDatum(result); -} - -/* ---------------------------------------------------------------- - * ExecEvalCoalesce - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalCoalesce(CoalesceExprState *coalesceExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ListCell *arg; - - if (isDone) - *isDone = ExprSingleResult; - - /* Simply loop through until something NOT NULL is found */ - foreach(arg, coalesceExpr->args) - { - ExprState *e = (ExprState *) lfirst(arg); - Datum value; - - value = ExecEvalExpr(e, econtext, isNull, NULL); - if (!*isNull) - return value; - } - - /* Else return NULL */ - *isNull = true; - return (Datum) 0; -} - -/* ---------------------------------------------------------------- - * ExecEvalMinMax - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalMinMax(MinMaxExprState *minmaxExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Datum result = (Datum) 0; - MinMaxExpr *minmax = (MinMaxExpr *) minmaxExpr->xprstate.expr; - Oid collation = minmax->inputcollid; - MinMaxOp op = minmax->op; - FunctionCallInfoData locfcinfo; - ListCell *arg; - - if (isDone) - *isDone = ExprSingleResult; - *isNull = true; /* until we get a result */ - - InitFunctionCallInfoData(locfcinfo, &minmaxExpr->cfunc, 2, - collation, NULL, NULL); - locfcinfo.argnull[0] = false; - locfcinfo.argnull[1] = false; - - foreach(arg, minmaxExpr->args) - { - ExprState *e = (ExprState *) lfirst(arg); - Datum value; - bool valueIsNull; - int32 cmpresult; - - value = ExecEvalExpr(e, econtext, &valueIsNull, NULL); - if (valueIsNull) - continue; /* ignore NULL inputs */ - - if (*isNull) - { - /* first nonnull input, adopt value */ - result = value; - *isNull = false; - } - else - { - /* apply comparison function */ - locfcinfo.arg[0] = result; - locfcinfo.arg[1] = value; - locfcinfo.isnull = false; - cmpresult = DatumGetInt32(FunctionCallInvoke(&locfcinfo)); - if (locfcinfo.isnull) /* probably should not happen */ - continue; - if (cmpresult > 0 && op == IS_LEAST) - result = value; - else if (cmpresult < 0 && op == IS_GREATEST) - result = value; - } - } - - return result; -} - -/* ---------------------------------------------------------------- - * ExecEvalXml - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalXml(XmlExprState *xmlExpr, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - XmlExpr *xexpr = (XmlExpr *) xmlExpr->xprstate.expr; - Datum value; - bool isnull; - ListCell *arg; - ListCell *narg; - - if (isDone) - *isDone = ExprSingleResult; - *isNull = true; /* until we get a result */ - - switch (xexpr->op) - { - case IS_XMLCONCAT: - { - List *values = NIL; - - foreach(arg, xmlExpr->args) - { - ExprState *e = (ExprState *) lfirst(arg); - - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (!isnull) - values = lappend(values, DatumGetPointer(value)); - } - - if (list_length(values) > 0) - { - *isNull = false; - return PointerGetDatum(xmlconcat(values)); - } - else - return (Datum) 0; - } - break; - - case IS_XMLFOREST: - { - StringInfoData buf; - - initStringInfo(&buf); - forboth(arg, xmlExpr->named_args, narg, xexpr->arg_names) - { - ExprState *e = (ExprState *) lfirst(arg); - char *argname = strVal(lfirst(narg)); - - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (!isnull) - { - appendStringInfo(&buf, "<%s>%s</%s>", - argname, - map_sql_value_to_xml_value(value, exprType((Node *) e->expr), true), - argname); - *isNull = false; - } - } - - if (*isNull) - { - pfree(buf.data); - return (Datum) 0; - } - else - { - text *result; - - result = cstring_to_text_with_len(buf.data, buf.len); - pfree(buf.data); - - return PointerGetDatum(result); - } - } - break; - - case IS_XMLELEMENT: - *isNull = false; - return PointerGetDatum(xmlelement(xmlExpr, econtext)); - break; - - case IS_XMLPARSE: - { - ExprState *e; - text *data; - bool preserve_whitespace; - - /* arguments are known to be text, bool */ - Assert(list_length(xmlExpr->args) == 2); - - e = (ExprState *) linitial(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) - return (Datum) 0; - data = DatumGetTextP(value); - - e = (ExprState *) lsecond(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) /* probably can't happen */ - return (Datum) 0; - preserve_whitespace = DatumGetBool(value); - - *isNull = false; - - return PointerGetDatum(xmlparse(data, - xexpr->xmloption, - preserve_whitespace)); - } - break; - - case IS_XMLPI: - { - ExprState *e; - text *arg; - - /* optional argument is known to be text */ - Assert(list_length(xmlExpr->args) <= 1); - - if (xmlExpr->args) - { - e = (ExprState *) linitial(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) - arg = NULL; - else - arg = DatumGetTextP(value); - } - else - { - arg = NULL; - isnull = false; - } - - return PointerGetDatum(xmlpi(xexpr->name, arg, isnull, isNull)); - } - break; - - case IS_XMLROOT: - { - ExprState *e; - xmltype *data; - text *version; - int standalone; - - /* arguments are known to be xml, text, int */ - Assert(list_length(xmlExpr->args) == 3); - - e = (ExprState *) linitial(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) - return (Datum) 0; - data = DatumGetXmlP(value); - - e = (ExprState *) lsecond(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) - version = NULL; - else - version = DatumGetTextP(value); - - e = (ExprState *) lthird(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - standalone = DatumGetInt32(value); - - *isNull = false; - - return PointerGetDatum(xmlroot(data, - version, - standalone)); - } - break; - - case IS_XMLSERIALIZE: - { - ExprState *e; - - /* argument type is known to be xml */ - Assert(list_length(xmlExpr->args) == 1); - - e = (ExprState *) linitial(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) - return (Datum) 0; - - *isNull = false; - - return PointerGetDatum(xmltotext_with_xmloption(DatumGetXmlP(value), xexpr->xmloption)); - } - break; - - case IS_DOCUMENT: - { - ExprState *e; - - /* optional argument is known to be xml */ - Assert(list_length(xmlExpr->args) == 1); - - e = (ExprState *) linitial(xmlExpr->args); - value = ExecEvalExpr(e, econtext, &isnull, NULL); - if (isnull) - return (Datum) 0; - else - { - *isNull = false; - return BoolGetDatum(xml_is_document(DatumGetXmlP(value))); - } - } - break; - } - - elog(ERROR, "unrecognized XML operation"); - return (Datum) 0; -} - -/* ---------------------------------------------------------------- - * ExecEvalNullIf - * - * Note that this is *always* derived from the equals operator, - * but since we need special processing of the arguments - * we can not simply reuse ExecEvalOper() or ExecEvalFunc(). - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalNullIf(FuncExprState *nullIfExpr, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Datum result; - FunctionCallInfo fcinfo; - ExprDoneCond argDone; - - if (isDone) - *isDone = ExprSingleResult; - - /* - * Initialize function cache if first time through - */ - if (nullIfExpr->func.fn_oid == InvalidOid) - { - NullIfExpr *op = (NullIfExpr *) nullIfExpr->xprstate.expr; - - init_fcache(op->opfuncid, op->inputcollid, nullIfExpr, - econtext->ecxt_per_query_memory, true); - Assert(!nullIfExpr->func.fn_retset); - } - - /* - * Evaluate arguments - */ - fcinfo = &nullIfExpr->fcinfo_data; - argDone = ExecEvalFuncArgs(fcinfo, nullIfExpr->args, econtext); - if (argDone != ExprSingleResult) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("NULLIF does not support set arguments"))); - Assert(fcinfo->nargs == 2); - - /* if either argument is NULL they can't be equal */ - if (!fcinfo->argnull[0] && !fcinfo->argnull[1]) - { - fcinfo->isnull = false; - result = FunctionCallInvoke(fcinfo); - /* if the arguments are equal return null */ - if (!fcinfo->isnull && DatumGetBool(result)) - { - *isNull = true; - return (Datum) 0; - } - } - - /* else return first argument */ - *isNull = fcinfo->argnull[0]; - return fcinfo->arg[0]; -} - -/* ---------------------------------------------------------------- - * ExecEvalNullTest - * - * Evaluate a NullTest node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalNullTest(NullTestState *nstate, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - NullTest *ntest = (NullTest *) nstate->xprstate.expr; - Datum result; - - result = ExecEvalExpr(nstate->arg, econtext, isNull, isDone); - - if (isDone && *isDone == ExprEndResult) - return result; /* nothing to check */ - - if (ntest->argisrow && !(*isNull)) - { - /* - * The SQL standard defines IS [NOT] NULL for a non-null rowtype - * argument as: - * - * "R IS NULL" is true if every field is the null value. - * - * "R IS NOT NULL" is true if no field is the null value. - * - * This definition is (apparently intentionally) not recursive; so our - * tests on the fields are primitive attisnull tests, not recursive - * checks to see if they are all-nulls or no-nulls rowtypes. - * - * The standard does not consider the possibility of zero-field rows, - * but here we consider them to vacuously satisfy both predicates. - */ - HeapTupleHeader tuple; - Oid tupType; - int32 tupTypmod; - TupleDesc tupDesc; - HeapTupleData tmptup; - int att; - - tuple = DatumGetHeapTupleHeader(result); - - tupType = HeapTupleHeaderGetTypeId(tuple); - tupTypmod = HeapTupleHeaderGetTypMod(tuple); - - /* Lookup tupdesc if first time through or if type changes */ - tupDesc = get_cached_rowtype(tupType, tupTypmod, - &nstate->argdesc, econtext); - - /* - * heap_attisnull needs a HeapTuple not a bare HeapTupleHeader. - */ - tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); - tmptup.t_data = tuple; - - for (att = 1; att <= tupDesc->natts; att++) - { - /* ignore dropped columns */ - if (tupDesc->attrs[att - 1]->attisdropped) - continue; - if (heap_attisnull(&tmptup, att)) - { - /* null field disproves IS NOT NULL */ - if (ntest->nulltesttype == IS_NOT_NULL) - return BoolGetDatum(false); - } - else - { - /* non-null field disproves IS NULL */ - if (ntest->nulltesttype == IS_NULL) - return BoolGetDatum(false); - } - } - - return BoolGetDatum(true); - } - else - { - /* Simple scalar-argument case, or a null rowtype datum */ - switch (ntest->nulltesttype) - { - case IS_NULL: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(true); - } - else - return BoolGetDatum(false); - case IS_NOT_NULL: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(false); - } - else - return BoolGetDatum(true); - default: - elog(ERROR, "unrecognized nulltesttype: %d", - (int) ntest->nulltesttype); - return (Datum) 0; /* keep compiler quiet */ - } - } -} - -/* ---------------------------------------------------------------- - * ExecEvalBooleanTest - * - * Evaluate a BooleanTest node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalBooleanTest(GenericExprState *bstate, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - BooleanTest *btest = (BooleanTest *) bstate->xprstate.expr; - Datum result; - - result = ExecEvalExpr(bstate->arg, econtext, isNull, isDone); - - if (isDone && *isDone == ExprEndResult) - return result; /* nothing to check */ - - switch (btest->booltesttype) - { - case IS_TRUE: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(false); - } - else if (DatumGetBool(result)) - return BoolGetDatum(true); - else - return BoolGetDatum(false); - case IS_NOT_TRUE: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(true); - } - else if (DatumGetBool(result)) - return BoolGetDatum(false); - else - return BoolGetDatum(true); - case IS_FALSE: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(false); - } - else if (DatumGetBool(result)) - return BoolGetDatum(false); - else - return BoolGetDatum(true); - case IS_NOT_FALSE: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(true); - } - else if (DatumGetBool(result)) - return BoolGetDatum(true); - else - return BoolGetDatum(false); - case IS_UNKNOWN: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(true); - } - else - return BoolGetDatum(false); - case IS_NOT_UNKNOWN: - if (*isNull) - { - *isNull = false; - return BoolGetDatum(false); - } - else - return BoolGetDatum(true); - default: - elog(ERROR, "unrecognized booltesttype: %d", - (int) btest->booltesttype); - return (Datum) 0; /* keep compiler quiet */ - } -} - -/* - * ExecEvalCoerceToDomain - * - * Test the provided data against the domain constraint(s). If the data - * passes the constraint specifications, pass it through (return the - * datum) otherwise throw an error. - */ -static Datum -ExecEvalCoerceToDomain(CoerceToDomainState *cstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - CoerceToDomain *ctest = (CoerceToDomain *) cstate->xprstate.expr; - Datum result; - ListCell *l; - - result = ExecEvalExpr(cstate->arg, econtext, isNull, isDone); - - if (isDone && *isDone == ExprEndResult) - return result; /* nothing to check */ - - /* Make sure we have up-to-date constraints */ - UpdateDomainConstraintRef(cstate->constraint_ref); - - foreach(l, cstate->constraint_ref->constraints) - { - DomainConstraintState *con = (DomainConstraintState *) lfirst(l); - - switch (con->constrainttype) - { - case DOM_CONSTRAINT_NOTNULL: - if (*isNull) - ereport(ERROR, - (errcode(ERRCODE_NOT_NULL_VIOLATION), - errmsg("domain %s does not allow null values", - format_type_be(ctest->resulttype)), - errdatatype(ctest->resulttype))); - break; - case DOM_CONSTRAINT_CHECK: - { - Datum conResult; - bool conIsNull; - Datum save_datum; - bool save_isNull; - - /* - * Set up value to be returned by CoerceToDomainValue - * nodes. We must save and restore prior setting of - * econtext's domainValue fields, in case this node is - * itself within a check expression for another domain. - */ - save_datum = econtext->domainValue_datum; - save_isNull = econtext->domainValue_isNull; - - econtext->domainValue_datum = result; - econtext->domainValue_isNull = *isNull; - - conResult = ExecEvalExpr(con->check_expr, - econtext, &conIsNull, NULL); - - if (!conIsNull && - !DatumGetBool(conResult)) - ereport(ERROR, - (errcode(ERRCODE_CHECK_VIOLATION), - errmsg("value for domain %s violates check constraint \"%s\"", - format_type_be(ctest->resulttype), - con->name), - errdomainconstraint(ctest->resulttype, - con->name))); - econtext->domainValue_datum = save_datum; - econtext->domainValue_isNull = save_isNull; - - break; - } - default: - elog(ERROR, "unrecognized constraint type: %d", - (int) con->constrainttype); - break; - } - } - - /* If all has gone well (constraints did not fail) return the datum */ - return result; -} - -/* - * ExecEvalCoerceToDomainValue - * - * Return the value stored by CoerceToDomain. - */ -static Datum -ExecEvalCoerceToDomainValue(ExprState *exprstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - if (isDone) - *isDone = ExprSingleResult; - *isNull = econtext->domainValue_isNull; - return econtext->domainValue_datum; -} - -/* ---------------------------------------------------------------- - * ExecEvalFieldSelect - * - * Evaluate a FieldSelect node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalFieldSelect(FieldSelectState *fstate, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - FieldSelect *fselect = (FieldSelect *) fstate->xprstate.expr; - AttrNumber fieldnum = fselect->fieldnum; - Datum result; - Datum tupDatum; - HeapTupleHeader tuple; - Oid tupType; - int32 tupTypmod; - TupleDesc tupDesc; - Form_pg_attribute attr; - HeapTupleData tmptup; - - tupDatum = ExecEvalExpr(fstate->arg, econtext, isNull, isDone); - - /* this test covers the isDone exception too: */ - if (*isNull) - return tupDatum; - - tuple = DatumGetHeapTupleHeader(tupDatum); - - tupType = HeapTupleHeaderGetTypeId(tuple); - tupTypmod = HeapTupleHeaderGetTypMod(tuple); - - /* Lookup tupdesc if first time through or if type changes */ - tupDesc = get_cached_rowtype(tupType, tupTypmod, - &fstate->argdesc, econtext); - - /* - * Find field's attr record. Note we don't support system columns here: a - * datum tuple doesn't have valid values for most of the interesting - * system columns anyway. - */ - if (fieldnum <= 0) /* should never happen */ - elog(ERROR, "unsupported reference to system column %d in FieldSelect", - fieldnum); - if (fieldnum > tupDesc->natts) /* should never happen */ - elog(ERROR, "attribute number %d exceeds number of columns %d", - fieldnum, tupDesc->natts); - attr = tupDesc->attrs[fieldnum - 1]; - - /* Check for dropped column, and force a NULL result if so */ - if (attr->attisdropped) - { - *isNull = true; - return (Datum) 0; - } - - /* Check for type mismatch --- possible after ALTER COLUMN TYPE? */ - /* As in ExecEvalScalarVar, we should but can't check typmod */ - if (fselect->resulttype != attr->atttypid) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("attribute %d has wrong type", fieldnum), - errdetail("Table has type %s, but query expects %s.", - format_type_be(attr->atttypid), - format_type_be(fselect->resulttype)))); - - /* heap_getattr needs a HeapTuple not a bare HeapTupleHeader */ - tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); - tmptup.t_data = tuple; - - result = heap_getattr(&tmptup, - fieldnum, - tupDesc, - isNull); - return result; -} - -/* ---------------------------------------------------------------- - * ExecEvalFieldStore - * - * Evaluate a FieldStore node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalFieldStore(FieldStoreState *fstate, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - FieldStore *fstore = (FieldStore *) fstate->xprstate.expr; - HeapTuple tuple; - Datum tupDatum; - TupleDesc tupDesc; - Datum *values; - bool *isnull; - Datum save_datum; - bool save_isNull; - ListCell *l1, - *l2; - - tupDatum = ExecEvalExpr(fstate->arg, econtext, isNull, isDone); - - if (isDone && *isDone == ExprEndResult) - return tupDatum; - - /* Lookup tupdesc if first time through or after rescan */ - tupDesc = get_cached_rowtype(fstore->resulttype, -1, - &fstate->argdesc, econtext); - - /* Allocate workspace */ - values = (Datum *) palloc(tupDesc->natts * sizeof(Datum)); - isnull = (bool *) palloc(tupDesc->natts * sizeof(bool)); - - if (!*isNull) - { - /* - * heap_deform_tuple needs a HeapTuple not a bare HeapTupleHeader. We - * set all the fields in the struct just in case. - */ - HeapTupleHeader tuphdr; - HeapTupleData tmptup; - - tuphdr = DatumGetHeapTupleHeader(tupDatum); - tmptup.t_len = HeapTupleHeaderGetDatumLength(tuphdr); - ItemPointerSetInvalid(&(tmptup.t_self)); - tmptup.t_tableOid = InvalidOid; -#ifdef PGXC - tmptup.t_xc_node_id = 0; -#endif - tmptup.t_data = tuphdr; - - heap_deform_tuple(&tmptup, tupDesc, values, isnull); - } - else - { - /* Convert null input tuple into an all-nulls row */ - memset(isnull, true, tupDesc->natts * sizeof(bool)); - } - - /* Result is never null */ - *isNull = false; - - save_datum = econtext->caseValue_datum; - save_isNull = econtext->caseValue_isNull; - - forboth(l1, fstate->newvals, l2, fstore->fieldnums) - { - ExprState *newval = (ExprState *) lfirst(l1); - AttrNumber fieldnum = lfirst_int(l2); - - Assert(fieldnum > 0 && fieldnum <= tupDesc->natts); - - /* - * Use the CaseTestExpr mechanism to pass down the old value of the - * field being replaced; this is needed in case the newval is itself a - * FieldStore or ArrayRef that has to obtain and modify the old value. - * It's safe to reuse the CASE mechanism because there cannot be a - * CASE between here and where the value would be needed, and a field - * assignment can't be within a CASE either. (So saving and restoring - * the caseValue is just paranoia, but let's do it anyway.) - */ - econtext->caseValue_datum = values[fieldnum - 1]; - econtext->caseValue_isNull = isnull[fieldnum - 1]; - - values[fieldnum - 1] = ExecEvalExpr(newval, - econtext, - &isnull[fieldnum - 1], - NULL); - } - - econtext->caseValue_datum = save_datum; - econtext->caseValue_isNull = save_isNull; - - tuple = heap_form_tuple(tupDesc, values, isnull); - - pfree(values); - pfree(isnull); - - return HeapTupleGetDatum(tuple); -} - -/* ---------------------------------------------------------------- - * ExecEvalRelabelType - * - * Evaluate a RelabelType node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalRelabelType(GenericExprState *exprstate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - return ExecEvalExpr(exprstate->arg, econtext, isNull, isDone); -} - -/* ---------------------------------------------------------------- - * ExecEvalCoerceViaIO - * - * Evaluate a CoerceViaIO node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalCoerceViaIO(CoerceViaIOState *iostate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - Datum result; - Datum inputval; - char *string; - - inputval = ExecEvalExpr(iostate->arg, econtext, isNull, isDone); - - if (isDone && *isDone == ExprEndResult) - return inputval; /* nothing to do */ - - if (*isNull) - string = NULL; /* output functions are not called on nulls */ - else - string = OutputFunctionCall(&iostate->outfunc, inputval); - - result = InputFunctionCall(&iostate->infunc, - string, - iostate->intypioparam, - -1); - - /* The input function cannot change the null/not-null status */ - return result; -} - -/* ---------------------------------------------------------------- - * ExecEvalArrayCoerceExpr - * - * Evaluate an ArrayCoerceExpr node. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalArrayCoerceExpr(ArrayCoerceExprState *astate, - ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ArrayCoerceExpr *acoerce = (ArrayCoerceExpr *) astate->xprstate.expr; - Datum result; - FunctionCallInfoData locfcinfo; - - result = ExecEvalExpr(astate->arg, econtext, isNull, isDone); - - if (isDone && *isDone == ExprEndResult) - return result; /* nothing to do */ - if (*isNull) - return result; /* nothing to do */ - - /* - * If it's binary-compatible, modify the element type in the array header, - * but otherwise leave the array as we received it. - */ - if (!OidIsValid(acoerce->elemfuncid)) - { - /* Detoast input array if necessary, and copy in any case */ - ArrayType *array = DatumGetArrayTypePCopy(result); - - ARR_ELEMTYPE(array) = astate->resultelemtype; - PG_RETURN_ARRAYTYPE_P(array); - } - - /* Initialize function cache if first time through */ - if (astate->elemfunc.fn_oid == InvalidOid) - { - AclResult aclresult; - - /* Check permission to call function */ - aclresult = pg_proc_aclcheck(acoerce->elemfuncid, GetUserId(), - ACL_EXECUTE); - if (aclresult != ACLCHECK_OK) - aclcheck_error(aclresult, ACL_KIND_PROC, - get_func_name(acoerce->elemfuncid)); - InvokeFunctionExecuteHook(acoerce->elemfuncid); - - /* Set up the primary fmgr lookup information */ - fmgr_info_cxt(acoerce->elemfuncid, &(astate->elemfunc), - econtext->ecxt_per_query_memory); - fmgr_info_set_expr((Node *) acoerce, &(astate->elemfunc)); - } - - /* - * Use array_map to apply the function to each array element. - * - * We pass on the desttypmod and isExplicit flags whether or not the - * function wants them. - * - * Note: coercion functions are assumed to not use collation. - */ - InitFunctionCallInfoData(locfcinfo, &(astate->elemfunc), 3, - InvalidOid, NULL, NULL); - locfcinfo.arg[0] = result; - locfcinfo.arg[1] = Int32GetDatum(acoerce->resulttypmod); - locfcinfo.arg[2] = BoolGetDatum(acoerce->isExplicit); - locfcinfo.argnull[0] = false; - locfcinfo.argnull[1] = false; - locfcinfo.argnull[2] = false; - - return array_map(&locfcinfo, astate->resultelemtype, astate->amstate); -} - -/* ---------------------------------------------------------------- - * ExecEvalCurrentOfExpr - * - * The planner should convert CURRENT OF into a TidScan qualification, or some - * other special handling in a ForeignScan node. So we have to be able to do - * ExecInitExpr on a CurrentOfExpr, but we shouldn't ever actually execute it. - * If we get here, we suppose we must be dealing with CURRENT OF on a foreign - * table whose FDW doesn't handle it, and complain accordingly. - * ---------------------------------------------------------------- - */ -static Datum -ExecEvalCurrentOfExpr(ExprState *exprstate, ExprContext *econtext, - bool *isNull, ExprDoneCond *isDone) -{ - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("WHERE CURRENT OF is not supported for this table type"))); - return 0; /* keep compiler quiet */ -} - - -/* - * ExecEvalExprSwitchContext - * - * Same as ExecEvalExpr, but get into the right allocation context explicitly. - */ -Datum -ExecEvalExprSwitchContext(ExprState *expression, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) -{ - Datum retDatum; - MemoryContext oldContext; - - oldContext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); - retDatum = ExecEvalExpr(expression, econtext, isNull, isDone); - MemoryContextSwitchTo(oldContext); - return retDatum; -} - - -/* - * ExecInitExpr: prepare an expression tree for execution - * - * This function builds and returns an ExprState tree paralleling the given - * Expr node tree. The ExprState tree can then be handed to ExecEvalExpr - * for execution. Because the Expr tree itself is read-only as far as - * ExecInitExpr and ExecEvalExpr are concerned, several different executions - * of the same plan tree can occur concurrently. - * - * This must be called in a memory context that will last as long as repeated - * executions of the expression are needed. Typically the context will be - * the same as the per-query context of the associated ExprContext. - * - * Any Aggref, WindowFunc, or SubPlan nodes found in the tree are added to the - * lists of such nodes held by the parent PlanState. Otherwise, we do very - * little initialization here other than building the state-node tree. Any - * nontrivial work associated with initializing runtime info for a node should - * happen during the first actual evaluation of that node. (This policy lets - * us avoid work if the node is never actually evaluated.) - * - * Note: there is no ExecEndExpr function; we assume that any resource - * cleanup needed will be handled by just releasing the memory context - * in which the state tree is built. Functions that require additional - * 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. - * - * '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) -{ - ExprState *state; - - if (node == NULL) - return NULL; - - /* Guard against stack overflow due to overly complex expressions */ - check_stack_depth(); - - switch (nodeTag(node)) - { - case T_Var: - /* varattno == InvalidAttrNumber means it's a whole-row Var */ - if (((Var *) node)->varattno == InvalidAttrNumber) - { - WholeRowVarExprState *wstate = makeNode(WholeRowVarExprState); - - wstate->parent = parent; - wstate->wrv_tupdesc = NULL; - wstate->wrv_junkFilter = NULL; - state = (ExprState *) wstate; - state->evalfunc = (ExprStateEvalFunc) ExecEvalWholeRowVar; - } - else - { - state = (ExprState *) makeNode(ExprState); - state->evalfunc = ExecEvalScalarVar; - } - break; - case T_Const: - state = (ExprState *) makeNode(ExprState); - state->evalfunc = ExecEvalConst; - break; - case T_Param: - state = (ExprState *) makeNode(ExprState); - switch (((Param *) node)->paramkind) - { - case PARAM_EXEC: - state->evalfunc = ExecEvalParamExec; - break; - case PARAM_EXTERN: - state->evalfunc = ExecEvalParamExtern; - break; - default: - elog(ERROR, "unrecognized paramkind: %d", - (int) ((Param *) node)->paramkind); - break; - } - break; - case T_CoerceToDomainValue: - state = (ExprState *) makeNode(ExprState); - state->evalfunc = ExecEvalCoerceToDomainValue; - break; - case T_CaseTestExpr: - state = (ExprState *) makeNode(ExprState); - state->evalfunc = ExecEvalCaseTestExpr; - break; - case T_Aggref: - { - AggrefExprState *astate = makeNode(AggrefExprState); - - astate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalAggref; - if (parent && IsA(parent, AggState)) - { - AggState *aggstate = (AggState *) parent; - - aggstate->aggs = lcons(astate, aggstate->aggs); - aggstate->numaggs++; - } - else - { - /* planner messed up */ - elog(ERROR, "Aggref found in non-Agg plan node"); - } - state = (ExprState *) astate; - } - break; - case T_GroupingFunc: - { - GroupingFunc *grp_node = (GroupingFunc *) node; - GroupingFuncExprState *grp_state = makeNode(GroupingFuncExprState); - Agg *agg = NULL; - - if (!parent || !IsA(parent, AggState) ||!IsA(parent->plan, Agg)) - elog(ERROR, "parent of GROUPING is not Agg node"); - - grp_state->aggstate = (AggState *) parent; - - agg = (Agg *) (parent->plan); - - if (agg->groupingSets) - grp_state->clauses = grp_node->cols; - else - grp_state->clauses = NIL; - - state = (ExprState *) grp_state; - state->evalfunc = (ExprStateEvalFunc) ExecEvalGroupingFuncExpr; - } - break; - case T_WindowFunc: - { - WindowFunc *wfunc = (WindowFunc *) node; - WindowFuncExprState *wfstate = makeNode(WindowFuncExprState); - - wfstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalWindowFunc; - if (parent && IsA(parent, WindowAggState)) - { - WindowAggState *winstate = (WindowAggState *) parent; - int nfuncs; - - winstate->funcs = lcons(wfstate, winstate->funcs); - nfuncs = ++winstate->numfuncs; - if (wfunc->winagg) - winstate->numaggs++; - - wfstate->args = (List *) ExecInitExpr((Expr *) wfunc->args, - parent); - wfstate->aggfilter = ExecInitExpr(wfunc->aggfilter, - parent); - - /* - * Complain if the windowfunc's arguments contain any - * windowfuncs; nested window functions are semantically - * nonsensical. (This should have been caught earlier, - * but we defend against it here anyway.) - */ - if (nfuncs != winstate->numfuncs) - ereport(ERROR, - (errcode(ERRCODE_WINDOWING_ERROR), - errmsg("window function calls cannot be nested"))); - } - else - { - /* planner messed up */ - elog(ERROR, "WindowFunc found in non-WindowAgg plan node"); - } - state = (ExprState *) wfstate; - } - break; - case T_ArrayRef: - { - ArrayRef *aref = (ArrayRef *) node; - ArrayRefExprState *astate = makeNode(ArrayRefExprState); - - astate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalArrayRef; - astate->refupperindexpr = (List *) - ExecInitExpr((Expr *) aref->refupperindexpr, parent); - astate->reflowerindexpr = (List *) - ExecInitExpr((Expr *) aref->reflowerindexpr, parent); - astate->refexpr = ExecInitExpr(aref->refexpr, parent); - astate->refassgnexpr = ExecInitExpr(aref->refassgnexpr, - parent); - /* do one-time catalog lookups for type info */ - astate->refattrlength = get_typlen(aref->refarraytype); - get_typlenbyvalalign(aref->refelemtype, - &astate->refelemlength, - &astate->refelembyval, - &astate->refelemalign); - state = (ExprState *) astate; - } - break; - case T_FuncExpr: - { - FuncExpr *funcexpr = (FuncExpr *) node; - FuncExprState *fstate = makeNode(FuncExprState); - - fstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalFunc; - fstate->args = (List *) - ExecInitExpr((Expr *) funcexpr->args, parent); - fstate->func.fn_oid = InvalidOid; /* not initialized */ - state = (ExprState *) fstate; - } - break; - case T_OpExpr: - { - OpExpr *opexpr = (OpExpr *) node; - FuncExprState *fstate = makeNode(FuncExprState); - - fstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalOper; - fstate->args = (List *) - ExecInitExpr((Expr *) opexpr->args, parent); - fstate->func.fn_oid = InvalidOid; /* not initialized */ - state = (ExprState *) fstate; - } - break; - case T_DistinctExpr: - { - DistinctExpr *distinctexpr = (DistinctExpr *) node; - FuncExprState *fstate = makeNode(FuncExprState); - - fstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalDistinct; - fstate->args = (List *) - ExecInitExpr((Expr *) distinctexpr->args, parent); - fstate->func.fn_oid = InvalidOid; /* not initialized */ - state = (ExprState *) fstate; - } - break; - case T_NullIfExpr: - { - NullIfExpr *nullifexpr = (NullIfExpr *) node; - FuncExprState *fstate = makeNode(FuncExprState); - - fstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalNullIf; - fstate->args = (List *) - ExecInitExpr((Expr *) nullifexpr->args, parent); - fstate->func.fn_oid = InvalidOid; /* not initialized */ - state = (ExprState *) fstate; - } - break; - case T_ScalarArrayOpExpr: - { - ScalarArrayOpExpr *opexpr = (ScalarArrayOpExpr *) node; - ScalarArrayOpExprState *sstate = makeNode(ScalarArrayOpExprState); - - sstate->fxprstate.xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalScalarArrayOp; - sstate->fxprstate.args = (List *) - ExecInitExpr((Expr *) opexpr->args, parent); - sstate->fxprstate.func.fn_oid = InvalidOid; /* not initialized */ - sstate->element_type = InvalidOid; /* ditto */ - state = (ExprState *) sstate; - } - break; - case T_BoolExpr: - { - BoolExpr *boolexpr = (BoolExpr *) node; - BoolExprState *bstate = makeNode(BoolExprState); - - switch (boolexpr->boolop) - { - case AND_EXPR: - bstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalAnd; - break; - case OR_EXPR: - bstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalOr; - break; - case NOT_EXPR: - bstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalNot; - break; - default: - elog(ERROR, "unrecognized boolop: %d", - (int) boolexpr->boolop); - break; - } - bstate->args = (List *) - ExecInitExpr((Expr *) boolexpr->args, parent); - state = (ExprState *) bstate; - } - break; - case T_SubPlan: - { - SubPlan *subplan = (SubPlan *) node; - SubPlanState *sstate; - - if (!parent) - elog(ERROR, "SubPlan found with no parent plan"); - - sstate = ExecInitSubPlan(subplan, parent); - - /* Add SubPlanState nodes to parent->subPlan */ - parent->subPlan = lappend(parent->subPlan, sstate); - - state = (ExprState *) sstate; - } - break; - case T_AlternativeSubPlan: - { - AlternativeSubPlan *asplan = (AlternativeSubPlan *) node; - AlternativeSubPlanState *asstate; - - if (!parent) - elog(ERROR, "AlternativeSubPlan found with no parent plan"); - - asstate = ExecInitAlternativeSubPlan(asplan, parent); - - state = (ExprState *) asstate; - } - break; - case T_FieldSelect: - { - FieldSelect *fselect = (FieldSelect *) node; - FieldSelectState *fstate = makeNode(FieldSelectState); - - fstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalFieldSelect; - fstate->arg = ExecInitExpr(fselect->arg, parent); - fstate->argdesc = NULL; - state = (ExprState *) fstate; - } - break; - case T_FieldStore: - { - FieldStore *fstore = (FieldStore *) node; - FieldStoreState *fstate = makeNode(FieldStoreState); - - fstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalFieldStore; - fstate->arg = ExecInitExpr(fstore->arg, parent); - fstate->newvals = (List *) ExecInitExpr((Expr *) fstore->newvals, parent); - fstate->argdesc = NULL; - state = (ExprState *) fstate; - } - break; - case T_RelabelType: - { - RelabelType *relabel = (RelabelType *) node; - GenericExprState *gstate = makeNode(GenericExprState); - - gstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalRelabelType; - gstate->arg = ExecInitExpr(relabel->arg, parent); - state = (ExprState *) gstate; - } - break; - case T_CoerceViaIO: - { - CoerceViaIO *iocoerce = (CoerceViaIO *) node; - CoerceViaIOState *iostate = makeNode(CoerceViaIOState); - Oid iofunc; - bool typisvarlena; - - iostate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalCoerceViaIO; - iostate->arg = ExecInitExpr(iocoerce->arg, parent); - /* lookup the result type's input function */ - getTypeInputInfo(iocoerce->resulttype, &iofunc, - &iostate->intypioparam); - fmgr_info(iofunc, &iostate->infunc); - /* lookup the input type's output function */ - getTypeOutputInfo(exprType((Node *) iocoerce->arg), - &iofunc, &typisvarlena); - fmgr_info(iofunc, &iostate->outfunc); - state = (ExprState *) iostate; - } - break; - case T_ArrayCoerceExpr: - { - ArrayCoerceExpr *acoerce = (ArrayCoerceExpr *) node; - ArrayCoerceExprState *astate = makeNode(ArrayCoerceExprState); - - astate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalArrayCoerceExpr; - astate->arg = ExecInitExpr(acoerce->arg, parent); - astate->resultelemtype = get_element_type(acoerce->resulttype); - if (astate->resultelemtype == InvalidOid) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("target type is not an array"))); - /* Arrays over domains aren't supported yet */ - Assert(getBaseType(astate->resultelemtype) == - astate->resultelemtype); - astate->elemfunc.fn_oid = InvalidOid; /* not initialized */ - astate->amstate = (ArrayMapState *) palloc0(sizeof(ArrayMapState)); - state = (ExprState *) astate; - } - break; - case T_ConvertRowtypeExpr: - { - ConvertRowtypeExpr *convert = (ConvertRowtypeExpr *) node; - ConvertRowtypeExprState *cstate = makeNode(ConvertRowtypeExprState); - - cstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalConvertRowtype; - cstate->arg = ExecInitExpr(convert->arg, parent); - state = (ExprState *) cstate; - } - break; - case T_CaseExpr: - { - CaseExpr *caseexpr = (CaseExpr *) node; - CaseExprState *cstate = makeNode(CaseExprState); - List *outlist = NIL; - ListCell *l; - - cstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalCase; - cstate->arg = ExecInitExpr(caseexpr->arg, parent); - foreach(l, caseexpr->args) - { - CaseWhen *when = (CaseWhen *) lfirst(l); - CaseWhenState *wstate = makeNode(CaseWhenState); - - Assert(IsA(when, CaseWhen)); - wstate->xprstate.evalfunc = NULL; /* not used */ - wstate->xprstate.expr = (Expr *) when; - wstate->expr = ExecInitExpr(when->expr, parent); - wstate->result = ExecInitExpr(when->result, parent); - outlist = lappend(outlist, wstate); - } - cstate->args = outlist; - cstate->defresult = ExecInitExpr(caseexpr->defresult, parent); - state = (ExprState *) cstate; - } - break; - case T_ArrayExpr: - { - ArrayExpr *arrayexpr = (ArrayExpr *) node; - ArrayExprState *astate = makeNode(ArrayExprState); - List *outlist = NIL; - ListCell *l; - - astate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalArray; - foreach(l, arrayexpr->elements) - { - Expr *e = (Expr *) lfirst(l); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - astate->elements = outlist; - /* do one-time catalog lookup for type info */ - get_typlenbyvalalign(arrayexpr->element_typeid, - &astate->elemlength, - &astate->elembyval, - &astate->elemalign); - state = (ExprState *) astate; - } - break; - case T_RowExpr: - { - RowExpr *rowexpr = (RowExpr *) node; - RowExprState *rstate = makeNode(RowExprState); - Form_pg_attribute *attrs; - List *outlist = NIL; - ListCell *l; - int i; - - rstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalRow; - /* Build tupdesc to describe result tuples */ - if (rowexpr->row_typeid == RECORDOID) - { - /* generic record, use types of given expressions */ - rstate->tupdesc = ExecTypeFromExprList(rowexpr->args); - } - else - { - /* it's been cast to a named type, use that */ - rstate->tupdesc = lookup_rowtype_tupdesc_copy(rowexpr->row_typeid, -1); - } - /* In either case, adopt RowExpr's column aliases */ - ExecTypeSetColNames(rstate->tupdesc, rowexpr->colnames); - /* Bless the tupdesc in case it's now of type RECORD */ - BlessTupleDesc(rstate->tupdesc); - /* Set up evaluation, skipping any deleted columns */ - Assert(list_length(rowexpr->args) <= rstate->tupdesc->natts); - attrs = rstate->tupdesc->attrs; - i = 0; - foreach(l, rowexpr->args) - { - Expr *e = (Expr *) lfirst(l); - ExprState *estate; - - if (!attrs[i]->attisdropped) - { - /* - * Guard against ALTER COLUMN TYPE on rowtype since - * the RowExpr was created. XXX should we check - * typmod too? Not sure we can be sure it'll be the - * same. - */ - if (exprType((Node *) e) != attrs[i]->atttypid) - ereport(ERROR, - (errcode(ERRCODE_DATATYPE_MISMATCH), - errmsg("ROW() column has type %s instead of type %s", - format_type_be(exprType((Node *) e)), - format_type_be(attrs[i]->atttypid)))); - } - else - { - /* - * Ignore original expression and insert a NULL. We - * don't really care what type of NULL it is, so - * always make an int4 NULL. - */ - e = (Expr *) makeNullConst(INT4OID, -1, InvalidOid); - } - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - i++; - } - rstate->args = outlist; - state = (ExprState *) rstate; - } - break; - case T_RowCompareExpr: - { - RowCompareExpr *rcexpr = (RowCompareExpr *) node; - RowCompareExprState *rstate = makeNode(RowCompareExprState); - int nopers = list_length(rcexpr->opnos); - List *outlist; - ListCell *l; - ListCell *l2; - ListCell *l3; - int i; - - rstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalRowCompare; - Assert(list_length(rcexpr->largs) == nopers); - outlist = NIL; - foreach(l, rcexpr->largs) - { - Expr *e = (Expr *) lfirst(l); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - rstate->largs = outlist; - Assert(list_length(rcexpr->rargs) == nopers); - outlist = NIL; - foreach(l, rcexpr->rargs) - { - Expr *e = (Expr *) lfirst(l); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - rstate->rargs = outlist; - Assert(list_length(rcexpr->opfamilies) == nopers); - rstate->funcs = (FmgrInfo *) palloc(nopers * sizeof(FmgrInfo)); - rstate->collations = (Oid *) palloc(nopers * sizeof(Oid)); - i = 0; - forthree(l, rcexpr->opnos, l2, rcexpr->opfamilies, l3, rcexpr->inputcollids) - { - Oid opno = lfirst_oid(l); - Oid opfamily = lfirst_oid(l2); - Oid inputcollid = lfirst_oid(l3); - int strategy; - Oid lefttype; - Oid righttype; - Oid proc; - - get_op_opfamily_properties(opno, opfamily, false, - &strategy, - &lefttype, - &righttype); - proc = get_opfamily_proc(opfamily, - lefttype, - righttype, - BTORDER_PROC); - - /* - * If we enforced permissions checks on index support - * functions, we'd need to make a check here. But the - * index support machinery doesn't do that, and neither - * does this code. - */ - fmgr_info(proc, &(rstate->funcs[i])); - rstate->collations[i] = inputcollid; - i++; - } - state = (ExprState *) rstate; - } - break; - case T_CoalesceExpr: - { - CoalesceExpr *coalesceexpr = (CoalesceExpr *) node; - CoalesceExprState *cstate = makeNode(CoalesceExprState); - List *outlist = NIL; - ListCell *l; - - cstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalCoalesce; - foreach(l, coalesceexpr->args) - { - Expr *e = (Expr *) lfirst(l); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - cstate->args = outlist; - state = (ExprState *) cstate; - } - break; - case T_MinMaxExpr: - { - MinMaxExpr *minmaxexpr = (MinMaxExpr *) node; - MinMaxExprState *mstate = makeNode(MinMaxExprState); - List *outlist = NIL; - ListCell *l; - TypeCacheEntry *typentry; - - mstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalMinMax; - foreach(l, minmaxexpr->args) - { - Expr *e = (Expr *) lfirst(l); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - mstate->args = outlist; - /* Look up the btree comparison function for the datatype */ - typentry = lookup_type_cache(minmaxexpr->minmaxtype, - TYPECACHE_CMP_PROC); - if (!OidIsValid(typentry->cmp_proc)) - ereport(ERROR, - (errcode(ERRCODE_UNDEFINED_FUNCTION), - errmsg("could not identify a comparison function for type %s", - format_type_be(minmaxexpr->minmaxtype)))); - - /* - * If we enforced permissions checks on index support - * functions, we'd need to make a check here. But the index - * support machinery doesn't do that, and neither does this - * code. - */ - fmgr_info(typentry->cmp_proc, &(mstate->cfunc)); - state = (ExprState *) mstate; - } - break; - case T_XmlExpr: - { - XmlExpr *xexpr = (XmlExpr *) node; - XmlExprState *xstate = makeNode(XmlExprState); - List *outlist; - ListCell *arg; - - xstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalXml; - outlist = NIL; - foreach(arg, xexpr->named_args) - { - Expr *e = (Expr *) lfirst(arg); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - xstate->named_args = outlist; - - outlist = NIL; - foreach(arg, xexpr->args) - { - Expr *e = (Expr *) lfirst(arg); - ExprState *estate; - - estate = ExecInitExpr(e, parent); - outlist = lappend(outlist, estate); - } - xstate->args = outlist; - - state = (ExprState *) xstate; - } - break; - case T_NullTest: - { - NullTest *ntest = (NullTest *) node; - NullTestState *nstate = makeNode(NullTestState); - - nstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalNullTest; - nstate->arg = ExecInitExpr(ntest->arg, parent); - nstate->argdesc = NULL; - state = (ExprState *) nstate; - } - break; - case T_BooleanTest: - { - BooleanTest *btest = (BooleanTest *) node; - GenericExprState *gstate = makeNode(GenericExprState); - - gstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalBooleanTest; - gstate->arg = ExecInitExpr(btest->arg, parent); - state = (ExprState *) gstate; - } - break; - case T_CoerceToDomain: - { - CoerceToDomain *ctest = (CoerceToDomain *) node; - CoerceToDomainState *cstate = makeNode(CoerceToDomainState); - - cstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalCoerceToDomain; - cstate->arg = ExecInitExpr(ctest->arg, parent); - /* We spend an extra palloc to reduce header inclusions */ - cstate->constraint_ref = (DomainConstraintRef *) - palloc(sizeof(DomainConstraintRef)); - InitDomainConstraintRef(ctest->resulttype, - cstate->constraint_ref, - CurrentMemoryContext); - state = (ExprState *) cstate; - } - break; - case T_CurrentOfExpr: - state = (ExprState *) makeNode(ExprState); - state->evalfunc = ExecEvalCurrentOfExpr; - break; - case T_TargetEntry: - { - TargetEntry *tle = (TargetEntry *) node; - GenericExprState *gstate = makeNode(GenericExprState); - - gstate->xprstate.evalfunc = NULL; /* not used */ - gstate->arg = ExecInitExpr(tle->expr, parent); - state = (ExprState *) gstate; - } - break; - case T_List: - { - List *outlist = NIL; - ListCell *l; - - foreach(l, (List *) node) - { - outlist = lappend(outlist, - ExecInitExpr((Expr *) lfirst(l), - parent)); - } - /* Don't fall through to the "common" code below */ - return (ExprState *) outlist; - } - default: - elog(ERROR, "unrecognized node type: %d", - (int) nodeTag(node)); - state = NULL; /* keep compiler quiet */ - break; - } - - /* Common code for all state-node types */ - state->expr = node; - - return state; -} - -/* - * 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 run the - * passed expression tree through expression_planner() to prepare it for - * execution. (In ordinary Plan trees the regular planning process will have - * made the appropriate transformations on expressions, but for standalone - * expressions this won't have happened.) - */ -ExprState * -ExecPrepareExpr(Expr *node, EState *estate) -{ - ExprState *result; - MemoryContext oldcontext; - - oldcontext = MemoryContextSwitchTo(estate->es_query_cxt); - - node = expression_planner(node); - - result = ExecInitExpr(node, NULL); - - MemoryContextSwitchTo(oldcontext); - - return result; -} - - -/* ---------------------------------------------------------------- - * ExecQual / ExecTargetList / ExecProject - * ---------------------------------------------------------------- - */ - -/* ---------------------------------------------------------------- - * ExecQual - * - * Evaluates a conjunctive boolean expression (qual list) and - * returns true iff none of the subexpressions are false. - * (We also return true if the list is empty.) - * - * If some of the subexpressions yield NULL but none yield FALSE, - * then the result of the conjunction is NULL (ie, unknown) - * according to three-valued boolean logic. In this case, - * we return the value specified by the "resultForNull" parameter. - * - * Callers evaluating WHERE clauses should pass resultForNull=FALSE, - * since SQL specifies that tuples with null WHERE results do not - * get selected. On the other hand, callers evaluating constraint - * conditions should pass resultForNull=TRUE, since SQL also specifies - * that NULL constraint conditions are not failures. - * - * NOTE: it would not be correct to use this routine to evaluate an - * AND subclause of a boolean expression; for that purpose, a NULL - * result must be returned as NULL so that it can be properly treated - * in the next higher operator (cf. ExecEvalAnd and ExecEvalOr). - * This routine is only used in contexts where a complete expression - * is being evaluated and we know that NULL can be treated the same - * as one boolean result or the other. - * - * ---------------------------------------------------------------- - */ -bool -ExecQual(List *qual, ExprContext *econtext, bool resultForNull) -{ - bool result; - MemoryContext oldContext; - ListCell *l; - - /* - * debugging stuff - */ - EV_printf("ExecQual: qual is "); - EV_nodeDisplay(qual); - EV_printf("\n"); - - /* - * Run in short-lived per-tuple context while computing expressions. - */ - oldContext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); - - /* - * Evaluate the qual conditions one at a time. If we find a FALSE result, - * we can stop evaluating and return FALSE --- the AND result must be - * FALSE. Also, if we find a NULL result when resultForNull is FALSE, we - * can stop and return FALSE --- the AND result must be FALSE or NULL in - * that case, and the caller doesn't care which. - * - * If we get to the end of the list, we can return TRUE. This will happen - * when the AND result is indeed TRUE, or when the AND result is NULL (one - * or more NULL subresult, with all the rest TRUE) and the caller has - * specified resultForNull = TRUE. - */ - result = true; - - foreach(l, qual) - { - ExprState *clause = (ExprState *) lfirst(l); - Datum expr_value; - bool isNull; - - expr_value = ExecEvalExpr(clause, econtext, &isNull, NULL); - - if (isNull) - { - if (resultForNull == false) - { - result = false; /* treat NULL as FALSE */ - break; - } - } - else - { - if (!DatumGetBool(expr_value)) - { - result = false; /* definitely FALSE */ - break; - } - } - } - - MemoryContextSwitchTo(oldContext); - - return result; -} - -/* - * Number of items in a tlist (including any resjunk items!) - */ -int -ExecTargetListLength(List *targetlist) -{ - /* This used to be more complex, but fjoins are dead */ - return list_length(targetlist); -} - -/* - * Number of items in a tlist, not including any resjunk items - */ -int -ExecCleanTargetListLength(List *targetlist) -{ - int len = 0; - ListCell *tl; - - foreach(tl, targetlist) - { - TargetEntry *curTle = (TargetEntry *) lfirst(tl); - - Assert(IsA(curTle, TargetEntry)); - if (!curTle->resjunk) - len++; - } - return len; -} - -/* - * ExecTargetList - * Evaluates a targetlist with respect to the given - * expression context. Returns TRUE if we were able to create - * a result, FALSE if we have exhausted a set-valued expression. - * - * Results are stored into the passed values and isnull arrays. - * The caller must provide an itemIsDone array that persists across calls. - * - * As with ExecEvalExpr, the caller should pass isDone = NULL if not - * prepared to deal with sets of result tuples. Otherwise, a return - * of *isDone = ExprMultipleResult signifies a set element, and a return - * of *isDone = ExprEndResult signifies end of the set of tuple. - * We assume that *isDone has been initialized to ExprSingleResult by caller. - * - * Since fields of the result tuple might be multiply referenced in higher - * plan nodes, we have to force any read/write expanded values to read-only - * status. It's a bit annoying to have to do that for every projected - * expression; in the future, consider teaching the planner to detect - * actually-multiply-referenced Vars and insert an expression node that - * would do that only where really required. - */ -static bool -ExecTargetList(List *targetlist, - TupleDesc tupdesc, - ExprContext *econtext, - Datum *values, - bool *isnull, - ExprDoneCond *itemIsDone, - ExprDoneCond *isDone) -{ - Form_pg_attribute *att = tupdesc->attrs; - MemoryContext oldContext; - ListCell *tl; - bool haveDoneSets; - - /* - * Run in short-lived per-tuple context while computing expressions. - */ - oldContext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); - - /* - * evaluate all the expressions in the target list - */ - haveDoneSets = false; /* any exhausted set exprs in tlist? */ - - foreach(tl, targetlist) - { - GenericExprState *gstate = (GenericExprState *) lfirst(tl); - TargetEntry *tle = (TargetEntry *) gstate->xprstate.expr; - AttrNumber resind = tle->resno - 1; - - values[resind] = ExecEvalExpr(gstate->arg, - econtext, - &isnull[resind], - &itemIsDone[resind]); - - values[resind] = MakeExpandedObjectReadOnly(values[resind], - isnull[resind], - att[resind]->attlen); - - if (itemIsDone[resind] != ExprSingleResult) - { - /* We have a set-valued expression in the tlist */ - if (isDone == NULL) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("set-valued function called in context that cannot accept a set"))); - if (itemIsDone[resind] == ExprMultipleResult) - { - /* we have undone sets in the tlist, set flag */ - *isDone = ExprMultipleResult; - } - else - { - /* we have done sets in the tlist, set flag for that */ - haveDoneSets = true; - } - } - } - - if (haveDoneSets) - { - /* - * note: can't get here unless we verified isDone != NULL - */ - if (*isDone == ExprSingleResult) - { - /* - * all sets are done, so report that tlist expansion is complete. - */ - *isDone = ExprEndResult; - MemoryContextSwitchTo(oldContext); - return false; - } - else - { - /* - * We have some done and some undone sets. Restart the done ones - * so that we can deliver a tuple (if possible). - */ - foreach(tl, targetlist) - { - GenericExprState *gstate = (GenericExprState *) lfirst(tl); - TargetEntry *tle = (TargetEntry *) gstate->xprstate.expr; - AttrNumber resind = tle->resno - 1; - - if (itemIsDone[resind] == ExprEndResult) - { - values[resind] = ExecEvalExpr(gstate->arg, - econtext, - &isnull[resind], - &itemIsDone[resind]); - - values[resind] = MakeExpandedObjectReadOnly(values[resind], - isnull[resind], - att[resind]->attlen); - - if (itemIsDone[resind] == ExprEndResult) - { - /* - * Oh dear, this item is returning an empty set. Guess - * we can't make a tuple after all. - */ - *isDone = ExprEndResult; - break; - } - } - } - - /* - * If we cannot make a tuple because some sets are empty, we still - * have to cycle the nonempty sets to completion, else resources - * will not be released from subplans etc. - * - * XXX is that still necessary? - */ - if (*isDone == ExprEndResult) - { - foreach(tl, targetlist) - { - GenericExprState *gstate = (GenericExprState *) lfirst(tl); - TargetEntry *tle = (TargetEntry *) gstate->xprstate.expr; - AttrNumber resind = tle->resno - 1; - - while (itemIsDone[resind] == ExprMultipleResult) - { - values[resind] = ExecEvalExpr(gstate->arg, - econtext, - &isnull[resind], - &itemIsDone[resind]); - /* no need for MakeExpandedObjectReadOnly */ - } - } - - MemoryContextSwitchTo(oldContext); - return false; - } - } - } - - /* Report success */ - MemoryContextSwitchTo(oldContext); - - return true; -} - -/* - * ExecProject - * - * projects a tuple based on projection info and stores - * it in the previously specified tuple table slot. - * - * Note: the result is always a virtual tuple; therefore it - * may reference the contents of the exprContext's scan tuples - * and/or temporary results constructed in the exprContext. - * If the caller wishes the result to be valid longer than that - * data will be valid, he must call ExecMaterializeSlot on the - * result slot. - */ -TupleTableSlot * -ExecProject(ProjectionInfo *projInfo, ExprDoneCond *isDone) -{ - TupleTableSlot *slot; - ExprContext *econtext; - int numSimpleVars; - - /* - * sanity checks - */ - Assert(projInfo != NULL); - - /* - * get the projection info we want - */ - slot = projInfo->pi_slot; - econtext = projInfo->pi_exprContext; - - /* Assume single result row until proven otherwise */ - if (isDone) - *isDone = ExprSingleResult; - - /* - * Clear any former contents of the result slot. This makes it safe for - * us to use the slot's Datum/isnull arrays as workspace. (Also, we can - * return the slot as-is if we decide no rows can be projected.) - */ - ExecClearTuple(slot); - - /* - * Force extraction of all input values that we'll need. The - * Var-extraction loops below depend on this, and we are also prefetching - * all attributes that will be referenced in the generic expressions. - */ - if (projInfo->pi_lastInnerVar > 0) - slot_getsomeattrs(econtext->ecxt_innertuple, - projInfo->pi_lastInnerVar); - if (projInfo->pi_lastOuterVar > 0) - slot_getsomeattrs(econtext->ecxt_outertuple, - projInfo->pi_lastOuterVar); - if (projInfo->pi_lastScanVar > 0) - slot_getsomeattrs(econtext->ecxt_scantuple, - projInfo->pi_lastScanVar); - - /* - * Assign simple Vars to result by direct extraction of fields from source - * slots ... a mite ugly, but fast ... - */ - numSimpleVars = projInfo->pi_numSimpleVars; - if (numSimpleVars > 0) - { - Datum *values = slot->tts_values; - bool *isnull = slot->tts_isnull; - int *varSlotOffsets = projInfo->pi_varSlotOffsets; - int *varNumbers = projInfo->pi_varNumbers; - int i; - - if (projInfo->pi_directMap) - { - /* especially simple case where vars go to output in order */ - for (i = 0; i < numSimpleVars; i++) - { - char *slotptr = ((char *) econtext) + varSlotOffsets[i]; - TupleTableSlot *varSlot = *((TupleTableSlot **) slotptr); - int varNumber = varNumbers[i] - 1; - - values[i] = varSlot->tts_values[varNumber]; - isnull[i] = varSlot->tts_isnull[varNumber]; - } - } - else - { - /* we have to pay attention to varOutputCols[] */ - int *varOutputCols = projInfo->pi_varOutputCols; - - for (i = 0; i < numSimpleVars; i++) - { - char *slotptr = ((char *) econtext) + varSlotOffsets[i]; - TupleTableSlot *varSlot = *((TupleTableSlot **) slotptr); - int varNumber = varNumbers[i] - 1; - int varOutputCol = varOutputCols[i] - 1; - - values[varOutputCol] = varSlot->tts_values[varNumber]; - isnull[varOutputCol] = varSlot->tts_isnull[varNumber]; - } - } - } - - /* - * If there are any generic expressions, evaluate them. It's possible - * that there are set-returning functions in such expressions; if so and - * we have reached the end of the set, we return the result slot, which we - * already marked empty. - */ - if (projInfo->pi_targetlist) - { - if (!ExecTargetList(projInfo->pi_targetlist, - slot->tts_tupleDescriptor, - econtext, - slot->tts_values, - slot->tts_isnull, - projInfo->pi_itemIsDone, - isDone)) - return slot; /* no more result rows, return empty slot */ - } - - /* - * Successfully formed a result row. Mark the result slot as containing a - * valid virtual tuple. - */ - return ExecStoreVirtualTuple(slot); -} diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c new file mode 100644 index 0000000000..c6a66b6195 --- /dev/null +++ b/src/backend/executor/execReplication.c @@ -0,0 +1,573 @@ +/*------------------------------------------------------------------------- + * + * execReplication.c + * miscellaneous executor routines for logical replication + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * IDENTIFICATION + * src/backend/executor/execReplication.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "access/relscan.h" +#include "access/transam.h" +#include "access/xact.h" +#include "commands/trigger.h" +#include "executor/executor.h" +#include "nodes/nodeFuncs.h" +#include "parser/parse_relation.h" +#include "parser/parsetree.h" +#include "storage/bufmgr.h" +#include "storage/lmgr.h" +#include "utils/datum.h" +#include "utils/lsyscache.h" +#include "utils/memutils.h" +#include "utils/rel.h" +#include "utils/snapmgr.h" +#include "utils/syscache.h" +#include "utils/tqual.h" + + +/* + * Setup a ScanKey for a search in the relation 'rel' for a tuple 'key' that + * is setup to match 'rel' (*NOT* idxrel!). + * + * Returns whether any column contains NULLs. + * + * This is not generic routine, it expects the idxrel to be replication + * identity of a rel and meet all limitations associated with that. + */ +static bool +build_replindex_scan_key(ScanKey skey, Relation rel, Relation idxrel, + TupleTableSlot *searchslot) +{ + int attoff; + bool isnull; + Datum indclassDatum; + oidvector *opclass; + int2vector *indkey = &idxrel->rd_index->indkey; + bool hasnulls = false; + + Assert(RelationGetReplicaIndex(rel) == RelationGetRelid(idxrel)); + + indclassDatum = SysCacheGetAttr(INDEXRELID, idxrel->rd_indextuple, + Anum_pg_index_indclass, &isnull); + Assert(!isnull); + opclass = (oidvector *) DatumGetPointer(indclassDatum); + + /* Build scankey for every attribute in the index. */ + for (attoff = 0; attoff < RelationGetNumberOfAttributes(idxrel); attoff++) + { + Oid operator; + Oid opfamily; + RegProcedure regop; + int pkattno = attoff + 1; + int mainattno = indkey->values[attoff]; + Oid optype = get_opclass_input_type(opclass->values[attoff]); + + /* + * Load the operator info. We need this to get the equality operator + * function for the scan key. + */ + opfamily = get_opclass_family(opclass->values[attoff]); + + operator = get_opfamily_member(opfamily, optype, + optype, + BTEqualStrategyNumber); + + if (!OidIsValid(operator)) + elog(ERROR, "could not find member %d(%u,%u) of opfamily %u", + BTEqualStrategyNumber, optype, optype, opfamily); + + regop = get_opcode(operator); + + /* Initialize the scankey. */ + ScanKeyInit(&skey[attoff], + pkattno, + BTEqualStrategyNumber, + regop, + searchslot->tts_values[mainattno - 1]); + + /* Check for null value. */ + if (searchslot->tts_isnull[mainattno - 1]) + { + hasnulls = true; + skey[attoff].sk_flags |= SK_ISNULL; + } + } + + return hasnulls; +} + +/* + * Search the relation 'rel' for tuple using the index. + * + * If a matching tuple is found, lock it with lockmode, fill the slot with its + * contents, and return true. Return false otherwise. + */ +bool +RelationFindReplTupleByIndex(Relation rel, Oid idxoid, + LockTupleMode lockmode, + TupleTableSlot *searchslot, + TupleTableSlot *outslot) +{ + HeapTuple scantuple; + ScanKeyData skey[INDEX_MAX_KEYS]; + IndexScanDesc scan; + SnapshotData snap; + TransactionId xwait; + Relation idxrel; + bool found; + + /* Open the index. */ + idxrel = index_open(idxoid, RowExclusiveLock); + + /* Start an index scan. */ + InitDirtySnapshot(snap); + scan = index_beginscan(rel, idxrel, &snap, + RelationGetNumberOfAttributes(idxrel), + 0); + + /* Build scan key. */ + build_replindex_scan_key(skey, rel, idxrel, searchslot); + +retry: + found = false; + + index_rescan(scan, skey, RelationGetNumberOfAttributes(idxrel), NULL, 0); + + /* Try to find the tuple */ + if ((scantuple = index_getnext(scan, ForwardScanDirection)) != NULL) + { + found = true; + ExecStoreTuple(scantuple, outslot, InvalidBuffer, false); + ExecMaterializeSlot(outslot); + + xwait = TransactionIdIsValid(snap.xmin) ? + snap.xmin : snap.xmax; + + /* + * If the tuple is locked, wait for locking transaction to finish and + * retry. + */ + if (TransactionIdIsValid(xwait)) + { + XactLockTableWait(xwait, NULL, NULL, XLTW_None); + goto retry; + } + } + + /* Found tuple, try to lock it in the lockmode. */ + if (found) + { + Buffer buf; + HeapUpdateFailureData hufd; + HTSU_Result res; + HeapTupleData locktup; + + ItemPointerCopy(&outslot->tts_tuple->t_self, &locktup.t_self); + + PushActiveSnapshot(GetLatestSnapshot()); + + res = heap_lock_tuple(rel, &locktup, GetCurrentCommandId(false), + lockmode, + LockWaitBlock, + false /* don't follow updates */ , + &buf, &hufd); + /* the tuple slot already has the buffer pinned */ + ReleaseBuffer(buf); + + PopActiveSnapshot(); + + switch (res) + { + case HeapTupleMayBeUpdated: + break; + case HeapTupleUpdated: + /* XXX: Improve handling here */ + ereport(LOG, + (errcode(ERRCODE_T_R_SERIALIZATION_FAILURE), + errmsg("concurrent update, retrying"))); + goto retry; + case HeapTupleInvisible: + elog(ERROR, "attempted to lock invisible tuple"); + default: + elog(ERROR, "unexpected heap_lock_tuple status: %u", res); + break; + } + } + + index_endscan(scan); + + /* Don't release lock until commit. */ + index_close(idxrel, NoLock); + + return found; +} + +/* + * Compare the tuple and slot and check if they have equal values. + * + * We use binary datum comparison which might return false negatives but + * that's the best we can do here as there may be multiple notions of + * equality for the data types and table columns don't specify which one + * to use. + */ +static bool +tuple_equals_slot(TupleDesc desc, HeapTuple tup, TupleTableSlot *slot) +{ + Datum values[MaxTupleAttributeNumber]; + bool isnull[MaxTupleAttributeNumber]; + int attrnum; + Form_pg_attribute att; + + heap_deform_tuple(tup, desc, values, isnull); + + /* Check equality of the attributes. */ + for (attrnum = 0; attrnum < desc->natts; attrnum++) + { + /* + * If one value is NULL and other is not, then they are certainly not + * equal + */ + if (isnull[attrnum] != slot->tts_isnull[attrnum]) + return false; + + /* + * If both are NULL, they can be considered equal. + */ + if (isnull[attrnum]) + continue; + + att = desc->attrs[attrnum]; + if (!datumIsEqual(values[attrnum], slot->tts_values[attrnum], + att->attbyval, att->attlen)) + return false; + } + + return true; +} + +/* + * Search the relation 'rel' for tuple using the sequential scan. + * + * If a matching tuple is found, lock it with lockmode, fill the slot with its + * contents, and return true. Return false otherwise. + * + * Note that this stops on the first matching tuple. + * + * This can obviously be quite slow on tables that have more than few rows. + */ +bool +RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode, + TupleTableSlot *searchslot, TupleTableSlot *outslot) +{ + HeapTuple scantuple; + HeapScanDesc scan; + SnapshotData snap; + TransactionId xwait; + bool found; + TupleDesc desc = RelationGetDescr(rel); + + Assert(equalTupleDescs(desc, outslot->tts_tupleDescriptor)); + + /* Start an index scan. */ + InitDirtySnapshot(snap); + scan = heap_beginscan(rel, &snap, 0, NULL); + +retry: + found = false; + + heap_rescan(scan, NULL); + + /* Try to find the tuple */ + while ((scantuple = heap_getnext(scan, ForwardScanDirection)) != NULL) + { + if (!tuple_equals_slot(desc, scantuple, searchslot)) + continue; + + found = true; + ExecStoreTuple(scantuple, outslot, InvalidBuffer, false); + ExecMaterializeSlot(outslot); + + xwait = TransactionIdIsValid(snap.xmin) ? + snap.xmin : snap.xmax; + + /* + * If the tuple is locked, wait for locking transaction to finish and + * retry. + */ + if (TransactionIdIsValid(xwait)) + { + XactLockTableWait(xwait, NULL, NULL, XLTW_None); + goto retry; + } + } + + /* Found tuple, try to lock it in the lockmode. */ + if (found) + { + Buffer buf; + HeapUpdateFailureData hufd; + HTSU_Result res; + HeapTupleData locktup; + + ItemPointerCopy(&outslot->tts_tuple->t_self, &locktup.t_self); + + PushActiveSnapshot(GetLatestSnapshot()); + + res = heap_lock_tuple(rel, &locktup, GetCurrentCommandId(false), + lockmode, + LockWaitBlock, + false /* don't follow updates */ , + &buf, &hufd); + /* the tuple slot already has the buffer pinned */ + ReleaseBuffer(buf); + + PopActiveSnapshot(); + + switch (res) + { + case HeapTupleMayBeUpdated: + break; + case HeapTupleUpdated: + /* XXX: Improve handling here */ + ereport(LOG, + (errcode(ERRCODE_T_R_SERIALIZATION_FAILURE), + errmsg("concurrent update, retrying"))); + goto retry; + case HeapTupleInvisible: + elog(ERROR, "attempted to lock invisible tuple"); + default: + elog(ERROR, "unexpected heap_lock_tuple status: %u", res); + break; + } + } + + heap_endscan(scan); + + return found; +} + +/* + * Insert tuple represented in the slot to the relation, update the indexes, + * and execute any constraints and per-row triggers. + * + * Caller is responsible for opening the indexes. + */ +void +ExecSimpleRelationInsert(EState *estate, TupleTableSlot *slot) +{ + bool skip_tuple = false; + HeapTuple tuple; + ResultRelInfo *resultRelInfo = estate->es_result_relation_info; + Relation rel = resultRelInfo->ri_RelationDesc; + + /* For now we support only tables. */ + Assert(rel->rd_rel->relkind == RELKIND_RELATION); + + CheckCmdReplicaIdentity(rel, CMD_INSERT); + + /* BEFORE ROW INSERT Triggers */ + if (resultRelInfo->ri_TrigDesc && + resultRelInfo->ri_TrigDesc->trig_insert_before_row) + { + slot = ExecBRInsertTriggers(estate, resultRelInfo, slot); + + if (slot == NULL) /* "do nothing" */ + skip_tuple = true; + } + + if (!skip_tuple) + { + List *recheckIndexes = NIL; + + /* Check the constraints of the tuple */ + if (rel->rd_att->constr) + ExecConstraints(resultRelInfo, slot, estate); + + /* Store the slot into tuple that we can inspect. */ + tuple = ExecMaterializeSlot(slot); + + /* OK, store the tuple and create index entries for it */ + simple_heap_insert(rel, tuple); + + if (resultRelInfo->ri_NumIndices > 0) + recheckIndexes = ExecInsertIndexTuples(slot, &(tuple->t_self), + estate, false, NULL, + NIL); + + /* AFTER ROW INSERT Triggers */ + ExecARInsertTriggers(estate, resultRelInfo, tuple, + recheckIndexes); + + list_free(recheckIndexes); + } +} + +/* + * Find the searchslot tuple and update it with data in the slot, + * update the indexes, and execute any constraints and per-row triggers. + * + * Caller is responsible for opening the indexes. + */ +void +ExecSimpleRelationUpdate(EState *estate, EPQState *epqstate, + TupleTableSlot *searchslot, TupleTableSlot *slot) +{ + bool skip_tuple = false; + HeapTuple tuple; + ResultRelInfo *resultRelInfo = estate->es_result_relation_info; + Relation rel = resultRelInfo->ri_RelationDesc; + + /* For now we support only tables. */ + Assert(rel->rd_rel->relkind == RELKIND_RELATION); + + CheckCmdReplicaIdentity(rel, CMD_UPDATE); + + /* BEFORE ROW INSERT Triggers */ + if (resultRelInfo->ri_TrigDesc && + resultRelInfo->ri_TrigDesc->trig_update_before_row) + { + slot = ExecBRUpdateTriggers(estate, epqstate, resultRelInfo, + &searchslot->tts_tuple->t_self, + NULL, slot); + + if (slot == NULL) /* "do nothing" */ + skip_tuple = true; + } + + if (!skip_tuple) + { + List *recheckIndexes = NIL; + + /* Check the constraints of the tuple */ + if (rel->rd_att->constr) + ExecConstraints(resultRelInfo, slot, estate); + + /* Store the slot into tuple that we can write. */ + tuple = ExecMaterializeSlot(slot); + + /* OK, update the tuple and index entries for it */ + simple_heap_update(rel, &searchslot->tts_tuple->t_self, + slot->tts_tuple); + + if (resultRelInfo->ri_NumIndices > 0 && + !HeapTupleIsHeapOnly(slot->tts_tuple)) + recheckIndexes = ExecInsertIndexTuples(slot, &(tuple->t_self), + estate, false, NULL, + NIL); + + /* AFTER ROW UPDATE Triggers */ + ExecARUpdateTriggers(estate, resultRelInfo, + &searchslot->tts_tuple->t_self, + NULL, tuple, recheckIndexes); + + list_free(recheckIndexes); + } +} + +/* + * Find the searchslot tuple and delete it, and execute any constraints + * and per-row triggers. + * + * Caller is responsible for opening the indexes. + */ +void +ExecSimpleRelationDelete(EState *estate, EPQState *epqstate, + TupleTableSlot *searchslot) +{ + bool skip_tuple = false; + ResultRelInfo *resultRelInfo = estate->es_result_relation_info; + Relation rel = resultRelInfo->ri_RelationDesc; + + /* For now we support only tables. */ + Assert(rel->rd_rel->relkind == RELKIND_RELATION); + + CheckCmdReplicaIdentity(rel, CMD_DELETE); + + /* BEFORE ROW INSERT Triggers */ + if (resultRelInfo->ri_TrigDesc && + resultRelInfo->ri_TrigDesc->trig_update_before_row) + { + skip_tuple = !ExecBRDeleteTriggers(estate, epqstate, resultRelInfo, + &searchslot->tts_tuple->t_self, + NULL); + } + + if (!skip_tuple) + { + List *recheckIndexes = NIL; + + /* OK, delete the tuple */ + simple_heap_delete(rel, &searchslot->tts_tuple->t_self); + + /* AFTER ROW DELETE Triggers */ + ExecARDeleteTriggers(estate, resultRelInfo, + &searchslot->tts_tuple->t_self, NULL); + + list_free(recheckIndexes); + } +} + +/* + * Check if command can be executed with current replica identity. + */ +void +CheckCmdReplicaIdentity(Relation rel, CmdType cmd) +{ + PublicationActions *pubactions; + + /* We only need to do checks for UPDATE and DELETE. */ + if (cmd != CMD_UPDATE && cmd != CMD_DELETE) + return; + + /* If relation has replica identity we are always good. */ + if (rel->rd_rel->relreplident == REPLICA_IDENTITY_FULL || + OidIsValid(RelationGetReplicaIndex(rel))) + return; + + /* + * This is either UPDATE OR DELETE and there is no replica identity. + * + * Check if the table publishes UPDATES or DELETES. + */ + pubactions = GetRelationPublicationActions(rel); + if (cmd == CMD_UPDATE && pubactions->pubupdate) + ereport(ERROR, + (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("cannot update table \"%s\" because it does not have replica identity and publishes updates", + RelationGetRelationName(rel)), + errhint("To enable updating the table, set REPLICA IDENTITY using ALTER TABLE."))); + else if (cmd == CMD_DELETE && pubactions->pubdelete) + ereport(ERROR, + (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("cannot delete from table \"%s\" because it does not have replica identity and publishes deletes", + RelationGetRelationName(rel)), + errhint("To enable deleting from the table, set REPLICA IDENTITY using ALTER TABLE."))); +} + + +/* + * Check if we support writing into specific relkind. + * + * The nspname and relname are only needed for error reporting. + */ +void +CheckSubscriptionRelkind(char relkind, const char *nspname, + const char *relname) +{ + /* + * We currently only support writing to regular tables. + */ + if (relkind != RELKIND_RELATION) + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("logical replication target relation \"%s.%s\" is not a table", + nspname, relname))); +} diff --git a/src/backend/executor/execSRF.c b/src/backend/executor/execSRF.c new file mode 100644 index 0000000000..077ac208c1 --- /dev/null +++ b/src/backend/executor/execSRF.c @@ -0,0 +1,928 @@ +/*------------------------------------------------------------------------- + * + * execSRF.c + * Routines implementing the API for set-returning functions + * + * This file serves nodeFunctionscan.c and nodeProjectSet.c, providing + * common code for calling set-returning functions according to the + * ReturnSetInfo API. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * + * IDENTIFICATION + * src/backend/executor/execSRF.c + * + *------------------------------------------------------------------------- + */ +#include "postgres.h" + +#include "access/htup_details.h" +#include "catalog/objectaccess.h" +#include "executor/execdebug.h" +#include "funcapi.h" +#include "miscadmin.h" +#include "nodes/nodeFuncs.h" +#include "parser/parse_coerce.h" +#include "pgstat.h" +#include "utils/acl.h" +#include "utils/builtins.h" +#include "utils/lsyscache.h" +#include "utils/memutils.h" +#include "utils/typcache.h" + + +/* static function decls */ +static void init_sexpr(Oid foid, Oid input_collation, Expr *node, + SetExprState *sexpr, PlanState *parent, + MemoryContext sexprCxt, bool allowSRF, bool needDescForSRF); +static void ShutdownSetExpr(Datum arg); +static void ExecEvalFuncArgs(FunctionCallInfo fcinfo, + List *argList, ExprContext *econtext); +static void ExecPrepareTuplestoreResult(SetExprState *sexpr, + ExprContext *econtext, + Tuplestorestate *resultStore, + TupleDesc resultDesc); +static void tupledesc_match(TupleDesc dst_tupdesc, TupleDesc src_tupdesc); + + +/* + * Prepare function call in FROM (ROWS FROM) for execution. + * + * This is used by nodeFunctionscan.c. + */ +SetExprState * +ExecInitTableFunctionResult(Expr *expr, + ExprContext *econtext, PlanState *parent) +{ + SetExprState *state = makeNode(SetExprState); + + state->funcReturnsSet = false; + state->expr = expr; + state->func.fn_oid = InvalidOid; + + /* + * Normally the passed expression tree will be a FuncExpr, since the + * grammar only allows a function call at the top level of a table + * function reference. However, if the function doesn't return set then + * the planner might have replaced the function call via constant-folding + * or inlining. So if we see any other kind of expression node, execute + * it via the general ExecEvalExpr() code. That code path will not + * support set-returning functions buried in the expression, though. + */ + if (IsA(expr, FuncExpr)) + { + FuncExpr *func = (FuncExpr *) expr; + + state->funcReturnsSet = func->funcretset; + state->args = ExecInitExprList(func->args, parent); + + init_sexpr(func->funcid, func->inputcollid, expr, state, parent, + econtext->ecxt_per_query_memory, func->funcretset, false); + } + else + { + state->elidedFuncState = ExecInitExpr(expr, parent); + } + + return state; +} + +/* + * ExecMakeTableFunctionResult + * + * Evaluate a table function, producing a materialized result in a Tuplestore + * object. + * + * This is used by nodeFunctionscan.c. + */ +Tuplestorestate * +ExecMakeTableFunctionResult(SetExprState *setexpr, + ExprContext *econtext, + MemoryContext argContext, + TupleDesc expectedDesc, + bool randomAccess) +{ + Tuplestorestate *tupstore = NULL; + TupleDesc tupdesc = NULL; + Oid funcrettype; + bool returnsTuple; + bool returnsSet = false; + FunctionCallInfoData fcinfo; + PgStat_FunctionCallUsage fcusage; + ReturnSetInfo rsinfo; + HeapTupleData tmptup; + MemoryContext callerContext; + MemoryContext oldcontext; + bool first_time = true; + + callerContext = CurrentMemoryContext; + + funcrettype = exprType((Node *) setexpr->expr); + + returnsTuple = type_is_rowtype(funcrettype); + + /* + * Prepare a resultinfo node for communication. We always do this even if + * not expecting a set result, so that we can pass expectedDesc. In the + * generic-expression case, the expression doesn't actually get to see the + * resultinfo, but set it up anyway because we use some of the fields as + * our own state variables. + */ + rsinfo.type = T_ReturnSetInfo; + rsinfo.econtext = econtext; + rsinfo.expectedDesc = expectedDesc; + rsinfo.allowedModes = (int) (SFRM_ValuePerCall | SFRM_Materialize | SFRM_Materialize_Preferred); + if (randomAccess) + rsinfo.allowedModes |= (int) SFRM_Materialize_Random; + rsinfo.returnMode = SFRM_ValuePerCall; + /* isDone is filled below */ + rsinfo.setResult = NULL; + rsinfo.setDesc = NULL; + + /* + * Normally the passed expression tree will be a SetExprState, since the + * grammar only allows a function call at the top level of a table + * function reference. However, if the function doesn't return set then + * the planner might have replaced the function call via constant-folding + * or inlining. So if we see any other kind of expression node, execute + * it via the general ExecEvalExpr() code; the only difference is that we + * don't get a chance to pass a special ReturnSetInfo to any functions + * buried in the expression. + */ + if (!setexpr->elidedFuncState) + { + /* + * This path is similar to ExecMakeFunctionResultSet. + */ + returnsSet = setexpr->funcReturnsSet; + InitFunctionCallInfoData(fcinfo, &(setexpr->func), + list_length(setexpr->args), + setexpr->fcinfo_data.fncollation, + NULL, (Node *) &rsinfo); + + /* + * Evaluate the function's argument list. + * + * We can't do this in the per-tuple context: the argument values + * would disappear when we reset that context in the inner loop. And + * the caller's CurrentMemoryContext is typically a query-lifespan + * context, so we don't want to leak memory there. We require the + * caller to pass a separate memory context that can be used for this, + * and can be reset each time through to avoid bloat. + */ + MemoryContextReset(argContext); + oldcontext = MemoryContextSwitchTo(argContext); + ExecEvalFuncArgs(&fcinfo, setexpr->args, econtext); + MemoryContextSwitchTo(oldcontext); + + /* + * If function is strict, and there are any NULL arguments, skip + * calling the function and act like it returned NULL (or an empty + * set, in the returns-set case). + */ + if (setexpr->func.fn_strict) + { + int i; + + for (i = 0; i < fcinfo.nargs; i++) + { + if (fcinfo.argnull[i]) + goto no_function_result; + } + } + } + else + { + /* Treat setexpr as a generic expression */ + InitFunctionCallInfoData(fcinfo, NULL, 0, InvalidOid, NULL, NULL); + } + + /* + * Switch to short-lived context for calling the function or expression. + */ + MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); + + /* + * Loop to handle the ValuePerCall protocol (which is also the same + * behavior needed in the generic ExecEvalExpr path). + */ + for (;;) + { + Datum result; + + CHECK_FOR_INTERRUPTS(); + + /* + * reset per-tuple memory context before each call of the function or + * expression. This cleans up any local memory the function may leak + * when called. + */ + ResetExprContext(econtext); + + /* Call the function or expression one time */ + if (!setexpr->elidedFuncState) + { + pgstat_init_function_usage(&fcinfo, &fcusage); + + fcinfo.isnull = false; + rsinfo.isDone = ExprSingleResult; + result = FunctionCallInvoke(&fcinfo); + + pgstat_end_function_usage(&fcusage, + rsinfo.isDone != ExprMultipleResult); + } + else + { + result = + ExecEvalExpr(setexpr->elidedFuncState, econtext, &fcinfo.isnull); + rsinfo.isDone = ExprSingleResult; + } + + /* Which protocol does function want to use? */ + if (rsinfo.returnMode == SFRM_ValuePerCall) + { + /* + * Check for end of result set. + */ + if (rsinfo.isDone == ExprEndResult) + break; + + /* + * If first time through, build tuplestore for result. For a + * scalar function result type, also make a suitable tupdesc. + */ + if (first_time) + { + oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + tupstore = tuplestore_begin_heap(randomAccess, false, work_mem); + rsinfo.setResult = tupstore; + if (!returnsTuple) + { + tupdesc = CreateTemplateTupleDesc(1, false); + TupleDescInitEntry(tupdesc, + (AttrNumber) 1, + "column", + funcrettype, + -1, + 0); + rsinfo.setDesc = tupdesc; + } + MemoryContextSwitchTo(oldcontext); + } + + /* + * Store current resultset item. + */ + if (returnsTuple) + { + if (!fcinfo.isnull) + { + HeapTupleHeader td = DatumGetHeapTupleHeader(result); + + if (tupdesc == NULL) + { + /* + * This is the first non-NULL result from the + * function. Use the type info embedded in the + * rowtype Datum to look up the needed tupdesc. Make + * a copy for the query. + */ + oldcontext = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + tupdesc = lookup_rowtype_tupdesc_copy(HeapTupleHeaderGetTypeId(td), + HeapTupleHeaderGetTypMod(td)); + rsinfo.setDesc = tupdesc; + MemoryContextSwitchTo(oldcontext); + } + else + { + /* + * Verify all later returned rows have same subtype; + * necessary in case the type is RECORD. + */ + if (HeapTupleHeaderGetTypeId(td) != tupdesc->tdtypeid || + HeapTupleHeaderGetTypMod(td) != tupdesc->tdtypmod) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("rows returned by function are not all of the same row type"))); + } + + /* + * tuplestore_puttuple needs a HeapTuple not a bare + * HeapTupleHeader, but it doesn't need all the fields. + */ + tmptup.t_len = HeapTupleHeaderGetDatumLength(td); + tmptup.t_data = td; + + tuplestore_puttuple(tupstore, &tmptup); + } + else + { + /* + * NULL result from a tuple-returning function; expand it + * to a row of all nulls. We rely on the expectedDesc to + * form such rows. (Note: this would be problematic if + * tuplestore_putvalues saved the tdtypeid/tdtypmod from + * the provided descriptor, since that might not match + * what we get from the function itself. But it doesn't.) + */ + int natts = expectedDesc->natts; + bool *nullflags; + + nullflags = (bool *) palloc(natts * sizeof(bool)); + memset(nullflags, true, natts * sizeof(bool)); + tuplestore_putvalues(tupstore, expectedDesc, NULL, nullflags); + } + } + else + { + /* Scalar-type case: just store the function result */ + tuplestore_putvalues(tupstore, tupdesc, &result, &fcinfo.isnull); + } + + /* + * Are we done? + */ + if (rsinfo.isDone != ExprMultipleResult) + break; + } + else if (rsinfo.returnMode == SFRM_Materialize) + { + /* check we're on the same page as the function author */ + if (!first_time || rsinfo.isDone != ExprSingleResult) + ereport(ERROR, + (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), + errmsg("table-function protocol for materialize mode was not followed"))); + /* Done evaluating the set result */ + break; + } + else + ereport(ERROR, + (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), + errmsg("unrecognized table-function returnMode: %d", + (int) rsinfo.returnMode))); + + first_time = false; + } + +no_function_result: + + /* + * If we got nothing from the function (ie, an empty-set or NULL result), + * we have to create the tuplestore to return, and if it's a + * non-set-returning function then insert a single all-nulls row. As + * above, we depend on the expectedDesc to manufacture the dummy row. + */ + if (rsinfo.setResult == NULL) + { + MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + tupstore = tuplestore_begin_heap(randomAccess, false, work_mem); + rsinfo.setResult = tupstore; + if (!returnsSet) + { + int natts = expectedDesc->natts; + bool *nullflags; + + MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); + nullflags = (bool *) palloc(natts * sizeof(bool)); + memset(nullflags, true, natts * sizeof(bool)); + tuplestore_putvalues(tupstore, expectedDesc, NULL, nullflags); + } + } + + /* + * If function provided a tupdesc, cross-check it. We only really need to + * do this for functions returning RECORD, but might as well do it always. + */ + if (rsinfo.setDesc) + { + tupledesc_match(expectedDesc, rsinfo.setDesc); + + /* + * If it is a dynamically-allocated TupleDesc, free it: it is + * typically allocated in a per-query context, so we must avoid + * leaking it across multiple usages. + */ + if (rsinfo.setDesc->tdrefcount == -1) + FreeTupleDesc(rsinfo.setDesc); + } + + MemoryContextSwitchTo(callerContext); + + /* All done, pass back the tuplestore */ + return rsinfo.setResult; +} + + +/* + * Prepare targetlist SRF function call for execution. + * + * This is used by nodeProjectSet.c. + */ +SetExprState * +ExecInitFunctionResultSet(Expr *expr, + ExprContext *econtext, PlanState *parent) +{ + SetExprState *state = makeNode(SetExprState); + + state->funcReturnsSet = true; + state->expr = expr; + state->func.fn_oid = InvalidOid; + + /* + * Initialize metadata. The expression node could be either a FuncExpr or + * an OpExpr. + */ + if (IsA(expr, FuncExpr)) + { + FuncExpr *func = (FuncExpr *) expr; + + state->args = ExecInitExprList(func->args, parent); + init_sexpr(func->funcid, func->inputcollid, expr, state, parent, + econtext->ecxt_per_query_memory, true, true); + } + else if (IsA(expr, OpExpr)) + { + OpExpr *op = (OpExpr *) expr; + + state->args = ExecInitExprList(op->args, parent); + init_sexpr(op->opfuncid, op->inputcollid, expr, state, parent, + econtext->ecxt_per_query_memory, true, true); + } + else + elog(ERROR, "unrecognized node type: %d", + (int) nodeTag(expr)); + + /* shouldn't get here unless the selected function returns set */ + Assert(state->func.fn_retset); + + return state; +} + +/* + * ExecMakeFunctionResultSet + * + * Evaluate the arguments to a set-returning function and then call the + * function itself. The argument expressions may not contain set-returning + * functions (the planner is supposed to have separated evaluation for those). + * + * This is used by nodeProjectSet.c. + */ +Datum +ExecMakeFunctionResultSet(SetExprState *fcache, + ExprContext *econtext, + bool *isNull, + ExprDoneCond *isDone) +{ + List *arguments; + Datum result; + FunctionCallInfo fcinfo; + PgStat_FunctionCallUsage fcusage; + ReturnSetInfo rsinfo; + bool callit; + int i; + +restart: + + /* Guard against stack overflow due to overly complex expressions */ + check_stack_depth(); + + /* + * If a previous call of the function returned a set result in the form of + * a tuplestore, continue reading rows from the tuplestore until it's + * empty. + */ + if (fcache->funcResultStore) + { + if (tuplestore_gettupleslot(fcache->funcResultStore, true, false, + fcache->funcResultSlot)) + { + *isDone = ExprMultipleResult; + if (fcache->funcReturnsTuple) + { + /* We must return the whole tuple as a Datum. */ + *isNull = false; + return ExecFetchSlotTupleDatum(fcache->funcResultSlot); + } + else + { + /* Extract the first column and return it as a scalar. */ + return slot_getattr(fcache->funcResultSlot, 1, isNull); + } + } + /* Exhausted the tuplestore, so clean up */ + tuplestore_end(fcache->funcResultStore); + fcache->funcResultStore = NULL; + *isDone = ExprEndResult; + *isNull = true; + return (Datum) 0; + } + + /* + * arguments is a list of expressions to evaluate before passing to the + * function manager. We skip the evaluation if it was already done in the + * previous call (ie, we are continuing the evaluation of a set-valued + * function). Otherwise, collect the current argument values into fcinfo. + */ + fcinfo = &fcache->fcinfo_data; + arguments = fcache->args; + if (!fcache->setArgsValid) + ExecEvalFuncArgs(fcinfo, arguments, econtext); + else + { + /* Reset flag (we may set it again below) */ + fcache->setArgsValid = false; + } + + /* + * Now call the function, passing the evaluated parameter values. + */ + + /* Prepare a resultinfo node for communication. */ + fcinfo->resultinfo = (Node *) &rsinfo; + rsinfo.type = T_ReturnSetInfo; + rsinfo.econtext = econtext; + rsinfo.expectedDesc = fcache->funcResultDesc; + rsinfo.allowedModes = (int) (SFRM_ValuePerCall | SFRM_Materialize); + /* note we do not set SFRM_Materialize_Random or _Preferred */ + rsinfo.returnMode = SFRM_ValuePerCall; + /* isDone is filled below */ + rsinfo.setResult = NULL; + rsinfo.setDesc = NULL; + + /* + * If function is strict, and there are any NULL arguments, skip calling + * the function. + */ + callit = true; + if (fcache->func.fn_strict) + { + for (i = 0; i < fcinfo->nargs; i++) + { + if (fcinfo->argnull[i]) + { + callit = false; + break; + } + } + } + + if (callit) + { + pgstat_init_function_usage(fcinfo, &fcusage); + + fcinfo->isnull = false; + rsinfo.isDone = ExprSingleResult; + result = FunctionCallInvoke(fcinfo); + *isNull = fcinfo->isnull; + *isDone = rsinfo.isDone; + + pgstat_end_function_usage(&fcusage, + rsinfo.isDone != ExprMultipleResult); + } + else + { + /* for a strict SRF, result for NULL is an empty set */ + result = (Datum) 0; + *isNull = true; + *isDone = ExprEndResult; + } + + /* Which protocol does function want to use? */ + if (rsinfo.returnMode == SFRM_ValuePerCall) + { + if (*isDone != ExprEndResult) + { + /* + * Save the current argument values to re-use on the next call. + */ + if (*isDone == ExprMultipleResult) + { + fcache->setArgsValid = true; + /* Register cleanup callback if we didn't already */ + if (!fcache->shutdown_reg) + { + RegisterExprContextCallback(econtext, + ShutdownSetExpr, + PointerGetDatum(fcache)); + fcache->shutdown_reg = true; + } + } + } + } + else if (rsinfo.returnMode == SFRM_Materialize) + { + /* check we're on the same page as the function author */ + if (rsinfo.isDone != ExprSingleResult) + ereport(ERROR, + (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), + errmsg("table-function protocol for materialize mode was not followed"))); + if (rsinfo.setResult != NULL) + { + /* prepare to return values from the tuplestore */ + ExecPrepareTuplestoreResult(fcache, econtext, + rsinfo.setResult, + rsinfo.setDesc); + /* loop back to top to start returning from tuplestore */ + goto restart; + } + /* if setResult was left null, treat it as empty set */ + *isDone = ExprEndResult; + *isNull = true; + result = (Datum) 0; + } + else + ereport(ERROR, + (errcode(ERRCODE_E_R_I_E_SRF_PROTOCOL_VIOLATED), + errmsg("unrecognized table-function returnMode: %d", + (int) rsinfo.returnMode))); + + return result; +} + + +/* + * init_sexpr - initialize a SetExprState node during first use + */ +static void +init_sexpr(Oid foid, Oid input_collation, Expr *node, + SetExprState *sexpr, PlanState *parent, + MemoryContext sexprCxt, bool allowSRF, bool needDescForSRF) +{ + AclResult aclresult; + + /* Check permission to call function */ + aclresult = pg_proc_aclcheck(foid, GetUserId(), ACL_EXECUTE); + if (aclresult != ACLCHECK_OK) + aclcheck_error(aclresult, ACL_KIND_PROC, get_func_name(foid)); + InvokeFunctionExecuteHook(foid); + + /* + * Safety check on nargs. Under normal circumstances this should never + * fail, as parser should check sooner. But possibly it might fail if + * server has been compiled with FUNC_MAX_ARGS smaller than some functions + * declared in pg_proc? + */ + if (list_length(sexpr->args) > FUNC_MAX_ARGS) + ereport(ERROR, + (errcode(ERRCODE_TOO_MANY_ARGUMENTS), + errmsg_plural("cannot pass more than %d argument to a function", + "cannot pass more than %d arguments to a function", + FUNC_MAX_ARGS, + FUNC_MAX_ARGS))); + + /* Set up the primary fmgr lookup information */ + fmgr_info_cxt(foid, &(sexpr->func), sexprCxt); + fmgr_info_set_expr((Node *) sexpr->expr, &(sexpr->func)); + + /* Initialize the function call parameter struct as well */ + InitFunctionCallInfoData(sexpr->fcinfo_data, &(sexpr->func), + list_length(sexpr->args), + input_collation, NULL, NULL); + + /* If function returns set, check if that's allowed by caller */ + if (sexpr->func.fn_retset && !allowSRF) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("set-valued function called in context that cannot accept a set"), + parent ? executor_errposition(parent->state, + exprLocation((Node *) node)) : 0)); + + /* Otherwise, caller should have marked the sexpr correctly */ + Assert(sexpr->func.fn_retset == sexpr->funcReturnsSet); + + /* If function returns set, prepare expected tuple descriptor */ + if (sexpr->func.fn_retset && needDescForSRF) + { + TypeFuncClass functypclass; + Oid funcrettype; + TupleDesc tupdesc; + MemoryContext oldcontext; + + functypclass = get_expr_result_type(sexpr->func.fn_expr, + &funcrettype, + &tupdesc); + + /* Must save tupdesc in sexpr's context */ + oldcontext = MemoryContextSwitchTo(sexprCxt); + + if (functypclass == TYPEFUNC_COMPOSITE) + { + /* Composite data type, e.g. a table's row type */ + Assert(tupdesc); + /* Must copy it out of typcache for safety */ + sexpr->funcResultDesc = CreateTupleDescCopy(tupdesc); + sexpr->funcReturnsTuple = true; + } + else if (functypclass == TYPEFUNC_SCALAR) + { + /* Base data type, i.e. scalar */ + tupdesc = CreateTemplateTupleDesc(1, false); + TupleDescInitEntry(tupdesc, + (AttrNumber) 1, + NULL, + funcrettype, + -1, + 0); + sexpr->funcResultDesc = tupdesc; + sexpr->funcReturnsTuple = false; + } + else if (functypclass == TYPEFUNC_RECORD) + { + /* This will work if function doesn't need an expectedDesc */ + sexpr->funcResultDesc = NULL; + sexpr->funcReturnsTuple = true; + } + else + { + /* Else, we will fail if function needs an expectedDesc */ + sexpr->funcResultDesc = NULL; + } + + MemoryContextSwitchTo(oldcontext); + } + else + sexpr->funcResultDesc = NULL; + + /* Initialize additional state */ + sexpr->funcResultStore = NULL; + sexpr->funcResultSlot = NULL; + sexpr->shutdown_reg = false; +} + +/* + * callback function in case a SetExprState needs to be shut down before it + * has been run to completion + */ +static void +ShutdownSetExpr(Datum arg) +{ + SetExprState *sexpr = castNode(SetExprState, DatumGetPointer(arg)); + + /* If we have a slot, make sure it's let go of any tuplestore pointer */ + if (sexpr->funcResultSlot) + ExecClearTuple(sexpr->funcResultSlot); + + /* Release any open tuplestore */ + if (sexpr->funcResultStore) + tuplestore_end(sexpr->funcResultStore); + sexpr->funcResultStore = NULL; + + /* Clear any active set-argument state */ + sexpr->setArgsValid = false; + + /* execUtils will deregister the callback... */ + sexpr->shutdown_reg = false; +} + +/* + * Evaluate arguments for a function. + */ +static void +ExecEvalFuncArgs(FunctionCallInfo fcinfo, + List *argList, + ExprContext *econtext) +{ + int i; + ListCell *arg; + + i = 0; + foreach(arg, argList) + { + ExprState *argstate = (ExprState *) lfirst(arg); + + fcinfo->arg[i] = ExecEvalExpr(argstate, + econtext, + &fcinfo->argnull[i]); + i++; + } + + Assert(i == fcinfo->nargs); +} + +/* + * ExecPrepareTuplestoreResult + * + * Subroutine for ExecMakeFunctionResultSet: prepare to extract rows from a + * tuplestore function result. We must set up a funcResultSlot (unless + * already done in a previous call cycle) and verify that the function + * returned the expected tuple descriptor. + */ +static void +ExecPrepareTuplestoreResult(SetExprState *sexpr, + ExprContext *econtext, + Tuplestorestate *resultStore, + TupleDesc resultDesc) +{ + sexpr->funcResultStore = resultStore; + + if (sexpr->funcResultSlot == NULL) + { + /* Create a slot so we can read data out of the tuplestore */ + TupleDesc slotDesc; + MemoryContext oldcontext; + + oldcontext = MemoryContextSwitchTo(sexpr->func.fn_mcxt); + + /* + * If we were not able to determine the result rowtype from context, + * and the function didn't return a tupdesc, we have to fail. + */ + if (sexpr->funcResultDesc) + slotDesc = sexpr->funcResultDesc; + else if (resultDesc) + { + /* don't assume resultDesc is long-lived */ + slotDesc = CreateTupleDescCopy(resultDesc); + } + else + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("function returning setof record called in " + "context that cannot accept type record"))); + slotDesc = NULL; /* keep compiler quiet */ + } + + sexpr->funcResultSlot = MakeSingleTupleTableSlot(slotDesc); + MemoryContextSwitchTo(oldcontext); + } + + /* + * If function provided a tupdesc, cross-check it. We only really need to + * do this for functions returning RECORD, but might as well do it always. + */ + if (resultDesc) + { + if (sexpr->funcResultDesc) + tupledesc_match(sexpr->funcResultDesc, resultDesc); + + /* + * If it is a dynamically-allocated TupleDesc, free it: it is + * typically allocated in a per-query context, so we must avoid + * leaking it across multiple usages. + */ + if (resultDesc->tdrefcount == -1) + FreeTupleDesc(resultDesc); + } + + /* Register cleanup callback if we didn't already */ + if (!sexpr->shutdown_reg) + { + RegisterExprContextCallback(econtext, + ShutdownSetExpr, + PointerGetDatum(sexpr)); + sexpr->shutdown_reg = true; + } +} + +/* + * Check that function result tuple type (src_tupdesc) matches or can + * be considered to match what the query expects (dst_tupdesc). If + * they don't match, ereport. + * + * We really only care about number of attributes and data type. + * Also, we can ignore type mismatch on columns that are dropped in the + * destination type, so long as the physical storage matches. This is + * helpful in some cases involving out-of-date cached plans. + */ +static void +tupledesc_match(TupleDesc dst_tupdesc, TupleDesc src_tupdesc) +{ + int i; + + if (dst_tupdesc->natts != src_tupdesc->natts) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("function return row and query-specified return row do not match"), + errdetail_plural("Returned row contains %d attribute, but query expects %d.", + "Returned row contains %d attributes, but query expects %d.", + src_tupdesc->natts, + src_tupdesc->natts, dst_tupdesc->natts))); + + for (i = 0; i < dst_tupdesc->natts; i++) + { + Form_pg_attribute dattr = dst_tupdesc->attrs[i]; + Form_pg_attribute sattr = src_tupdesc->attrs[i]; + + if (IsBinaryCoercible(sattr->atttypid, dattr->atttypid)) + continue; /* no worries */ + if (!dattr->attisdropped) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("function return row and query-specified return row do not match"), + errdetail("Returned type %s at ordinal position %d, but query expects %s.", + format_type_be(sattr->atttypid), + i + 1, + format_type_be(dattr->atttypid)))); + + if (dattr->attlen != sattr->attlen || + dattr->attalign != sattr->attalign) + ereport(ERROR, + (errcode(ERRCODE_DATATYPE_MISMATCH), + errmsg("function return row and query-specified return row do not match"), + errdetail("Physical storage mismatch on dropped attribute at ordinal position %d.", + i + 1))); + } +} diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c index fb0013dc53..4f131b3ee0 100644 --- a/src/backend/executor/execScan.c +++ b/src/backend/executor/execScan.c @@ -7,7 +7,7 @@ * stuff - checking the qualification and projecting the tuple * appropriately. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -123,10 +123,8 @@ ExecScan(ScanState *node, ExecScanRecheckMtd recheckMtd) { ExprContext *econtext; - List *qual; + ExprState *qual; ProjectionInfo *projInfo; - ExprDoneCond isDone; - TupleTableSlot *resultSlot; /* * Fetch data from node @@ -146,24 +144,8 @@ ExecScan(ScanState *node, } /* - * Check to see if we're still projecting out tuples from a previous scan - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->ps.ps_TupFromTlist) - { - Assert(projInfo); /* can't get here if not projecting */ - resultSlot = ExecProject(projInfo, &isDone); - if (isDone == ExprMultipleResult) - return resultSlot; - /* Done with that source tuple... */ - node->ps.ps_TupFromTlist = false; - } - - /* * Reset per-tuple memory context to free any expression evaluation - * storage allocated in the previous tuple cycle. Note this can't happen - * until we're done projecting out tuples from a scan tuple. + * storage allocated in the previous tuple cycle. */ ResetExprContext(econtext); @@ -188,7 +170,7 @@ ExecScan(ScanState *node, if (TupIsNull(slot)) { if (projInfo) - return ExecClearTuple(projInfo->pi_slot); + return ExecClearTuple(projInfo->pi_state.resultslot); else return slot; } @@ -201,11 +183,11 @@ ExecScan(ScanState *node, /* * check that the current tuple satisfies the qual-clause * - * check for non-nil qual here to avoid a function call to ExecQual() - * when the qual is nil ... saves only a few cycles, but they add up + * check for non-null qual here to avoid a function call to ExecQual() + * when the qual is null ... saves only a few cycles, but they add up * ... */ - if (!qual || ExecQual(qual, econtext, false)) + if (qual == NULL || ExecQual(qual, econtext)) { /* * Found a satisfactory scan tuple. @@ -214,15 +196,9 @@ ExecScan(ScanState *node, { /* * Form a projection tuple, store it in the result tuple slot - * and return it --- unless we find we can project no tuples - * from this scan tuple, in which case continue scan. + * and return it. */ - resultSlot = ExecProject(projInfo, &isDone); - if (isDone != ExprEndResult) - { - node->ps.ps_TupFromTlist = (isDone == ExprMultipleResult); - return resultSlot; - } + return ExecProject(projInfo); } else { @@ -352,9 +328,6 @@ ExecScanReScan(ScanState *node) { EState *estate = node->ps.state; - /* Stop projecting any tuples from SRFs in the targetlist */ - node->ps.ps_TupFromTlist = false; - /* Rescan EvalPlanQual tuple if we're inside an EvalPlanQual recheck */ if (estate->es_epqScanDone != NULL) { diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c index 63375dc825..489ca5edb9 100644 --- a/src/backend/executor/execTuples.c +++ b/src/backend/executor/execTuples.c @@ -13,7 +13,7 @@ * (getattribute, formtuple, etc.). * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -170,10 +170,7 @@ ExecResetTupleTable(List *tupleTable, /* tuple table */ foreach(lc, tupleTable) { - TupleTableSlot *slot = (TupleTableSlot *) lfirst(lc); - - /* Sanity checks */ - Assert(IsA(slot, TupleTableSlot)); + TupleTableSlot *slot = lfirst_node(TupleTableSlot, lc); /* Always release resources and reset the slot to empty */ ExecClearTuple(slot); diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c index 4bdf76cf56..b1178552e5 100644 --- a/src/backend/executor/execUtils.c +++ b/src/backend/executor/execUtils.c @@ -4,7 +4,7 @@ * miscellaneous executor utility routines * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -29,9 +29,14 @@ * ExecOpenScanRelation Common code for scan node init routines. * ExecCloseScanRelation * + * executor_errposition Report syntactic position of an error. + * * RegisterExprContextCallback Register function shutdown callback * UnregisterExprContextCallback Deregister function shutdown callback * + * GetAttributeByName Runtime extraction of columns from tuples. + * GetAttributeByNum + * * NOTES * This file has traditionally been the place to stick misc. * executor support stuff that doesn't really go anyplace else. @@ -42,13 +47,16 @@ #include "access/relscan.h" #include "access/transam.h" #include "executor/executor.h" +#include "mb/pg_wchar.h" #include "nodes/nodeFuncs.h" #include "parser/parsetree.h" +#include "storage/lmgr.h" +#include "utils/builtins.h" #include "utils/memutils.h" #include "utils/rel.h" +#include "utils/typcache.h" -static bool get_last_attnums(Node *node, ProjectionInfo *projInfo); static void ShutdownExprContext(ExprContext *econtext, bool isCommit); @@ -81,9 +89,7 @@ CreateExecutorState(void) */ qcontext = AllocSetContextCreate(CurrentMemoryContext, "ExecutorState", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* * Make the EState node within the per-query context. This way, we don't @@ -118,6 +124,8 @@ CreateExecutorState(void) estate->es_param_list_info = NULL; estate->es_param_exec_vals = NULL; + estate->es_queryEnv = NULL; + estate->es_query_cxt = qcontext; estate->es_tupleTable = NIL; @@ -142,6 +150,7 @@ CreateExecutorState(void) estate->es_epqTuple = NULL; estate->es_epqTupleSet = NULL; estate->es_epqScanDone = NULL; + estate->es_sourceText = NULL; /* * Return the executor state structure @@ -230,9 +239,7 @@ CreateExprContext(EState *estate) econtext->ecxt_per_tuple_memory = AllocSetContextCreate(estate->es_query_cxt, "ExprContext", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); econtext->ecxt_param_exec_vals = estate->es_param_exec_vals; econtext->ecxt_param_list_info = estate->es_param_list_info; @@ -301,9 +308,7 @@ CreateStandaloneExprContext(void) econtext->ecxt_per_tuple_memory = AllocSetContextCreate(CurrentMemoryContext, "ExprContext", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); econtext->ecxt_param_exec_vals = NULL; econtext->ecxt_param_list_info = NULL; @@ -469,192 +474,6 @@ ExecGetResultType(PlanState *planstate) return slot->tts_tupleDescriptor; } -/* ---------------- - * ExecBuildProjectionInfo - * - * Build a ProjectionInfo node for evaluating the given tlist in the given - * econtext, and storing the result into the tuple slot. (Caller must have - * ensured that tuple slot has a descriptor matching the tlist!) Note that - * the given tlist should be a list of ExprState nodes, not Expr nodes. - * - * inputDesc can be NULL, but if it is not, we check to see whether simple - * Vars in the tlist match the descriptor. It is important to provide - * inputDesc for relation-scan plan nodes, as a cross check that the relation - * hasn't been changed since the plan was made. At higher levels of a plan, - * there is no need to recheck. - * ---------------- - */ -ProjectionInfo * -ExecBuildProjectionInfo(List *targetList, - ExprContext *econtext, - TupleTableSlot *slot, - TupleDesc inputDesc) -{ - ProjectionInfo *projInfo = makeNode(ProjectionInfo); - int len = ExecTargetListLength(targetList); - int *workspace; - int *varSlotOffsets; - int *varNumbers; - int *varOutputCols; - List *exprlist; - int numSimpleVars; - bool directMap; - ListCell *tl; - - projInfo->pi_exprContext = econtext; - projInfo->pi_slot = slot; - /* since these are all int arrays, we need do just one palloc */ - workspace = (int *) palloc(len * 3 * sizeof(int)); - projInfo->pi_varSlotOffsets = varSlotOffsets = workspace; - projInfo->pi_varNumbers = varNumbers = workspace + len; - projInfo->pi_varOutputCols = varOutputCols = workspace + len * 2; - projInfo->pi_lastInnerVar = 0; - projInfo->pi_lastOuterVar = 0; - projInfo->pi_lastScanVar = 0; - - /* - * We separate the target list elements into simple Var references and - * expressions which require the full ExecTargetList machinery. To be a - * simple Var, a Var has to be a user attribute and not mismatch the - * inputDesc. (Note: if there is a type mismatch then ExecEvalScalarVar - * will probably throw an error at runtime, but we leave that to it.) - */ - exprlist = NIL; - numSimpleVars = 0; - directMap = true; - foreach(tl, targetList) - { - GenericExprState *gstate = (GenericExprState *) lfirst(tl); - Var *variable = (Var *) gstate->arg->expr; - bool isSimpleVar = false; - - if (variable != NULL && - IsA(variable, Var) && - variable->varattno > 0) - { - if (!inputDesc) - isSimpleVar = true; /* can't check type, assume OK */ - else if (variable->varattno <= inputDesc->natts) - { - Form_pg_attribute attr; - - attr = inputDesc->attrs[variable->varattno - 1]; - if (!attr->attisdropped && variable->vartype == attr->atttypid) - isSimpleVar = true; - } - } - - if (isSimpleVar) - { - TargetEntry *tle = (TargetEntry *) gstate->xprstate.expr; - AttrNumber attnum = variable->varattno; - - varNumbers[numSimpleVars] = attnum; - varOutputCols[numSimpleVars] = tle->resno; - if (tle->resno != numSimpleVars + 1) - directMap = false; - - switch (variable->varno) - { - case INNER_VAR: - varSlotOffsets[numSimpleVars] = offsetof(ExprContext, - ecxt_innertuple); - if (projInfo->pi_lastInnerVar < attnum) - projInfo->pi_lastInnerVar = attnum; - break; - - case OUTER_VAR: - varSlotOffsets[numSimpleVars] = offsetof(ExprContext, - ecxt_outertuple); - if (projInfo->pi_lastOuterVar < attnum) - projInfo->pi_lastOuterVar = attnum; - break; - - /* INDEX_VAR is handled by default case */ - - default: - varSlotOffsets[numSimpleVars] = offsetof(ExprContext, - ecxt_scantuple); - if (projInfo->pi_lastScanVar < attnum) - projInfo->pi_lastScanVar = attnum; - break; - } - numSimpleVars++; - } - else - { - /* Not a simple variable, add it to generic targetlist */ - exprlist = lappend(exprlist, gstate); - /* Examine expr to include contained Vars in lastXXXVar counts */ - get_last_attnums((Node *) variable, projInfo); - } - } - projInfo->pi_targetlist = exprlist; - projInfo->pi_numSimpleVars = numSimpleVars; - projInfo->pi_directMap = directMap; - - if (exprlist == NIL) - projInfo->pi_itemIsDone = NULL; /* not needed */ - else - projInfo->pi_itemIsDone = (ExprDoneCond *) - palloc(len * sizeof(ExprDoneCond)); - - return projInfo; -} - -/* - * get_last_attnums: expression walker for ExecBuildProjectionInfo - * - * Update the lastXXXVar counts to be at least as large as the largest - * attribute numbers found in the expression - */ -static bool -get_last_attnums(Node *node, ProjectionInfo *projInfo) -{ - if (node == NULL) - return false; - if (IsA(node, Var)) - { - Var *variable = (Var *) node; - AttrNumber attnum = variable->varattno; - - switch (variable->varno) - { - case INNER_VAR: - if (projInfo->pi_lastInnerVar < attnum) - projInfo->pi_lastInnerVar = attnum; - break; - - case OUTER_VAR: - if (projInfo->pi_lastOuterVar < attnum) - projInfo->pi_lastOuterVar = attnum; - break; - - /* INDEX_VAR is handled by default case */ - - default: - if (projInfo->pi_lastScanVar < attnum) - projInfo->pi_lastScanVar = attnum; - break; - } - return false; - } - - /* - * Don't examine the arguments or filters of Aggrefs or WindowFuncs, - * because those do not represent expressions to be evaluated within the - * overall targetlist's econtext. GroupingFunc arguments are never - * evaluated at all. - */ - if (IsA(node, Aggref)) - return false; - if (IsA(node, WindowFunc)) - return false; - if (IsA(node, GroupingFunc)) - return false; - return expression_tree_walker(node, get_last_attnums, - (void *) projInfo); -} /* ---------------- * ExecAssignProjectionInfo @@ -670,9 +489,10 @@ ExecAssignProjectionInfo(PlanState *planstate, TupleDesc inputDesc) { planstate->ps_ProjInfo = - ExecBuildProjectionInfo(planstate->targetlist, + ExecBuildProjectionInfo(planstate->plan->targetlist, planstate->ps_ExprContext, planstate->ps_ResultTupleSlot, + planstate, inputDesc); } @@ -870,6 +690,36 @@ UpdateChangedParamSet(PlanState *node, Bitmapset *newchg) } /* + * executor_errposition + * Report an execution-time cursor position, if possible. + * + * This is expected to be used within an ereport() call. The return value + * is a dummy (always 0, in fact). + * + * The locations stored in parsetrees are byte offsets into the source string. + * We have to convert them to 1-based character indexes for reporting to + * clients. (We do things this way to avoid unnecessary overhead in the + * normal non-error case: computing character indexes would be much more + * expensive than storing token offsets.) + */ +int +executor_errposition(EState *estate, int location) +{ + int pos; + + /* No-op if location was not provided */ + if (location < 0) + return 0; + /* Can't do anything if source text is not available */ + if (estate == NULL || estate->es_sourceText == NULL) + return 0; + /* Convert offset to character number */ + pos = pg_mbstrlen_with_len(estate->es_sourceText, location) + 1; + /* And pass it to the ereport mechanism */ + return errposition(pos); +} + +/* * Register a shutdown callback in an ExprContext. * * Shutdown callbacks will be called (in reverse order of registration) @@ -965,3 +815,207 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit) MemoryContextSwitchTo(oldcontext); } + +/* + * ExecLockNonLeafAppendTables + * + * Locks, if necessary, the tables indicated by the RT indexes contained in + * the partitioned_rels list. These are the non-leaf tables in the partition + * tree controlled by a given Append or MergeAppend node. + */ +void +ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate) +{ + PlannedStmt *stmt = estate->es_plannedstmt; + ListCell *lc; + + foreach(lc, partitioned_rels) + { + ListCell *l; + Index rti = lfirst_int(lc); + bool is_result_rel = false; + Oid relid = getrelid(rti, estate->es_range_table); + + /* If this is a result relation, already locked in InitPlan */ + foreach(l, stmt->nonleafResultRelations) + { + if (rti == lfirst_int(l)) + { + is_result_rel = true; + break; + } + } + + /* + * Not a result relation; check if there is a RowMark that requires + * taking a RowShareLock on this rel. + */ + if (!is_result_rel) + { + PlanRowMark *rc = NULL; + + foreach(l, stmt->rowMarks) + { + if (((PlanRowMark *) lfirst(l))->rti == rti) + { + rc = lfirst(l); + break; + } + } + + if (rc && RowMarkRequiresRowShareLock(rc->markType)) + LockRelationOid(relid, RowShareLock); + else + LockRelationOid(relid, AccessShareLock); + } + } +} + +/* + * GetAttributeByName + * GetAttributeByNum + * + * These functions return the value of the requested attribute + * out of the given tuple Datum. + * C functions which take a tuple as an argument are expected + * to use these. Ex: overpaid(EMP) might call GetAttributeByNum(). + * Note: these are actually rather slow because they do a typcache + * lookup on each call. + */ +Datum +GetAttributeByName(HeapTupleHeader tuple, const char *attname, bool *isNull) +{ + AttrNumber attrno; + Datum result; + Oid tupType; + int32 tupTypmod; + TupleDesc tupDesc; + HeapTupleData tmptup; + int i; + + if (attname == NULL) + elog(ERROR, "invalid attribute name"); + + if (isNull == NULL) + elog(ERROR, "a NULL isNull pointer was passed"); + + if (tuple == NULL) + { + /* Kinda bogus but compatible with old behavior... */ + *isNull = true; + return (Datum) 0; + } + + tupType = HeapTupleHeaderGetTypeId(tuple); + tupTypmod = HeapTupleHeaderGetTypMod(tuple); + tupDesc = lookup_rowtype_tupdesc(tupType, tupTypmod); + + attrno = InvalidAttrNumber; + for (i = 0; i < tupDesc->natts; i++) + { + if (namestrcmp(&(tupDesc->attrs[i]->attname), attname) == 0) + { + attrno = tupDesc->attrs[i]->attnum; + break; + } + } + + if (attrno == InvalidAttrNumber) + elog(ERROR, "attribute \"%s\" does not exist", attname); + + /* + * heap_getattr needs a HeapTuple not a bare HeapTupleHeader. We set all + * the fields in the struct just in case user tries to inspect system + * columns. + */ + tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); + ItemPointerSetInvalid(&(tmptup.t_self)); + tmptup.t_tableOid = InvalidOid; + tmptup.t_data = tuple; + + result = heap_getattr(&tmptup, + attrno, + tupDesc, + isNull); + + ReleaseTupleDesc(tupDesc); + + return result; +} + +Datum +GetAttributeByNum(HeapTupleHeader tuple, + AttrNumber attrno, + bool *isNull) +{ + Datum result; + Oid tupType; + int32 tupTypmod; + TupleDesc tupDesc; + HeapTupleData tmptup; + + if (!AttributeNumberIsValid(attrno)) + elog(ERROR, "invalid attribute number %d", attrno); + + if (isNull == NULL) + elog(ERROR, "a NULL isNull pointer was passed"); + + if (tuple == NULL) + { + /* Kinda bogus but compatible with old behavior... */ + *isNull = true; + return (Datum) 0; + } + + tupType = HeapTupleHeaderGetTypeId(tuple); + tupTypmod = HeapTupleHeaderGetTypMod(tuple); + tupDesc = lookup_rowtype_tupdesc(tupType, tupTypmod); + + /* + * heap_getattr needs a HeapTuple not a bare HeapTupleHeader. We set all + * the fields in the struct just in case user tries to inspect system + * columns. + */ + tmptup.t_len = HeapTupleHeaderGetDatumLength(tuple); + ItemPointerSetInvalid(&(tmptup.t_self)); + tmptup.t_tableOid = InvalidOid; + tmptup.t_data = tuple; + + result = heap_getattr(&tmptup, + attrno, + tupDesc, + isNull); + + ReleaseTupleDesc(tupDesc); + + return result; +} + +/* + * Number of items in a tlist (including any resjunk items!) + */ +int +ExecTargetListLength(List *targetlist) +{ + /* This used to be more complex, but fjoins are dead */ + return list_length(targetlist); +} + +/* + * Number of items in a tlist, not including any resjunk items + */ +int +ExecCleanTargetListLength(List *targetlist) +{ + int len = 0; + ListCell *tl; + + foreach(tl, targetlist) + { + TargetEntry *curTle = lfirst_node(TargetEntry, tl); + + if (!curTle->resjunk) + len++; + } + return len; +} diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c index 06b4a57656..3f40fa65ef 100644 --- a/src/backend/executor/functions.c +++ b/src/backend/executor/functions.c @@ -3,7 +3,7 @@ * functions.c * Execution of SQL-language functions * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -70,7 +70,7 @@ typedef struct execution_state ExecStatus status; bool setsResult; /* true if this query produces func's result */ bool lazyEval; /* true if should fetch one row at a time */ - Node *stmt; /* PlannedStmt or utility statement */ + PlannedStmt *stmt; /* plan for this query */ QueryDesc *qd; /* null unless status == RUN */ char *src; /* source query resulting in this state */ } execution_state; @@ -486,7 +486,7 @@ init_execution_state(List *queryTree_list, forboth(lc1, queryTree_list, lc3, querySource_list) { - List *qtlist = (List *) lfirst(lc1); + List *qtlist = lfirst_node(List, lc1); char *querysource = (char *) lfirst(lc3); execution_state *firstes = NULL; execution_state *preves = NULL; @@ -494,34 +494,52 @@ init_execution_state(List *queryTree_list, foreach(lc2, qtlist) { - Query *queryTree = (Query *) lfirst(lc2); - Node *stmt; + Query *queryTree = lfirst_node(Query, lc2); + PlannedStmt *stmt; execution_state *newes; - Assert(IsA(queryTree, Query)); - /* Plan the query if needed */ if (queryTree->commandType == CMD_UTILITY) - stmt = queryTree->utilityStmt; + { + /* Utility commands require no planning. */ + stmt = makeNode(PlannedStmt); + stmt->commandType = CMD_UTILITY; + stmt->canSetTag = queryTree->canSetTag; + stmt->utilityStmt = queryTree->utilityStmt; + stmt->stmt_location = queryTree->stmt_location; + stmt->stmt_len = queryTree->stmt_len; + } else - stmt = (Node *) pg_plan_query(queryTree, - fcache->readonly_func ? CURSOR_OPT_PARALLEL_OK : 0, - NULL); + stmt = pg_plan_query(queryTree, + CURSOR_OPT_PARALLEL_OK, + NULL); - /* Precheck all commands for validity in a function */ - if (IsA(stmt, TransactionStmt)) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - /* translator: %s is a SQL statement name */ - errmsg("%s is not allowed in a SQL function", - CreateCommandTag(stmt)))); + /* + * Precheck all commands for validity in a function. This should + * generally match the restrictions spi.c applies. + */ + if (stmt->commandType == CMD_UTILITY) + { + if (IsA(stmt->utilityStmt, CopyStmt) && + ((CopyStmt *) stmt->utilityStmt)->filename == NULL) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot COPY to/from client in a SQL function"))); + + if (IsA(stmt->utilityStmt, TransactionStmt)) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + /* translator: %s is a SQL statement name */ + errmsg("%s is not allowed in a SQL function", + CreateCommandTag(stmt->utilityStmt)))); + } if (fcache->readonly_func && !CommandIsReadOnly(stmt)) ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), /* translator: %s is a SQL statement name */ errmsg("%s is not allowed in a non-volatile function", - CreateCommandTag(stmt)))); + CreateCommandTag((Node *) stmt)))); #ifdef PGXC if (IS_PGXC_LOCAL_COORDINATOR) @@ -542,7 +560,7 @@ init_execution_state(List *queryTree_list, #endif /* PGXC */ if (IsInParallelMode() && !CommandIsReadOnly(stmt)) - PreventCommandIfParallelMode(CreateCommandTag(stmt)); + PreventCommandIfParallelMode(CreateCommandTag((Node *) stmt)); /* OK, build the execution_state for this query */ newes = (execution_state *) palloc(sizeof(execution_state)); @@ -587,15 +605,9 @@ init_execution_state(List *queryTree_list, { lasttages->setsResult = true; if (lazyEvalOK && - IsA(lasttages->stmt, PlannedStmt)) - { - PlannedStmt *ps = (PlannedStmt *) lasttages->stmt; - - if (ps->commandType == CMD_SELECT && - ps->utilityStmt == NULL && - !ps->hasModifyingCTE) - fcache->lazyEval = lasttages->lazyEval = true; - } + lasttages->stmt->commandType == CMD_SELECT && + !lasttages->stmt->hasModifyingCTE) + fcache->lazyEval = lasttages->lazyEval = true; } return eslist; @@ -628,9 +640,7 @@ init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK) */ fcontext = AllocSetContextCreate(finfo->fn_mcxt, "SQL function data", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); oldcontext = MemoryContextSwitchTo(fcontext); @@ -725,14 +735,15 @@ init_sql_fcache(FmgrInfo *finfo, Oid collation, bool lazyEvalOK) flat_query_list = NIL; forboth(lc, raw_parsetree_list, lc2, querysource_list) { - Node *parsetree = (Node *) lfirst(lc); + RawStmt *parsetree = lfirst_node(RawStmt, lc); char *querysource = (char *) lfirst(lc2); List *queryTree_sublist; queryTree_sublist = pg_analyze_and_rewrite_params(parsetree, querysource, (ParserSetupHook) sql_fn_parser_setup, - fcache->pinfo); + fcache->pinfo, + NULL); queryTree_list = lappend(queryTree_list, queryTree_sublist); flat_query_list = list_concat(flat_query_list, list_copy(queryTree_sublist)); @@ -824,22 +835,17 @@ postquel_start(execution_state *es, SQLFunctionCachePtr fcache) else dest = None_Receiver; - if (IsA(es->stmt, PlannedStmt)) - es->qd = CreateQueryDesc((PlannedStmt *) es->stmt, - es->src, - GetActiveSnapshot(), - InvalidSnapshot, - dest, - fcache->paramLI, 0); - else - es->qd = CreateUtilityQueryDesc(es->stmt, - es->src, - GetActiveSnapshot(), - dest, - fcache->paramLI); + es->qd = CreateQueryDesc(es->stmt, + es->src, + GetActiveSnapshot(), + InvalidSnapshot, + dest, + fcache->paramLI, + es->qd ? es->qd->queryEnv : NULL, + 0); /* Utility commands don't need Executor. */ - if (es->qd->utilitystmt == NULL) + if (es->qd->operation != CMD_UTILITY) { /* * In lazyEval mode, do not let the executor set up an AfterTrigger @@ -867,15 +873,13 @@ postquel_getnext(execution_state *es, SQLFunctionCachePtr fcache) { bool result; - if (es->qd->utilitystmt) + if (es->qd->operation == CMD_UTILITY) { - /* ProcessUtility needs the PlannedStmt for DECLARE CURSOR */ - ProcessUtility((es->qd->plannedstmt ? - (Node *) es->qd->plannedstmt : - es->qd->utilitystmt), + ProcessUtility(es->qd->plannedstmt, es->src, PROCESS_UTILITY_QUERY, es->qd->params, + es->qd->queryEnv, es->qd->dest, #ifdef PGXC false, @@ -888,7 +892,7 @@ postquel_getnext(execution_state *es, SQLFunctionCachePtr fcache) /* Run regular commands to completion unless lazyEval */ uint64 count = (es->lazyEval) ? 1 : 0; - ExecutorRun(es->qd, ForwardScanDirection, count); + ExecutorRun(es->qd, ForwardScanDirection, count, !fcache->returnsSet || !es->lazyEval); /* * If we requested run to completion OR there was no tuple returned, @@ -908,7 +912,7 @@ postquel_end(execution_state *es) es->status = F_EXEC_DONE; /* Utility commands don't need Executor. */ - if (es->qd->utilitystmt == NULL) + if (es->qd->operation != CMD_UTILITY) { ExecutorFinish(es->qd); ExecutorEnd(es->qd); @@ -1312,7 +1316,7 @@ fmgr_sql(PG_FUNCTION_ARGS) rsi->returnMode = SFRM_Materialize; rsi->setResult = fcache->tstore; fcache->tstore = NULL; - /* must copy desc because execQual will free it */ + /* must copy desc because execSRF.c will free it */ if (fcache->junkFilter) rsi->setDesc = CreateTupleDescCopy(fcache->junkFilter->jf_cleanTupType); @@ -1584,7 +1588,7 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList, parse = NULL; foreach(lc, queryTreeList) { - Query *q = (Query *) lfirst(lc); + Query *q = lfirst_node(Query, lc); if (q->canSetTag) parse = q; @@ -1602,8 +1606,7 @@ check_sql_fn_retval(Oid func_id, Oid rettype, List *queryTreeList, * entities. */ if (parse && - parse->commandType == CMD_SELECT && - parse->utilityStmt == NULL) + parse->commandType == CMD_SELECT) { tlist_ptr = &parse->targetList; tlist = parse->targetList; diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c index 2614bf451f..6ec96ec371 100644 --- a/src/backend/executor/instrument.c +++ b/src/backend/executor/instrument.c @@ -4,7 +4,7 @@ * functions for instrumentation of plan execution * * - * Copyright (c) 2001-2016, PostgreSQL Global Development Group + * Copyright (c) 2001-2017, PostgreSQL Global Development Group * * IDENTIFICATION * src/backend/executor/instrument.c diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c index 8154522de4..1b94a66484 100644 --- a/src/backend/executor/nodeAgg.c +++ b/src/backend/executor/nodeAgg.c @@ -91,10 +91,13 @@ * transition value or a previous function result, and in either case its * value need not be preserved. See int8inc() for an example. Notice that * advance_transition_function() is coded to avoid a data copy step when - * the previous transition value pointer is returned. Also, some - * transition functions want to store working state in addition to the - * nominal transition value; they can use the memory context returned by - * AggCheckCallContext() to do that. + * the previous transition value pointer is returned. It is also possible + * to avoid repeated data copying when the transition value is an expanded + * object: to do that, the transition function must take care to return + * an expanded object that is in a child context of the memory context + * returned by AggCheckCallContext(). Also, some transition functions want + * to store working state in addition to the nominal transition value; they + * can use the memory context returned by AggCheckCallContext() to do that. * * Note: AggCheckCallContext() is available as of PostgreSQL 9.0. The * AggState is available as context in earlier releases (back to 8.1), @@ -119,12 +122,19 @@ * specific). * * Where more complex grouping sets are used, we break them down into - * "phases", where each phase has a different sort order. During each - * phase but the last, the input tuples are additionally stored in a - * tuplesort which is keyed to the next phase's sort order; during each - * phase but the first, the input tuples are drawn from the previously - * sorted data. (The sorting of the data for the first phase is handled by - * the planner, as it might be satisfied by underlying nodes.) + * "phases", where each phase has a different sort order (except phase 0 + * which is reserved for hashing). During each phase but the last, the + * input tuples are additionally stored in a tuplesort which is keyed to the + * next phase's sort order; during each phase but the first, the input + * tuples are drawn from the previously sorted data. (The sorting of the + * data for the first phase is handled by the planner, as it might be + * satisfied by underlying nodes.) + * + * Hashing can be mixed with sorted grouping. To do this, we have an + * AGG_MIXED strategy that populates the hashtables during the first sorted + * phase, and switches to reading them out after completing all sort phases. + * We can also support AGG_HASHED with multiple hash tables and no sorting + * at all. * * From the perspective of aggregate transition and final functions, the * only issue regarding grouping sets is this: a single call site (flinfo) @@ -136,10 +146,57 @@ * sensitive to the grouping set for which the aggregate function is * currently being called. * - * TODO: AGG_HASHED doesn't support multiple grouping sets yet. + * Plan structure: + * + * What we get from the planner is actually one "real" Agg node which is + * part of the plan tree proper, but which optionally has an additional list + * of Agg nodes hung off the side via the "chain" field. This is because an + * Agg node happens to be a convenient representation of all the data we + * need for grouping sets. + * + * For many purposes, we treat the "real" node as if it were just the first + * node in the chain. The chain must be ordered such that hashed entries + * come before sorted/plain entries; the real node is marked AGG_MIXED if + * there are both types present (in which case the real node describes one + * of the hashed groupings, other AGG_HASHED nodes may optionally follow in + * the chain, followed in turn by AGG_SORTED or (one) AGG_PLAIN node). If + * the real node is marked AGG_HASHED or AGG_SORTED, then all the chained + * nodes must be of the same type; if it is AGG_PLAIN, there can be no + * chained nodes. + * + * We collect all hashed nodes into a single "phase", numbered 0, and create + * a sorted phase (numbered 1..n) for each AGG_SORTED or AGG_PLAIN node. + * Phase 0 is allocated even if there are no hashes, but remains unused in + * that case. + * + * AGG_HASHED nodes actually refer to only a single grouping set each, + * because for each hashed grouping we need a separate grpColIdx and + * numGroups estimate. AGG_SORTED nodes represent a "rollup", a list of + * grouping sets that share a sort order. Each AGG_SORTED node other than + * the first one has an associated Sort node which describes the sort order + * to be used; the first sorted node takes its input from the outer subtree, + * which the planner has already arranged to provide ordered data. + * + * Memory and ExprContext usage: + * + * Because we're accumulating aggregate values across input rows, we need to + * use more memory contexts than just simple input/output tuple contexts. + * In fact, for a rollup, we need a separate context for each grouping set + * so that we can reset the inner (finer-grained) aggregates on their group + * boundaries while continuing to accumulate values for outer + * (coarser-grained) groupings. On top of this, we might be simultaneously + * populating hashtables; however, we only need one context for all the + * hashtables. + * + * So we create an array, aggcontexts, with an ExprContext for each grouping + * set in the largest rollup that we're going to process, and use the + * per-tuple memory context of those ExprContexts to store the aggregate + * transition values. hashcontext is the single context created to support + * all hash tables. + * * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * IDENTIFICATION @@ -158,6 +215,7 @@ #include "executor/executor.h" #include "executor/nodeAgg.h" #include "miscadmin.h" +#include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" #include "optimizer/clauses.h" #include "optimizer/tlist.h" @@ -211,6 +269,9 @@ typedef struct AggStatePerTransData */ int numInputs; + /* offset of input columns in AggState->evalslot */ + int inputoff; + /* * Number of aggregated input columns to pass to the transfn. This * includes the ORDER BY columns for ordered-set aggs, but not for plain @@ -232,7 +293,6 @@ typedef struct AggStatePerTransData /* ExprStates of the FILTER and argument expressions. */ ExprState *aggfilter; /* state of FILTER expression, if any */ - List *args; /* states of aggregated-argument expressions */ List *aggdirectargs; /* states of direct-argument expressions */ /* @@ -289,19 +349,19 @@ typedef struct AggStatePerTransData transtypeByVal; /* - * Stuff for evaluation of inputs. We used to just use ExecEvalExpr, but - * with the addition of ORDER BY we now need at least a slot for passing - * data to the sort object, which requires a tupledesc, so we might as - * well go whole hog and use ExecProject too. + * Stuff for evaluation of aggregate inputs in cases where the aggregate + * requires sorted input. The arguments themselves will be evaluated via + * AggState->evalslot/evalproj for all aggregates at once, but we only + * want to sort the relevant columns for individual aggregates. */ - TupleDesc evaldesc; /* descriptor of input tuples */ - ProjectionInfo *evalproj; /* projection machinery */ + TupleDesc sortdesc; /* descriptor of input tuples */ /* * Slots for holding the evaluated input arguments. These are set up - * during ExecInitAgg() and then used for each input row. + * during ExecInitAgg() and then used for each input row requiring + * processing besides what's done in AggState->evalproj. */ - TupleTableSlot *evalslot; /* current input tuple */ + TupleTableSlot *sortslot; /* current input tuple */ TupleTableSlot *uniqslot; /* used for multi-column DISTINCT */ /* @@ -412,7 +472,7 @@ typedef struct AggStatePerGroupData * NULL and not auto-replace it with a later input value. Only the first * non-NULL input will be auto-substituted. */ -} AggStatePerGroupData; +} AggStatePerGroupData; /* * AggStatePerPhaseData - per-grouping-set-phase state @@ -427,6 +487,7 @@ typedef struct AggStatePerGroupData */ typedef struct AggStatePerPhaseData { + AggStrategy aggstrategy; /* strategy for this phase */ int numsets; /* number of grouping sets (or 0) */ int *gset_lengths; /* lengths of grouping sets */ Bitmapset **grouped_cols; /* column groupings for rollup */ @@ -436,20 +497,29 @@ typedef struct AggStatePerPhaseData } AggStatePerPhaseData; /* - * To implement hashed aggregation, we need a hashtable that stores a - * representative tuple and an array of AggStatePerGroup structs for each - * distinct set of GROUP BY column values. We compute the hash key from - * the GROUP BY columns. + * AggStatePerHashData - per-hashtable state + * + * When doing grouping sets with hashing, we have one of these for each + * grouping set. (When doing hashing without grouping sets, we have just one of + * them.) */ -typedef struct AggHashEntryData *AggHashEntry; - -typedef struct AggHashEntryData +typedef struct AggStatePerHashData { - TupleHashEntryData shared; /* common header for hash table entries */ - /* per-aggregate transition status array */ - AggStatePerGroupData pergroup[FLEXIBLE_ARRAY_MEMBER]; -} AggHashEntryData; + TupleHashTable hashtable; /* hash table with one entry per group */ + TupleHashIterator hashiter; /* for iterating through hash table */ + TupleTableSlot *hashslot; /* slot for loading hash table */ + FmgrInfo *hashfunctions; /* per-grouping-field hash fns */ + FmgrInfo *eqfunctions; /* per-grouping-field equality fns */ + int numCols; /* number of hash key columns */ + int numhashGrpCols; /* number of columns in hash table */ + int largestGrpColIdx; /* largest col required for hashing */ + AttrNumber *hashGrpColIdxInput; /* hash col indices in input slot */ + AttrNumber *hashGrpColIdxHash; /* indices in hashtbl tuples */ + Agg *aggnode; /* original Agg node, for numGroups etc. */ +} AggStatePerHashData; + +static void select_current_set(AggState *aggstate, int setno, bool is_hash); static void initialize_phase(AggState *aggstate, int newphase); static TupleTableSlot *fetch_input_tuple(AggState *aggstate); static void initialize_aggregates(AggState *aggstate, @@ -458,7 +528,8 @@ static void initialize_aggregates(AggState *aggstate, static void advance_transition_function(AggState *aggstate, AggStatePerTrans pertrans, AggStatePerGroup pergroupstate); -static void advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup); +static void advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup, + AggStatePerGroup *pergroups); static void advance_combine_function(AggState *aggstate, AggStatePerTrans pertrans, AggStatePerGroup pergroupstate); @@ -482,20 +553,19 @@ static void prepare_projection_slot(AggState *aggstate, int currentSet); static void finalize_aggregates(AggState *aggstate, AggStatePerAgg peragg, - AggStatePerGroup pergroup, - int currentSet); + AggStatePerGroup pergroup); static TupleTableSlot *project_aggregates(AggState *aggstate); static Bitmapset *find_unaggregated_cols(AggState *aggstate); static bool find_unaggregated_cols_walker(Node *node, Bitmapset **colnos); static void build_hash_table(AggState *aggstate); -static AggHashEntry lookup_hash_entry(AggState *aggstate, - TupleTableSlot *inputslot); +static TupleHashEntryData *lookup_hash_entry(AggState *aggstate); +static AggStatePerGroup *lookup_hash_entries(AggState *aggstate); static TupleTableSlot *agg_retrieve_direct(AggState *aggstate); static void agg_fill_hash_table(AggState *aggstate); static TupleTableSlot *agg_retrieve_hash_table(AggState *aggstate); static Datum GetAggInitVal(Datum textInitVal, Oid transtype); static void build_pertrans_for_aggref(AggStatePerTrans pertrans, - AggState *aggsate, EState *estate, + AggState *aggstate, EState *estate, Aggref *aggref, Oid aggtransfn, Oid aggtranstype, Oid aggserialfn, Oid aggdeserialfn, Datum initValue, bool initValueIsNull, @@ -510,13 +580,31 @@ static int find_compatible_pertrans(AggState *aggstate, Aggref *newagg, /* - * Switch to phase "newphase", which must either be 0 (to reset) or + * Select the current grouping set; affects current_set and + * curaggcontext. + */ +static void +select_current_set(AggState *aggstate, int setno, bool is_hash) +{ + if (is_hash) + aggstate->curaggcontext = aggstate->hashcontext; + else + aggstate->curaggcontext = aggstate->aggcontexts[setno]; + + aggstate->current_set = setno; +} + +/* + * Switch to phase "newphase", which must either be 0 or 1 (to reset) or * current_phase + 1. Juggle the tuplesorts accordingly. + * + * Phase 0 is for hashing, which we currently handle last in the AGG_MIXED + * case, so when entering phase 0, all we need to do is drop open sorts. */ static void initialize_phase(AggState *aggstate, int newphase) { - Assert(newphase == 0 || newphase == aggstate->current_phase + 1); + Assert(newphase <= 1 || newphase == aggstate->current_phase + 1); /* * Whatever the previous state, we're now done with whatever input @@ -528,7 +616,7 @@ initialize_phase(AggState *aggstate, int newphase) aggstate->sort_in = NULL; } - if (newphase == 0) + if (newphase <= 1) { /* * Discard any existing output tuplesort. @@ -555,7 +643,7 @@ initialize_phase(AggState *aggstate, int newphase) * If this isn't the last phase, we need to sort appropriately for the * next phase in sequence. */ - if (newphase < aggstate->numphases - 1) + if (newphase > 0 && newphase < aggstate->numphases - 1) { Sort *sortnode = aggstate->phases[newphase + 1].sortnode; PlanState *outerNode = outerPlanState(aggstate); @@ -576,9 +664,12 @@ initialize_phase(AggState *aggstate, int newphase) } /* - * Fetch a tuple from either the outer plan (for phase 0) or from the sorter + * Fetch a tuple from either the outer plan (for phase 1) or from the sorter * populated by the previous phase. Copy it to the sorter for the next phase * if any. + * + * Callers cannot rely on memory for tuple in returned slot remaining valid + * past any subsequently fetched tuple. */ static TupleTableSlot * fetch_input_tuple(AggState *aggstate) @@ -587,8 +678,8 @@ fetch_input_tuple(AggState *aggstate) if (aggstate->sort_in) { - if (!tuplesort_gettupleslot(aggstate->sort_in, true, aggstate->sort_slot, - NULL)) + if (!tuplesort_gettupleslot(aggstate->sort_in, true, false, + aggstate->sort_slot, NULL)) return NULL; slot = aggstate->sort_slot; } @@ -604,8 +695,8 @@ fetch_input_tuple(AggState *aggstate) /* * (Re)Initialize an individual aggregate. * - * This function handles only one grouping set (already set in - * aggstate->current_set). + * This function handles only one grouping set, already set in + * aggstate->current_set. * * When called, CurrentMemoryContext should be the per-query context. */ @@ -633,14 +724,14 @@ initialize_aggregate(AggState *aggstate, AggStatePerTrans pertrans, */ if (pertrans->numInputs == 1) pertrans->sortstates[aggstate->current_set] = - tuplesort_begin_datum(pertrans->evaldesc->attrs[0]->atttypid, + tuplesort_begin_datum(pertrans->sortdesc->attrs[0]->atttypid, pertrans->sortOperators[0], pertrans->sortCollations[0], pertrans->sortNullsFirst[0], work_mem, false); else pertrans->sortstates[aggstate->current_set] = - tuplesort_begin_heap(pertrans->evaldesc, + tuplesort_begin_heap(pertrans->sortdesc, pertrans->numSortCols, pertrans->sortColIdx, pertrans->sortOperators, @@ -662,7 +753,7 @@ initialize_aggregate(AggState *aggstate, AggStatePerTrans pertrans, MemoryContext oldContext; oldContext = MemoryContextSwitchTo( - aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory); + aggstate->curaggcontext->ecxt_per_tuple_memory); pergroupstate->transValue = datumCopy(pertrans->initValue, pertrans->transtypeByVal, pertrans->transtypeLen); @@ -685,8 +776,9 @@ initialize_aggregate(AggState *aggstate, AggStatePerTrans pertrans, * * If there are multiple grouping sets, we initialize only the first numReset * of them (the grouping sets are ordered so that the most specific one, which - * is reset most often, is first). As a convenience, if numReset is < 1, we - * reinitialize all sets. + * is reset most often, is first). As a convenience, if numReset is 0, we + * reinitialize all sets. numReset is -1 to initialize a hashtable entry, in + * which case the caller must have used select_current_set appropriately. * * When called, CurrentMemoryContext should be the per-query context. */ @@ -698,25 +790,37 @@ initialize_aggregates(AggState *aggstate, int transno; int numGroupingSets = Max(aggstate->phase->numsets, 1); int setno = 0; + int numTrans = aggstate->numtrans; AggStatePerTrans transstates = aggstate->pertrans; - if (numReset < 1) + if (numReset == 0) numReset = numGroupingSets; - for (transno = 0; transno < aggstate->numtrans; transno++) + for (transno = 0; transno < numTrans; transno++) { AggStatePerTrans pertrans = &transstates[transno]; - for (setno = 0; setno < numReset; setno++) + if (numReset < 0) { AggStatePerGroup pergroupstate; - pergroupstate = &pergroup[transno + (setno * (aggstate->numtrans))]; - - aggstate->current_set = setno; + pergroupstate = &pergroup[transno]; initialize_aggregate(aggstate, pertrans, pergroupstate); } + else + { + for (setno = 0; setno < numReset; setno++) + { + AggStatePerGroup pergroupstate; + + pergroupstate = &pergroup[transno + (setno * numTrans)]; + + select_current_set(aggstate, setno, false); + + initialize_aggregate(aggstate, pertrans, pergroupstate); + } + } } } @@ -766,7 +870,7 @@ advance_transition_function(AggState *aggstate, * do not need to pfree the old transValue, since it's NULL. */ oldContext = MemoryContextSwitchTo( - aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory); + aggstate->curaggcontext->ecxt_per_tuple_memory); pergroupstate->transValue = datumCopy(fcinfo->arg[1], pertrans->transtypeByVal, pertrans->transtypeLen); @@ -806,21 +910,36 @@ advance_transition_function(AggState *aggstate, /* * If pass-by-ref datatype, must copy the new value into aggcontext and - * pfree the prior transValue. But if transfn returned a pointer to its - * first input, we don't need to do anything. + * free the prior transValue. But if transfn returned a pointer to its + * first input, we don't need to do anything. Also, if transfn returned a + * pointer to a R/W expanded object that is already a child of the + * aggcontext, assume we can adopt that value without copying it. */ if (!pertrans->transtypeByVal && DatumGetPointer(newVal) != DatumGetPointer(pergroupstate->transValue)) { if (!fcinfo->isnull) { - MemoryContextSwitchTo(aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory); - newVal = datumCopy(newVal, - pertrans->transtypeByVal, - pertrans->transtypeLen); + MemoryContextSwitchTo(aggstate->curaggcontext->ecxt_per_tuple_memory); + if (DatumIsReadWriteExpandedObject(newVal, + false, + pertrans->transtypeLen) && + MemoryContextGetParent(DatumGetEOHP(newVal)->eoh_context) == CurrentMemoryContext) + /* do nothing */ ; + else + newVal = datumCopy(newVal, + pertrans->transtypeByVal, + pertrans->transtypeLen); } if (!pergroupstate->transValueIsNull) - pfree(DatumGetPointer(pergroupstate->transValue)); + { + if (DatumIsReadWriteExpandedObject(pergroupstate->transValue, + false, + pertrans->transtypeLen)) + DeleteExpandedObject(pergroupstate->transValue); + else + pfree(DatumGetPointer(pergroupstate->transValue)); + } } pergroupstate->transValue = newVal; @@ -832,18 +951,27 @@ advance_transition_function(AggState *aggstate, /* * Advance each aggregate transition state for one input tuple. The input * tuple has been stored in tmpcontext->ecxt_outertuple, so that it is - * accessible to ExecEvalExpr. pergroup is the array of per-group structs to - * use (this might be in a hashtable entry). + * accessible to ExecEvalExpr. + * + * We have two sets of transition states to handle: one for sorted aggregation + * and one for hashed; we do them both here, to avoid multiple evaluation of + * the inputs. * * When called, CurrentMemoryContext should be the per-query context. */ static void -advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup) +advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup, AggStatePerGroup *pergroups) { int transno; int setno = 0; int numGroupingSets = Max(aggstate->phase->numsets, 1); + int numHashes = aggstate->num_hashes; int numTrans = aggstate->numtrans; + TupleTableSlot *slot = aggstate->evalslot; + + /* compute input for all aggregates */ + if (aggstate->evalproj) + aggstate->evalslot = ExecProject(aggstate->evalproj); for (transno = 0; transno < numTrans; transno++) { @@ -851,7 +979,7 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup) ExprState *filter = pertrans->aggfilter; int numTransInputs = pertrans->numTransInputs; int i; - TupleTableSlot *slot; + int inputoff = pertrans->inputoff; /* Skip anything FILTERed out */ if (filter) @@ -860,18 +988,16 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup) bool isnull; res = ExecEvalExprSwitchContext(filter, aggstate->tmpcontext, - &isnull, NULL); + &isnull); if (isnull || !DatumGetBool(res)) continue; } - /* Evaluate the current input expressions for this aggregate */ - slot = ExecProject(pertrans->evalproj, NULL); - if (pertrans->numSortCols > 0) { /* DISTINCT and/or ORDER BY case */ - Assert(slot->tts_nvalid == pertrans->numInputs); + Assert(slot->tts_nvalid >= (pertrans->numInputs + inputoff)); + Assert(!pergroups); /* * If the transfn is strict, we want to check for nullity before @@ -884,7 +1010,7 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup) { for (i = 0; i < numTransInputs; i++) { - if (slot->tts_isnull[i]) + if (slot->tts_isnull[i + inputoff]) break; } if (i < numTransInputs) @@ -896,10 +1022,25 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup) /* OK, put the tuple into the tuplesort object */ if (pertrans->numInputs == 1) tuplesort_putdatum(pertrans->sortstates[setno], - slot->tts_values[0], - slot->tts_isnull[0]); + slot->tts_values[inputoff], + slot->tts_isnull[inputoff]); else - tuplesort_puttupleslot(pertrans->sortstates[setno], slot); + { + /* + * Copy slot contents, starting from inputoff, into sort + * slot. + */ + ExecClearTuple(pertrans->sortslot); + memcpy(pertrans->sortslot->tts_values, + &slot->tts_values[inputoff], + pertrans->numInputs * sizeof(Datum)); + memcpy(pertrans->sortslot->tts_isnull, + &slot->tts_isnull[inputoff], + pertrans->numInputs * sizeof(bool)); + pertrans->sortslot->tts_nvalid = pertrans->numInputs; + ExecStoreVirtualTuple(pertrans->sortslot); + tuplesort_puttupleslot(pertrans->sortstates[setno], pertrans->sortslot); + } } } else @@ -909,20 +1050,44 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup) /* Load values into fcinfo */ /* Start from 1, since the 0th arg will be the transition value */ - Assert(slot->tts_nvalid >= numTransInputs); + Assert(slot->tts_nvalid >= (numTransInputs + inputoff)); + for (i = 0; i < numTransInputs; i++) { - fcinfo->arg[i + 1] = slot->tts_values[i]; - fcinfo->argnull[i + 1] = slot->tts_isnull[i]; + fcinfo->arg[i + 1] = slot->tts_values[i + inputoff]; + fcinfo->argnull[i + 1] = slot->tts_isnull[i + inputoff]; } - for (setno = 0; setno < numGroupingSets; setno++) + if (pergroup) { - AggStatePerGroup pergroupstate = &pergroup[transno + (setno * numTrans)]; + /* advance transition states for ordered grouping */ - aggstate->current_set = setno; + for (setno = 0; setno < numGroupingSets; setno++) + { + AggStatePerGroup pergroupstate; + + select_current_set(aggstate, setno, false); + + pergroupstate = &pergroup[transno + (setno * numTrans)]; - advance_transition_function(aggstate, pertrans, pergroupstate); + advance_transition_function(aggstate, pertrans, pergroupstate); + } + } + + if (pergroups) + { + /* advance transition states for hashed grouping */ + + for (setno = 0; setno < numHashes; setno++) + { + AggStatePerGroup pergroupstate; + + select_current_set(aggstate, setno, true); + + pergroupstate = &pergroups[setno][transno]; + + advance_transition_function(aggstate, pertrans, pergroupstate); + } } } } @@ -940,20 +1105,22 @@ combine_aggregates(AggState *aggstate, AggStatePerGroup pergroup) { int transno; int numTrans = aggstate->numtrans; + TupleTableSlot *slot; /* combine not supported with grouping sets */ - Assert(aggstate->phase->numsets == 0); + Assert(aggstate->phase->numsets <= 1); + + /* compute input for all aggregates */ + slot = ExecProject(aggstate->evalproj); for (transno = 0; transno < numTrans; transno++) { AggStatePerTrans pertrans = &aggstate->pertrans[transno]; AggStatePerGroup pergroupstate = &pergroup[transno]; - TupleTableSlot *slot; FunctionCallInfo fcinfo = &pertrans->transfn_fcinfo; + int inputoff = pertrans->inputoff; - /* Evaluate the current input expressions for this aggregate */ - slot = ExecProject(pertrans->evalproj, NULL); - Assert(slot->tts_nvalid >= 1); + Assert(slot->tts_nvalid > inputoff); /* * deserialfn_oid will be set if we must deserialize the input state @@ -962,18 +1129,18 @@ combine_aggregates(AggState *aggstate, AggStatePerGroup pergroup) if (OidIsValid(pertrans->deserialfn_oid)) { /* Don't call a strict deserialization function with NULL input */ - if (pertrans->deserialfn.fn_strict && slot->tts_isnull[0]) + if (pertrans->deserialfn.fn_strict && slot->tts_isnull[inputoff]) { - fcinfo->arg[1] = slot->tts_values[0]; - fcinfo->argnull[1] = slot->tts_isnull[0]; + fcinfo->arg[1] = slot->tts_values[inputoff]; + fcinfo->argnull[1] = slot->tts_isnull[inputoff]; } else { FunctionCallInfo dsinfo = &pertrans->deserialfn_fcinfo; MemoryContext oldContext; - dsinfo->arg[0] = slot->tts_values[0]; - dsinfo->argnull[0] = slot->tts_isnull[0]; + dsinfo->arg[0] = slot->tts_values[inputoff]; + dsinfo->argnull[0] = slot->tts_isnull[inputoff]; /* Dummy second argument for type-safety reasons */ dsinfo->arg[1] = PointerGetDatum(NULL); dsinfo->argnull[1] = false; @@ -992,8 +1159,8 @@ combine_aggregates(AggState *aggstate, AggStatePerGroup pergroup) } else { - fcinfo->arg[1] = slot->tts_values[0]; - fcinfo->argnull[1] = slot->tts_isnull[0]; + fcinfo->arg[1] = slot->tts_values[inputoff]; + fcinfo->argnull[1] = slot->tts_isnull[inputoff]; } advance_combine_function(aggstate, pertrans, pergroupstate); @@ -1034,7 +1201,7 @@ advance_combine_function(AggState *aggstate, if (!pertrans->transtypeByVal) { oldContext = MemoryContextSwitchTo( - aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory); + aggstate->curaggcontext->ecxt_per_tuple_memory); pergroupstate->transValue = datumCopy(fcinfo->arg[1], pertrans->transtypeByVal, pertrans->transtypeLen); @@ -1068,21 +1235,37 @@ advance_combine_function(AggState *aggstate, /* * If pass-by-ref datatype, must copy the new value into aggcontext and - * pfree the prior transValue. But if the combine function returned a - * pointer to its first input, we don't need to do anything. + * free the prior transValue. But if the combine function returned a + * pointer to its first input, we don't need to do anything. Also, if the + * combine function returned a pointer to a R/W expanded object that is + * already a child of the aggcontext, assume we can adopt that value + * without copying it. */ if (!pertrans->transtypeByVal && DatumGetPointer(newVal) != DatumGetPointer(pergroupstate->transValue)) { if (!fcinfo->isnull) { - MemoryContextSwitchTo(aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory); - newVal = datumCopy(newVal, - pertrans->transtypeByVal, - pertrans->transtypeLen); + MemoryContextSwitchTo(aggstate->curaggcontext->ecxt_per_tuple_memory); + if (DatumIsReadWriteExpandedObject(newVal, + false, + pertrans->transtypeLen) && + MemoryContextGetParent(DatumGetEOHP(newVal)->eoh_context) == CurrentMemoryContext) + /* do nothing */ ; + else + newVal = datumCopy(newVal, + pertrans->transtypeByVal, + pertrans->transtypeLen); } if (!pergroupstate->transValueIsNull) - pfree(DatumGetPointer(pergroupstate->transValue)); + { + if (DatumIsReadWriteExpandedObject(pergroupstate->transValue, + false, + pertrans->transtypeLen)) + DeleteExpandedObject(pergroupstate->transValue); + else + pfree(DatumGetPointer(pergroupstate->transValue)); + } } pergroupstate->transValue = newVal; @@ -1214,7 +1397,7 @@ process_ordered_aggregate_multi(AggState *aggstate, { MemoryContext workcontext = aggstate->tmpcontext->ecxt_per_tuple_memory; FunctionCallInfo fcinfo = &pertrans->transfn_fcinfo; - TupleTableSlot *slot1 = pertrans->evalslot; + TupleTableSlot *slot1 = pertrans->sortslot; TupleTableSlot *slot2 = pertrans->uniqslot; int numTransInputs = pertrans->numTransInputs; int numDistinctCols = pertrans->numDistinctCols; @@ -1230,7 +1413,7 @@ process_ordered_aggregate_multi(AggState *aggstate, ExecClearTuple(slot2); while (tuplesort_gettupleslot(pertrans->sortstates[aggstate->current_set], - true, slot1, &newAbbrevVal)) + true, true, slot1, &newAbbrevVal)) { /* * Extract the first numTransInputs columns as datums to pass to the @@ -1326,8 +1509,7 @@ finalize_aggregate(AggState *aggstate, fcinfo.arg[i] = ExecEvalExpr(expr, aggstate->ss.ps.ps_ExprContext, - &fcinfo.argnull[i], - NULL); + &fcinfo.argnull[i]); anynull |= fcinfo.argnull[i]; i++; } @@ -1348,7 +1530,9 @@ finalize_aggregate(AggState *aggstate, (void *) aggstate, NULL); /* Fill in the transition state value */ - fcinfo.arg[0] = pergroupstate->transValue; + fcinfo.arg[0] = MakeExpandedObjectReadOnly(pergroupstate->transValue, + pergroupstate->transValueIsNull, + pertrans->transtypeLen); fcinfo.argnull[0] = pergroupstate->transValueIsNull; anynull |= pergroupstate->transValueIsNull; @@ -1375,6 +1559,7 @@ finalize_aggregate(AggState *aggstate, } else { + /* Don't need MakeExpandedObjectReadOnly; datumCopy will copy it */ *resultVal = pergroupstate->transValue; *resultIsNull = pergroupstate->transValueIsNull; } @@ -1425,7 +1610,9 @@ finalize_partialaggregate(AggState *aggstate, { FunctionCallInfo fcinfo = &pertrans->serialfn_fcinfo; - fcinfo->arg[0] = pergroupstate->transValue; + fcinfo->arg[0] = MakeExpandedObjectReadOnly(pergroupstate->transValue, + pergroupstate->transValueIsNull, + pertrans->transtypeLen); fcinfo->argnull[0] = pergroupstate->transValueIsNull; *resultVal = FunctionCallInvoke(fcinfo); @@ -1434,6 +1621,7 @@ finalize_partialaggregate(AggState *aggstate, } else { + /* Don't need MakeExpandedObjectReadOnly; datumCopy will copy it */ *resultVal = pergroupstate->transValue; *resultIsNull = pergroupstate->transValueIsNull; } @@ -1512,38 +1700,38 @@ prepare_projection_slot(AggState *aggstate, TupleTableSlot *slot, int currentSet /* * Compute the final value of all aggregates for one group. * - * This function handles only one grouping set at a time. + * This function handles only one grouping set at a time, which the caller must + * have selected. It's also the caller's responsibility to adjust the supplied + * pergroup parameter to point to the current set's transvalues. * * Results are stored in the output econtext aggvalues/aggnulls. */ static void finalize_aggregates(AggState *aggstate, AggStatePerAgg peraggs, - AggStatePerGroup pergroup, - int currentSet) + AggStatePerGroup pergroup) { ExprContext *econtext = aggstate->ss.ps.ps_ExprContext; Datum *aggvalues = econtext->ecxt_aggvalues; bool *aggnulls = econtext->ecxt_aggnulls; int aggno; + int transno; - Assert(currentSet == 0 || - ((Agg *) aggstate->ss.ps.plan)->aggstrategy != AGG_HASHED); - - aggstate->current_set = currentSet; - - for (aggno = 0; aggno < aggstate->numaggs; aggno++) + /* + * If there were any DISTINCT and/or ORDER BY aggregates, sort their + * inputs and run the transition functions. + */ + for (transno = 0; transno < aggstate->numtrans; transno++) { - AggStatePerAgg peragg = &peraggs[aggno]; - int transno = peragg->transno; AggStatePerTrans pertrans = &aggstate->pertrans[transno]; AggStatePerGroup pergroupstate; - pergroupstate = &pergroup[transno + (currentSet * (aggstate->numtrans))]; + pergroupstate = &pergroup[transno]; if (pertrans->numSortCols > 0) { - Assert(((Agg *) aggstate->ss.ps.plan)->aggstrategy != AGG_HASHED); + Assert(aggstate->aggstrategy != AGG_HASHED && + aggstate->aggstrategy != AGG_MIXED); if (pertrans->numInputs == 1) process_ordered_aggregate_single(aggstate, @@ -1554,6 +1742,18 @@ finalize_aggregates(AggState *aggstate, pertrans, pergroupstate); } + } + + /* + * Run the final functions. + */ + for (aggno = 0; aggno < aggstate->numaggs; aggno++) + { + AggStatePerAgg peragg = &peraggs[aggno]; + int transno = peragg->transno; + AggStatePerGroup pergroupstate; + + pergroupstate = &pergroup[transno]; if (DO_AGGSPLIT_SKIPFINAL(aggstate->aggsplit)) finalize_partialaggregate(aggstate, peragg, pergroupstate, @@ -1567,7 +1767,7 @@ finalize_aggregates(AggState *aggstate, /* * Project the result of a group (whose aggs have already been calculated by * finalize_aggregates). Returns the result slot, or NULL if no row is - * projected (suppressed by qual or by an empty SRF). + * projected (suppressed by qual). */ static TupleTableSlot * project_aggregates(AggState *aggstate) @@ -1577,23 +1777,13 @@ project_aggregates(AggState *aggstate) /* * Check the qual (HAVING clause); if the group does not match, ignore it. */ - if (ExecQual(aggstate->ss.ps.qual, econtext, false)) + if (ExecQual(aggstate->ss.ps.qual, econtext)) { /* - * Form and return or store a projection tuple using the aggregate - * results and the representative input tuple. + * Form and return projection tuple using the aggregate results and + * the representative input tuple. */ - ExprDoneCond isDone; - TupleTableSlot *result; - - result = ExecProject(aggstate->ss.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - aggstate->ss.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } + return ExecProject(aggstate->ss.ps.ps_ProjInfo); } else InstrCountFiltered1(aggstate, 1); @@ -1645,78 +1835,162 @@ find_unaggregated_cols_walker(Node *node, Bitmapset **colnos) } /* - * Initialize the hash table to empty. + * Initialize the hash table(s) to empty. + * + * To implement hashed aggregation, we need a hashtable that stores a + * representative tuple and an array of AggStatePerGroup structs for each + * distinct set of GROUP BY column values. We compute the hash key from the + * GROUP BY columns. The per-group data is allocated in lookup_hash_entry(), + * for each entry. + * + * We have a separate hashtable and associated perhash data structure for each + * grouping set for which we're doing hashing. * - * The hash table always lives in the aggcontext memory context. + * The hash tables always live in the hashcontext's per-tuple memory context + * (there is only one of these for all tables together, since they are all + * reset at the same time). */ static void build_hash_table(AggState *aggstate) { - Agg *node = (Agg *) aggstate->ss.ps.plan; MemoryContext tmpmem = aggstate->tmpcontext->ecxt_per_tuple_memory; - Size entrysize; + Size additionalsize; + int i; - Assert(node->aggstrategy == AGG_HASHED); - Assert(node->numGroups > 0); + Assert(aggstate->aggstrategy == AGG_HASHED || aggstate->aggstrategy == AGG_MIXED); - entrysize = offsetof(AggHashEntryData, pergroup) + - aggstate->numaggs * sizeof(AggStatePerGroupData); + additionalsize = aggstate->numtrans * sizeof(AggStatePerGroupData); - aggstate->hashtable = BuildTupleHashTable(node->numCols, - node->grpColIdx, - aggstate->phase->eqfunctions, - aggstate->hashfunctions, - node->numGroups, - entrysize, - aggstate->aggcontexts[0]->ecxt_per_tuple_memory, - tmpmem); + for (i = 0; i < aggstate->num_hashes; ++i) + { + AggStatePerHash perhash = &aggstate->perhash[i]; + + Assert(perhash->aggnode->numGroups > 0); + + perhash->hashtable = BuildTupleHashTable(perhash->numCols, + perhash->hashGrpColIdxHash, + perhash->eqfunctions, + perhash->hashfunctions, + perhash->aggnode->numGroups, + additionalsize, + aggstate->hashcontext->ecxt_per_tuple_memory, + tmpmem, + DO_AGGSPLIT_SKIPFINAL(aggstate->aggsplit)); + } } /* - * Create a list of the tuple columns that actually need to be stored in - * hashtable entries. The incoming tuples from the child plan node will - * contain grouping columns, other columns referenced in our targetlist and - * qual, columns used to compute the aggregate functions, and perhaps just - * junk columns we don't use at all. Only columns of the first two types - * need to be stored in the hashtable, and getting rid of the others can - * make the table entries significantly smaller. To avoid messing up Var - * numbering, we keep the same tuple descriptor for hashtable entries as the - * incoming tuples have, but set unwanted columns to NULL in the tuples that - * go into the table. - * - * To eliminate duplicates, we build a bitmapset of the needed columns, then - * convert it to an integer list (cheaper to scan at runtime). The list is - * in decreasing order so that the first entry is the largest; - * lookup_hash_entry depends on this to use slot_getsomeattrs correctly. - * Note that the list is preserved over ExecReScanAgg, so we allocate it in - * the per-query context (unlike the hash table itself). - * - * Note: at present, searching the tlist/qual is not really necessary since - * the parser should disallow any unaggregated references to ungrouped - * columns. However, the search will be needed when we add support for - * SQL99 semantics that allow use of "functionally dependent" columns that - * haven't been explicitly grouped by. + * Compute columns that actually need to be stored in hashtable entries. The + * incoming tuples from the child plan node will contain grouping columns, + * other columns referenced in our targetlist and qual, columns used to + * compute the aggregate functions, and perhaps just junk columns we don't use + * at all. Only columns of the first two types need to be stored in the + * hashtable, and getting rid of the others can make the table entries + * significantly smaller. The hashtable only contains the relevant columns, + * and is packed/unpacked in lookup_hash_entry() / agg_retrieve_hash_table() + * into the format of the normal input descriptor. + * + * Additional columns, in addition to the columns grouped by, come from two + * sources: Firstly functionally dependent columns that we don't need to group + * by themselves, and secondly ctids for row-marks. + * + * To eliminate duplicates, we build a bitmapset of the needed columns, and + * then build an array of the columns included in the hashtable. Note that + * the array is preserved over ExecReScanAgg, so we allocate it in the + * per-query context (unlike the hash table itself). */ -static List * +static void find_hash_columns(AggState *aggstate) { - Agg *node = (Agg *) aggstate->ss.ps.plan; - Bitmapset *colnos; - List *collist; - int i; + Bitmapset *base_colnos; + List *outerTlist = outerPlanState(aggstate)->plan->targetlist; + int numHashes = aggstate->num_hashes; + int j; /* Find Vars that will be needed in tlist and qual */ - colnos = find_unaggregated_cols(aggstate); - /* Add in all the grouping columns */ - for (i = 0; i < node->numCols; i++) - colnos = bms_add_member(colnos, node->grpColIdx[i]); - /* Convert to list, using lcons so largest element ends up first */ - collist = NIL; - while ((i = bms_first_member(colnos)) >= 0) - collist = lcons_int(i, collist); - bms_free(colnos); - - return collist; + base_colnos = find_unaggregated_cols(aggstate); + + for (j = 0; j < numHashes; ++j) + { + AggStatePerHash perhash = &aggstate->perhash[j]; + Bitmapset *colnos = bms_copy(base_colnos); + AttrNumber *grpColIdx = perhash->aggnode->grpColIdx; + List *hashTlist = NIL; + TupleDesc hashDesc; + int i; + + perhash->largestGrpColIdx = 0; + + /* + * If we're doing grouping sets, then some Vars might be referenced in + * tlist/qual for the benefit of other grouping sets, but not needed + * when hashing; i.e. prepare_projection_slot will null them out, so + * there'd be no point storing them. Use prepare_projection_slot's + * logic to determine which. + */ + if (aggstate->phases[0].grouped_cols) + { + Bitmapset *grouped_cols = aggstate->phases[0].grouped_cols[j]; + ListCell *lc; + + foreach(lc, aggstate->all_grouped_cols) + { + int attnum = lfirst_int(lc); + + if (!bms_is_member(attnum, grouped_cols)) + colnos = bms_del_member(colnos, attnum); + } + } + /* Add in all the grouping columns */ + for (i = 0; i < perhash->numCols; i++) + colnos = bms_add_member(colnos, grpColIdx[i]); + + perhash->hashGrpColIdxInput = + palloc(bms_num_members(colnos) * sizeof(AttrNumber)); + perhash->hashGrpColIdxHash = + palloc(perhash->numCols * sizeof(AttrNumber)); + + /* + * First build mapping for columns directly hashed. These are the + * first, because they'll be accessed when computing hash values and + * comparing tuples for exact matches. We also build simple mapping + * for execGrouping, so it knows where to find the to-be-hashed / + * compared columns in the input. + */ + for (i = 0; i < perhash->numCols; i++) + { + perhash->hashGrpColIdxInput[i] = grpColIdx[i]; + perhash->hashGrpColIdxHash[i] = i + 1; + perhash->numhashGrpCols++; + /* delete already mapped columns */ + bms_del_member(colnos, grpColIdx[i]); + } + + /* and add the remaining columns */ + while ((i = bms_first_member(colnos)) >= 0) + { + perhash->hashGrpColIdxInput[perhash->numhashGrpCols] = i; + perhash->numhashGrpCols++; + } + + /* and build a tuple descriptor for the hashtable */ + for (i = 0; i < perhash->numhashGrpCols; i++) + { + int varNumber = perhash->hashGrpColIdxInput[i] - 1; + + hashTlist = lappend(hashTlist, list_nth(outerTlist, varNumber)); + perhash->largestGrpColIdx = + Max(varNumber + 1, perhash->largestGrpColIdx); + } + + hashDesc = ExecTypeFromTL(hashTlist, false); + ExecSetSlotDescriptor(perhash->hashslot, hashDesc); + + list_free(hashTlist); + bms_free(colnos); + } + + bms_free(base_colnos); } /* @@ -1724,6 +1998,8 @@ find_hash_columns(AggState *aggstate) * * Note that the estimate does not include space for pass-by-reference * transition data values, nor for the representative tuple of each group. + * Nor does this account of the target fill-factor and growth policy of the + * hash table. */ Size hash_agg_entry_size(int numAggs) @@ -1731,61 +2007,83 @@ hash_agg_entry_size(int numAggs) Size entrysize; /* This must match build_hash_table */ - entrysize = offsetof(AggHashEntryData, pergroup) + + entrysize = sizeof(TupleHashEntryData) + numAggs * sizeof(AggStatePerGroupData); entrysize = MAXALIGN(entrysize); - /* Account for hashtable overhead (assuming fill factor = 1) */ - entrysize += 3 * sizeof(void *); + return entrysize; } /* - * Find or create a hashtable entry for the tuple group containing the - * given tuple. + * Find or create a hashtable entry for the tuple group containing the current + * tuple (already set in tmpcontext's outertuple slot), in the current grouping + * set (which the caller must have selected - note that initialize_aggregate + * depends on this). * * When called, CurrentMemoryContext should be the per-query context. */ -static AggHashEntry -lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot) +static TupleHashEntryData * +lookup_hash_entry(AggState *aggstate) { - TupleTableSlot *hashslot = aggstate->hashslot; - ListCell *l; - AggHashEntry entry; + TupleTableSlot *inputslot = aggstate->tmpcontext->ecxt_outertuple; + AggStatePerHash perhash = &aggstate->perhash[aggstate->current_set]; + TupleTableSlot *hashslot = perhash->hashslot; + TupleHashEntryData *entry; bool isnew; - - /* if first time through, initialize hashslot by cloning input slot */ - if (hashslot->tts_tupleDescriptor == NULL) - { - ExecSetSlotDescriptor(hashslot, inputslot->tts_tupleDescriptor); - /* Make sure all unused columns are NULLs */ - ExecStoreAllNullTuple(hashslot); - } + int i; /* transfer just the needed columns into hashslot */ - slot_getsomeattrs(inputslot, linitial_int(aggstate->hash_needed)); - foreach(l, aggstate->hash_needed) + slot_getsomeattrs(inputslot, perhash->largestGrpColIdx); + ExecClearTuple(hashslot); + + for (i = 0; i < perhash->numhashGrpCols; i++) { - int varNumber = lfirst_int(l) - 1; + int varNumber = perhash->hashGrpColIdxInput[i] - 1; - hashslot->tts_values[varNumber] = inputslot->tts_values[varNumber]; - hashslot->tts_isnull[varNumber] = inputslot->tts_isnull[varNumber]; + hashslot->tts_values[i] = inputslot->tts_values[varNumber]; + hashslot->tts_isnull[i] = inputslot->tts_isnull[varNumber]; } + ExecStoreVirtualTuple(hashslot); /* find or create the hashtable entry using the filtered tuple */ - entry = (AggHashEntry) LookupTupleHashEntry(aggstate->hashtable, - hashslot, - &isnew); + entry = LookupTupleHashEntry(perhash->hashtable, hashslot, &isnew); if (isnew) { + entry->additional = (AggStatePerGroup) + MemoryContextAlloc(perhash->hashtable->tablecxt, + sizeof(AggStatePerGroupData) * aggstate->numtrans); /* initialize aggregates for new tuple group */ - initialize_aggregates(aggstate, entry->pergroup, 0); + initialize_aggregates(aggstate, (AggStatePerGroup) entry->additional, + -1); } return entry; } /* + * Look up hash entries for the current tuple in all hashed grouping sets, + * returning an array of pergroup pointers suitable for advance_aggregates. + * + * Be aware that lookup_hash_entry can reset the tmpcontext. + */ +static AggStatePerGroup * +lookup_hash_entries(AggState *aggstate) +{ + int numHashes = aggstate->num_hashes; + AggStatePerGroup *pergroup = aggstate->hash_pergroup; + int setno; + + for (setno = 0; setno < numHashes; setno++) + { + select_current_set(aggstate, setno, true); + pergroup[setno] = lookup_hash_entry(aggstate)->additional; + } + + return pergroup; +} + +/* * ExecAgg - * * ExecAgg receives tuples from its outer subplan and aggregates over @@ -1801,40 +2099,22 @@ lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot) TupleTableSlot * ExecAgg(AggState *node) { - TupleTableSlot *result; - - /* - * Check to see if we're still projecting out tuples from a previous agg - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->ss.ps.ps_TupFromTlist) - { - ExprDoneCond isDone; - - result = ExecProject(node->ss.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return result; - /* Done with that source tuple... */ - node->ss.ps.ps_TupFromTlist = false; - } + TupleTableSlot *result = NULL; - /* - * (We must do the ps_TupFromTlist check first, because in some cases - * agg_done gets set before we emit the final aggregate tuple, and we have - * to finish running SRFs for it.) - */ if (!node->agg_done) { /* Dispatch based on strategy */ - switch (node->phase->aggnode->aggstrategy) + switch (node->phase->aggstrategy) { case AGG_HASHED: if (!node->table_filled) agg_fill_hash_table(node); + /* FALLTHROUGH */ + case AGG_MIXED: result = agg_retrieve_hash_table(node); break; - default: + case AGG_PLAIN: + case AGG_SORTED: result = agg_retrieve_direct(node); break; } @@ -1857,6 +2137,7 @@ agg_retrieve_direct(AggState *aggstate) ExprContext *tmpcontext; AggStatePerAgg peragg; AggStatePerGroup pergroup; + AggStatePerGroup *hash_pergroups = NULL; TupleTableSlot *outerslot; TupleTableSlot *firstSlot; TupleTableSlot *result; @@ -1943,6 +2224,19 @@ agg_retrieve_direct(AggState *aggstate) node = aggstate->phase->aggnode; numReset = numGroupingSets; } + else if (aggstate->aggstrategy == AGG_MIXED) + { + /* + * Mixed mode; we've output all the grouped stuff and have + * full hashtables, so switch to outputting those. + */ + initialize_phase(aggstate, 0); + aggstate->table_filled = true; + ResetTupleHashIterator(aggstate->perhash[0].hashtable, + &aggstate->perhash[0].hashiter); + select_current_set(aggstate, 0, true); + return agg_retrieve_hash_table(aggstate); + } else { aggstate->agg_done = true; @@ -1979,7 +2273,7 @@ agg_retrieve_direct(AggState *aggstate) *---------- */ if (aggstate->input_done || - (node->aggstrategy == AGG_SORTED && + (node->aggstrategy != AGG_PLAIN && aggstate->projected_set != -1 && aggstate->projected_set < (numGroupingSets - 1) && nextSetSize > 0 && @@ -2092,10 +2386,22 @@ agg_retrieve_direct(AggState *aggstate) */ for (;;) { + /* + * During phase 1 only of a mixed agg, we need to update + * hashtables as well in advance_aggregates. + */ + if (aggstate->aggstrategy == AGG_MIXED && + aggstate->current_phase == 1) + { + hash_pergroups = lookup_hash_entries(aggstate); + } + else + hash_pergroups = NULL; + if (DO_AGGSPLIT_COMBINE(aggstate->aggsplit)) combine_aggregates(aggstate, pergroup); else - advance_aggregates(aggstate, pergroup); + advance_aggregates(aggstate, pergroup, hash_pergroups); /* Reset per-input-tuple context after each tuple */ ResetExprContext(tmpcontext); @@ -2122,7 +2428,7 @@ agg_retrieve_direct(AggState *aggstate) * If we are grouping, check whether we've crossed a group * boundary. */ - if (node->aggstrategy == AGG_SORTED) + if (node->aggstrategy != AGG_PLAIN) { if (!execTuplesMatch(firstSlot, outerslot, @@ -2155,7 +2461,11 @@ agg_retrieve_direct(AggState *aggstate) prepare_projection_slot(aggstate, econtext->ecxt_outertuple, currentSet); - finalize_aggregates(aggstate, peragg, pergroup, currentSet); + select_current_set(aggstate, currentSet, false); + + finalize_aggregates(aggstate, + peragg, + pergroup + (currentSet * aggstate->numtrans)); /* * If there's no row to project right now, we must continue rather @@ -2171,21 +2481,13 @@ agg_retrieve_direct(AggState *aggstate) } /* - * ExecAgg for hashed case: phase 1, read input and build hash table + * ExecAgg for hashed case: read input and build hash table */ static void agg_fill_hash_table(AggState *aggstate) { - ExprContext *tmpcontext; - AggHashEntry entry; TupleTableSlot *outerslot; - - /* - * get state info from node - * - * tmpcontext is the per-input-tuple expression context - */ - tmpcontext = aggstate->tmpcontext; + ExprContext *tmpcontext = aggstate->tmpcontext; /* * Process each outer-plan tuple, and then fetch the next one, until we @@ -2193,32 +2495,40 @@ agg_fill_hash_table(AggState *aggstate) */ for (;;) { + AggStatePerGroup *pergroups; + outerslot = fetch_input_tuple(aggstate); if (TupIsNull(outerslot)) break; - /* set up for advance_aggregates call */ + + /* set up for lookup_hash_entries and advance_aggregates */ tmpcontext->ecxt_outertuple = outerslot; - /* Find or build hashtable entry for this tuple's group */ - entry = lookup_hash_entry(aggstate, outerslot); + /* Find or build hashtable entries */ + pergroups = lookup_hash_entries(aggstate); /* Advance the aggregates */ if (DO_AGGSPLIT_COMBINE(aggstate->aggsplit)) - combine_aggregates(aggstate, entry->pergroup); + combine_aggregates(aggstate, pergroups[0]); else - advance_aggregates(aggstate, entry->pergroup); + advance_aggregates(aggstate, NULL, pergroups); - /* Reset per-input-tuple context after each tuple */ - ResetExprContext(tmpcontext); + /* + * Reset per-input-tuple context after each tuple, but note that the + * hash lookups do this too + */ + ResetExprContext(aggstate->tmpcontext); } aggstate->table_filled = true; - /* Initialize to walk the hash table */ - ResetTupleHashIterator(aggstate->hashtable, &aggstate->hashiter); + /* Initialize to walk the first hash table */ + select_current_set(aggstate, 0, true); + ResetTupleHashIterator(aggstate->perhash[0].hashtable, + &aggstate->perhash[0].hashiter); } /* - * ExecAgg for hashed case: phase 2, retrieving groups from hash table + * ExecAgg for hashed case: retrieving groups from hash table */ static TupleTableSlot * agg_retrieve_hash_table(AggState *aggstate) @@ -2226,33 +2536,63 @@ agg_retrieve_hash_table(AggState *aggstate) ExprContext *econtext; AggStatePerAgg peragg; AggStatePerGroup pergroup; - AggHashEntry entry; + TupleHashEntryData *entry; TupleTableSlot *firstSlot; TupleTableSlot *result; + AggStatePerHash perhash; /* - * get state info from node + * get state info from node. + * + * econtext is the per-output-tuple expression context. */ - /* econtext is the per-output-tuple expression context */ econtext = aggstate->ss.ps.ps_ExprContext; peragg = aggstate->peragg; firstSlot = aggstate->ss.ss_ScanTupleSlot; /* + * Note that perhash (and therefore anything accessed through it) can + * change inside the loop, as we change between grouping sets. + */ + perhash = &aggstate->perhash[aggstate->current_set]; + + /* * We loop retrieving groups until we find one satisfying * aggstate->ss.ps.qual */ while (!aggstate->agg_done) { + TupleTableSlot *hashslot = perhash->hashslot; + int i; + /* * Find the next entry in the hash table */ - entry = (AggHashEntry) ScanTupleHashTable(&aggstate->hashiter); + entry = ScanTupleHashTable(perhash->hashtable, &perhash->hashiter); if (entry == NULL) { - /* No more entries in hashtable, so done */ - aggstate->agg_done = TRUE; - return NULL; + int nextset = aggstate->current_set + 1; + + if (nextset < aggstate->num_hashes) + { + /* + * Switch to next grouping set, reinitialize, and restart the + * loop. + */ + select_current_set(aggstate, nextset, true); + + perhash = &aggstate->perhash[aggstate->current_set]; + + ResetTupleHashIterator(perhash->hashtable, &perhash->hashiter); + + continue; + } + else + { + /* No more hashtables, so done */ + aggstate->agg_done = TRUE; + return NULL; + } } /* @@ -2265,16 +2605,26 @@ agg_retrieve_hash_table(AggState *aggstate) ResetExprContext(econtext); /* - * Store the copied first input tuple in the tuple table slot reserved - * for it, so that it can be used in ExecProject. + * Transform representative tuple back into one with the right + * columns. */ - ExecStoreMinimalTuple(entry->shared.firstTuple, - firstSlot, - false); + ExecStoreMinimalTuple(entry->firstTuple, hashslot, false); + slot_getallattrs(hashslot); - pergroup = entry->pergroup; + ExecClearTuple(firstSlot); + memset(firstSlot->tts_isnull, true, + firstSlot->tts_tupleDescriptor->natts * sizeof(bool)); - finalize_aggregates(aggstate, peragg, pergroup, 0); + for (i = 0; i < perhash->numhashGrpCols; i++) + { + int varNumber = perhash->hashGrpColIdxInput[i] - 1; + + firstSlot->tts_values[varNumber] = hashslot->tts_values[i]; + firstSlot->tts_isnull[varNumber] = hashslot->tts_isnull[i]; + } + ExecStoreVirtualTuple(firstSlot); + + pergroup = (AggStatePerGroup) entry->additional; /* * Use the representative input tuple for any references to @@ -2282,6 +2632,12 @@ agg_retrieve_hash_table(AggState *aggstate) */ econtext->ecxt_outertuple = firstSlot; + prepare_projection_slot(aggstate, + econtext->ecxt_outertuple, + aggstate->current_set); + + finalize_aggregates(aggstate, peragg, pergroup); + result = project_aggregates(aggstate); if (result) return result; @@ -2295,7 +2651,8 @@ agg_retrieve_hash_table(AggState *aggstate) * ExecInitAgg * * Creates the run-time information for the agg node produced by the - * planner and initializes its outer subtree + * planner and initializes its outer subtree. + * * ----------------- */ AggState * @@ -2310,12 +2667,18 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) transno, aggno; int phase; + int phaseidx; + List *combined_inputeval; ListCell *l; Bitmapset *all_grouped_cols = NULL; int numGroupingSets = 1; int numPhases; + int numHashes; + int column_offset; int i = 0; int j = 0; + bool use_hashing = (node->aggstrategy == AGG_HASHED || + node->aggstrategy == AGG_MIXED); /* check for unsupported flags */ Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK))); @@ -2330,9 +2693,9 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->aggs = NIL; aggstate->numaggs = 0; aggstate->numtrans = 0; + aggstate->aggstrategy = node->aggstrategy; aggstate->aggsplit = node->aggsplit; aggstate->maxsets = 0; - aggstate->hashfunctions = NULL; aggstate->projected_set = -1; aggstate->current_set = 0; aggstate->peragg = NULL; @@ -2342,18 +2705,22 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->agg_done = false; aggstate->pergroup = NULL; aggstate->grp_firstTuple = NULL; - aggstate->hashtable = NULL; aggstate->sort_in = NULL; aggstate->sort_out = NULL; /* + * phases[0] always exists, but is dummy in sorted/plain mode + */ + numPhases = (use_hashing ? 1 : 2); + numHashes = (use_hashing ? 1 : 0); + + /* * Calculate the maximum number of grouping sets in any phase; this - * determines the size of some allocations. + * determines the size of some allocations. Also calculate the number of + * phases, since all hashed/mixed nodes contribute to only a single phase. */ if (node->groupingSets) { - Assert(node->aggstrategy != AGG_HASHED); - numGroupingSets = list_length(node->groupingSets); foreach(l, node->chain) @@ -2362,22 +2729,32 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) numGroupingSets = Max(numGroupingSets, list_length(agg->groupingSets)); + + /* + * additional AGG_HASHED aggs become part of phase 0, but all + * others add an extra phase. + */ + if (agg->aggstrategy != AGG_HASHED) + ++numPhases; + else + ++numHashes; } } aggstate->maxsets = numGroupingSets; - aggstate->numphases = numPhases = 1 + list_length(node->chain); + aggstate->numphases = numPhases; aggstate->aggcontexts = (ExprContext **) palloc0(sizeof(ExprContext *) * numGroupingSets); /* * Create expression contexts. We need three or more, one for - * per-input-tuple processing, one for per-output-tuple processing, and - * one for each grouping set. The per-tuple memory context of the - * per-grouping-set ExprContexts (aggcontexts) replaces the standalone - * memory context formerly used to hold transition values. We cheat a - * little by using ExecAssignExprContext() to build all of them. + * per-input-tuple processing, one for per-output-tuple processing, one + * for all the hashtables, and one for each grouping set. The per-tuple + * memory context of the per-grouping-set ExprContexts (aggcontexts) + * replaces the standalone memory context formerly used to hold transition + * values. We cheat a little by using ExecAssignExprContext() to build + * all of them. * * NOTE: the details of what is stored in aggcontexts and what is stored * in the regular per-query memory context are driven by a simple @@ -2393,31 +2770,37 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->aggcontexts[i] = aggstate->ss.ps.ps_ExprContext; } + if (use_hashing) + { + ExecAssignExprContext(estate, &aggstate->ss.ps); + aggstate->hashcontext = aggstate->ss.ps.ps_ExprContext; + } + ExecAssignExprContext(estate, &aggstate->ss.ps); /* - * tuple table initialization + * tuple table initialization. + * + * For hashtables, we create some additional slots below. */ ExecInitScanTupleSlot(estate, &aggstate->ss); ExecInitResultTupleSlot(estate, &aggstate->ss.ps); - aggstate->hashslot = ExecInitExtraTupleSlot(estate); aggstate->sort_slot = ExecInitExtraTupleSlot(estate); /* * initialize child expressions * - * Note: ExecInitExpr finds Aggrefs for us, and also checks that no aggs - * contain other agg calls in their arguments. This would make no sense - * under SQL semantics anyway (and it's forbidden by the spec). Because - * that is true, we don't need to worry about evaluating the aggs in any - * particular order. + * We rely on the parser to have checked that no aggs contain other agg + * calls in their arguments. This would make no sense under SQL semantics + * (and it's forbidden by the spec). Because it is true, we don't need to + * worry about evaluating the aggs in any particular order. + * + * Note: execExpr.c finds Aggrefs for us, and adds their AggrefExprState + * nodes to aggstate->aggs. Aggrefs in the qual are found here; Aggrefs + * in the targetlist are found during ExecAssignProjectionInfo, below. */ - aggstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) aggstate); - aggstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->plan.qual, - (PlanState *) aggstate); + aggstate->ss.ps.qual = + ExecInitQual(node->plan.qual, (PlanState *) aggstate); /* * Initialize child nodes. @@ -2444,10 +2827,8 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) ExecAssignResultTypeFromTL(&aggstate->ss.ps); ExecAssignProjectionInfo(&aggstate->ss.ps, NULL); - aggstate->ss.ps.ps_TupFromTlist = false; - /* - * get the count of aggregates in targetlist and quals + * We should now have found all Aggrefs in the targetlist and quals. */ numaggs = aggstate->numaggs; Assert(numaggs == list_length(aggstate->aggs)); @@ -2467,21 +2848,27 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) * For each phase, prepare grouping set data and fmgr lookup data for * compare functions. Accumulate all_grouped_cols in passing. */ - aggstate->phases = palloc0(numPhases * sizeof(AggStatePerPhaseData)); - for (phase = 0; phase < numPhases; ++phase) + aggstate->num_hashes = numHashes; + if (numHashes) + { + aggstate->perhash = palloc0(sizeof(AggStatePerHashData) * numHashes); + aggstate->phases[0].numsets = 0; + aggstate->phases[0].gset_lengths = palloc(numHashes * sizeof(int)); + aggstate->phases[0].grouped_cols = palloc(numHashes * sizeof(Bitmapset *)); + } + + phase = 0; + for (phaseidx = 0; phaseidx <= list_length(node->chain); ++phaseidx) { - AggStatePerPhase phasedata = &aggstate->phases[phase]; Agg *aggnode; Sort *sortnode; - int num_sets; - if (phase > 0) + if (phaseidx > 0) { - aggnode = list_nth(node->chain, phase - 1); - sortnode = (Sort *) aggnode->plan.lefttree; - Assert(IsA(sortnode, Sort)); + aggnode = list_nth_node(Agg, node->chain, phaseidx - 1); + sortnode = castNode(Sort, aggnode->plan.lefttree); } else { @@ -2489,53 +2876,91 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) sortnode = NULL; } - phasedata->numsets = num_sets = list_length(aggnode->groupingSets); + Assert(phase <= 1 || sortnode); - if (num_sets) + if (aggnode->aggstrategy == AGG_HASHED + || aggnode->aggstrategy == AGG_MIXED) { - phasedata->gset_lengths = palloc(num_sets * sizeof(int)); - phasedata->grouped_cols = palloc(num_sets * sizeof(Bitmapset *)); + AggStatePerPhase phasedata = &aggstate->phases[0]; + AggStatePerHash perhash; + Bitmapset *cols = NULL; - i = 0; - foreach(l, aggnode->groupingSets) - { - int current_length = list_length(lfirst(l)); - Bitmapset *cols = NULL; + Assert(phase == 0); + i = phasedata->numsets++; + perhash = &aggstate->perhash[i]; - /* planner forces this to be correct */ - for (j = 0; j < current_length; ++j) - cols = bms_add_member(cols, aggnode->grpColIdx[j]); + /* phase 0 always points to the "real" Agg in the hash case */ + phasedata->aggnode = node; + phasedata->aggstrategy = node->aggstrategy; - phasedata->grouped_cols[i] = cols; - phasedata->gset_lengths[i] = current_length; - ++i; - } + /* but the actual Agg node representing this hash is saved here */ + perhash->aggnode = aggnode; - all_grouped_cols = bms_add_members(all_grouped_cols, - phasedata->grouped_cols[0]); + phasedata->gset_lengths[i] = perhash->numCols = aggnode->numCols; + + for (j = 0; j < aggnode->numCols; ++j) + cols = bms_add_member(cols, aggnode->grpColIdx[j]); + + phasedata->grouped_cols[i] = cols; + + all_grouped_cols = bms_add_members(all_grouped_cols, cols); + continue; } else { - Assert(phase == 0); + AggStatePerPhase phasedata = &aggstate->phases[++phase]; + int num_sets; - phasedata->gset_lengths = NULL; - phasedata->grouped_cols = NULL; - } + phasedata->numsets = num_sets = list_length(aggnode->groupingSets); - /* - * If we are grouping, precompute fmgr lookup data for inner loop. - */ - if (aggnode->aggstrategy == AGG_SORTED) - { - Assert(aggnode->numCols > 0); + if (num_sets) + { + phasedata->gset_lengths = palloc(num_sets * sizeof(int)); + phasedata->grouped_cols = palloc(num_sets * sizeof(Bitmapset *)); - phasedata->eqfunctions = - execTuplesMatchPrepare(aggnode->numCols, - aggnode->grpOperators); - } + i = 0; + foreach(l, aggnode->groupingSets) + { + int current_length = list_length(lfirst(l)); + Bitmapset *cols = NULL; + + /* planner forces this to be correct */ + for (j = 0; j < current_length; ++j) + cols = bms_add_member(cols, aggnode->grpColIdx[j]); + + phasedata->grouped_cols[i] = cols; + phasedata->gset_lengths[i] = current_length; + + ++i; + } + + all_grouped_cols = bms_add_members(all_grouped_cols, + phasedata->grouped_cols[0]); + } + else + { + Assert(phaseidx == 0); + + phasedata->gset_lengths = NULL; + phasedata->grouped_cols = NULL; + } + + /* + * If we are grouping, precompute fmgr lookup data for inner loop. + */ + if (aggnode->aggstrategy == AGG_SORTED) + { + Assert(aggnode->numCols > 0); - phasedata->aggnode = aggnode; - phasedata->sortnode = sortnode; + phasedata->eqfunctions = + execTuplesMatchPrepare(aggnode->numCols, + aggnode->grpOperators); + } + + phasedata->aggnode = aggnode; + phasedata->aggstrategy = aggnode->aggstrategy; + phasedata->sortnode = sortnode; + } } /* @@ -2546,23 +2971,6 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->all_grouped_cols = lcons_int(i, aggstate->all_grouped_cols); /* - * Hashing can only appear in the initial phase. - */ - - if (node->aggstrategy == AGG_HASHED) - execTuplesHashPrepare(node->numCols, - node->grpOperators, - &aggstate->phases[0].eqfunctions, - &aggstate->hashfunctions); - - /* - * Initialize current phase-dependent values to initial phase - */ - - aggstate->current_phase = 0; - initialize_phase(aggstate, 0); - - /* * Set up aggregate-result storage in the output expr context, and also * allocate my private per-agg working storage */ @@ -2576,14 +2984,30 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->peragg = peraggs; aggstate->pertrans = pertransstates; - if (node->aggstrategy == AGG_HASHED) + /* + * Hashing can only appear in the initial phase. + */ + if (use_hashing) { + for (i = 0; i < numHashes; ++i) + { + aggstate->perhash[i].hashslot = ExecInitExtraTupleSlot(estate); + + execTuplesHashPrepare(aggstate->perhash[i].numCols, + aggstate->perhash[i].aggnode->grpOperators, + &aggstate->perhash[i].eqfunctions, + &aggstate->perhash[i].hashfunctions); + } + + /* this is an array of pointers, not structures */ + aggstate->hash_pergroup = palloc0(sizeof(AggStatePerGroup) * numHashes); + + find_hash_columns(aggstate); build_hash_table(aggstate); aggstate->table_filled = false; - /* Compute the columns we actually need to hash on */ - aggstate->hash_needed = find_hash_columns(aggstate); } - else + + if (node->aggstrategy != AGG_HASHED) { AggStatePerGroup pergroup; @@ -2594,6 +3018,25 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->pergroup = pergroup; } + /* + * Initialize current phase-dependent values to initial phase. The initial + * phase is 1 (first sort pass) for all strategies that use sorting (if + * hashing is being done too, then phase 0 is processed last); but if only + * hashing is being done, then phase 0 is all there is. + */ + if (node->aggstrategy == AGG_HASHED) + { + aggstate->current_phase = 0; + initialize_phase(aggstate, 0); + select_current_set(aggstate, 0, true); + } + else + { + aggstate->current_phase = 1; + initialize_phase(aggstate, 1); + select_current_set(aggstate, 0, false); + } + /* ----------------- * Perform lookups of aggregate function info, and initialize the * unchanging fields of the per-agg and per-trans data. @@ -2633,7 +3076,7 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) foreach(l, aggstate->aggs) { AggrefExprState *aggrefstate = (AggrefExprState *) lfirst(l); - Aggref *aggref = (Aggref *) aggrefstate->xprstate.expr; + Aggref *aggref = aggrefstate->aggref; AggStatePerAgg peragg; AggStatePerTrans pertrans; int existing_aggno; @@ -2895,6 +3338,51 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->numaggs = aggno + 1; aggstate->numtrans = transno + 1; + /* + * Build a single projection computing the aggregate arguments for all + * aggregates at once, that's considerably faster than doing it separately + * for each. + * + * First create a targetlist combining the targetlist of all the + * transitions. + */ + combined_inputeval = NIL; + column_offset = 0; + for (transno = 0; transno < aggstate->numtrans; transno++) + { + AggStatePerTrans pertrans = &pertransstates[transno]; + ListCell *arg; + + pertrans->inputoff = column_offset; + + /* + * Adjust resno in a copied target entries, to point into the combined + * slot. + */ + foreach(arg, pertrans->aggref->args) + { + TargetEntry *source_tle = lfirst_node(TargetEntry, arg); + TargetEntry *tle; + + tle = flatCopyTargetEntry(source_tle); + tle->resno += column_offset; + + combined_inputeval = lappend(combined_inputeval, tle); + } + + column_offset += list_length(pertrans->aggref->args); + } + + /* and then create a projection for that targetlist */ + aggstate->evaldesc = ExecTypeFromTL(combined_inputeval, false); + aggstate->evalslot = ExecInitExtraTupleSlot(estate); + aggstate->evalproj = ExecBuildProjectionInfo(combined_inputeval, + aggstate->tmpcontext, + aggstate->evalslot, + &aggstate->ss.ps, + NULL); + ExecSetSlotDescriptor(aggstate->evalslot, aggstate->evaldesc); + return aggstate; } @@ -3065,24 +3553,12 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans, } - /* - * Get a tupledesc corresponding to the aggregated inputs (including sort - * expressions) of the agg. - */ - pertrans->evaldesc = ExecTypeFromTL(aggref->args, false); - - /* Create slot we're going to do argument evaluation in */ - pertrans->evalslot = ExecInitExtraTupleSlot(estate); - ExecSetSlotDescriptor(pertrans->evalslot, pertrans->evaldesc); - /* Initialize the input and FILTER expressions */ naggs = aggstate->numaggs; pertrans->aggfilter = ExecInitExpr(aggref->aggfilter, (PlanState *) aggstate); - pertrans->aggdirectargs = (List *) ExecInitExpr((Expr *) aggref->aggdirectargs, - (PlanState *) aggstate); - pertrans->args = (List *) ExecInitExpr((Expr *) aggref->args, - (PlanState *) aggstate); + pertrans->aggdirectargs = ExecInitExprList(aggref->aggdirectargs, + (PlanState *) aggstate); /* * Complain if the aggregate's arguments contain any aggregates; nested @@ -3094,12 +3570,6 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans, (errcode(ERRCODE_GROUPING_ERROR), errmsg("aggregate function calls cannot be nested"))); - /* Set up projection info for evaluation */ - pertrans->evalproj = ExecBuildProjectionInfo(pertrans->args, - aggstate->tmpcontext, - pertrans->evalslot, - NULL); - /* * If we're doing either DISTINCT or ORDER BY for a plain agg, then we * have a list of SortGroupClause nodes; fish out the data in them and @@ -3133,10 +3603,18 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans, if (numSortCols > 0) { /* + * Get a tupledesc and slot corresponding to the aggregated inputs + * (including sort expressions) of the agg. + */ + pertrans->sortdesc = ExecTypeFromTL(aggref->args, false); + pertrans->sortslot = ExecInitExtraTupleSlot(estate); + ExecSetSlotDescriptor(pertrans->sortslot, pertrans->sortdesc); + + /* * We don't implement DISTINCT or ORDER BY aggs in the HASHED case * (yet) */ - Assert(((Agg *) aggstate->ss.ps.plan)->aggstrategy != AGG_HASHED); + Assert(aggstate->aggstrategy != AGG_HASHED && aggstate->aggstrategy != AGG_MIXED); /* If we have only one input, we need its len/byval info. */ if (numInputs == 1) @@ -3150,7 +3628,7 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans, /* we will need an extra slot to store prior values */ pertrans->uniqslot = ExecInitExtraTupleSlot(estate); ExecSetSlotDescriptor(pertrans->uniqslot, - pertrans->evaldesc); + pertrans->sortdesc); } /* Extract the sort information for use later */ @@ -3385,6 +3863,8 @@ ExecEndAgg(AggState *node) /* And ensure any agg shutdown callbacks have been called */ for (setno = 0; setno < numGroupingSets; setno++) ReScanExprContext(node->aggcontexts[setno]); + if (node->hashcontext) + ReScanExprContext(node->hashcontext); /* * We don't actually free any ExprContexts here (see comment in @@ -3412,9 +3892,7 @@ ExecReScanAgg(AggState *node) node->agg_done = false; - node->ss.ps.ps_TupFromTlist = false; - - if (aggnode->aggstrategy == AGG_HASHED) + if (node->aggstrategy == AGG_HASHED) { /* * In the hashed case, if we haven't yet built the hash table then we @@ -3426,13 +3904,17 @@ ExecReScanAgg(AggState *node) return; /* - * If we do have the hash table and the subplan does not have any - * parameter changes, then we can just rescan the existing hash table; - * no need to build it again. + * If we do have the hash table, and the subplan does not have any + * parameter changes, and none of our own parameter changes affect + * input expressions of the aggregated functions, then we can just + * rescan the existing hash table; no need to build it again. */ - if (outerPlan->chgParam == NULL) + if (outerPlan->chgParam == NULL && + !bms_overlap(node->ss.ps.chgParam, aggnode->aggParams)) { - ResetTupleHashIterator(node->hashtable, &node->hashiter); + ResetTupleHashIterator(node->perhash[0].hashtable, + &node->perhash[0].hashiter); + select_current_set(node, 0, true); return; } } @@ -3457,11 +3939,7 @@ ExecReScanAgg(AggState *node) * ExecReScan already did it. But we do need to reset our per-grouping-set * contexts, which may have transvalues stored in them. (We use rescan * rather than just reset because transfns may have registered callbacks - * that need to be run now.) - * - * Note that with AGG_HASHED, the hash table is allocated in a sub-context - * of the aggcontext. This used to be an issue, but now, resetting a - * context automatically deletes sub-contexts too. + * that need to be run now.) For the AGG_HASHED case, see below. */ for (setno = 0; setno < numGroupingSets; setno++) @@ -3481,13 +3959,21 @@ ExecReScanAgg(AggState *node) MemSet(econtext->ecxt_aggvalues, 0, sizeof(Datum) * node->numaggs); MemSet(econtext->ecxt_aggnulls, 0, sizeof(bool) * node->numaggs); - if (aggnode->aggstrategy == AGG_HASHED) + /* + * With AGG_HASHED/MIXED, the hash table is allocated in a sub-context of + * the hashcontext. This used to be an issue, but now, resetting a context + * automatically deletes sub-contexts too. + */ + if (node->aggstrategy == AGG_HASHED || node->aggstrategy == AGG_MIXED) { + ReScanExprContext(node->hashcontext); /* Rebuild an empty hash table */ build_hash_table(node); node->table_filled = false; + /* iterator will be reset when the table is filled */ } - else + + if (node->aggstrategy != AGG_HASHED) { /* * Reset the per-group state (in particular, mark transvalues null) @@ -3495,8 +3981,8 @@ ExecReScanAgg(AggState *node) MemSet(node->pergroup, 0, sizeof(AggStatePerGroupData) * node->numaggs * numGroupingSets); - /* reset to phase 0 */ - initialize_phase(node, 0); + /* reset to phase 1 */ + initialize_phase(node, 1); node->input_done = false; node->projected_set = -1; @@ -3537,7 +4023,7 @@ AggCheckCallContext(FunctionCallInfo fcinfo, MemoryContext *aggcontext) if (aggcontext) { AggState *aggstate = ((AggState *) fcinfo->context); - ExprContext *cxt = aggstate->aggcontexts[aggstate->current_set]; + ExprContext *cxt = aggstate->curaggcontext; *aggcontext = cxt->ecxt_per_tuple_memory; } @@ -3626,7 +4112,7 @@ AggRegisterCallback(FunctionCallInfo fcinfo, if (fcinfo->context && IsA(fcinfo->context, AggState)) { AggState *aggstate = (AggState *) fcinfo->context; - ExprContext *cxt = aggstate->aggcontexts[aggstate->current_set]; + ExprContext *cxt = aggstate->curaggcontext; RegisterExprContextCallback(cxt, func, arg); diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c index a26bd6354c..aae5e3fa63 100644 --- a/src/backend/executor/nodeAppend.c +++ b/src/backend/executor/nodeAppend.c @@ -3,7 +3,7 @@ * nodeAppend.c * routines to handle append nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -129,6 +129,12 @@ ExecInitAppend(Append *node, EState *estate, int eflags) Assert(!(eflags & EXEC_FLAG_MARK)); /* + * Lock the non-leaf tables in the partition tree controlled by this node. + * It's a no-op for non-partitioned parent tables. + */ + ExecLockNonLeafAppendTables(node->partitioned_rels, estate); + + /* * Set up empty vector of subplan states */ nplans = list_length(node->appendplans); diff --git a/src/backend/executor/nodeBitmapAnd.c b/src/backend/executor/nodeBitmapAnd.c index c39d790f82..e4eb028ff9 100644 --- a/src/backend/executor/nodeBitmapAnd.c +++ b/src/backend/executor/nodeBitmapAnd.c @@ -3,7 +3,7 @@ * nodeBitmapAnd.c * routines to handle BitmapAnd nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c index 449aacb6e7..c453362230 100644 --- a/src/backend/executor/nodeBitmapHeapscan.c +++ b/src/backend/executor/nodeBitmapHeapscan.c @@ -16,7 +16,7 @@ * required index qual conditions. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -35,6 +35,8 @@ */ #include "postgres.h" +#include <math.h> + #include "access/relscan.h" #include "access/transam.h" #include "executor/execdebug.h" @@ -51,6 +53,15 @@ static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node); static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres); +static inline void BitmapDoneInitializingSharedState( + ParallelBitmapHeapState *pstate); +static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node, + TBMIterateResult *tbmres); +static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node); +static inline void BitmapPrefetch(BitmapHeapScanState *node, + HeapScanDesc scan); +static bool BitmapShouldInitializeSharedState( + ParallelBitmapHeapState *pstate); /* ---------------------------------------------------------------- @@ -65,14 +76,13 @@ BitmapHeapNext(BitmapHeapScanState *node) ExprContext *econtext; HeapScanDesc scan; TIDBitmap *tbm; - TBMIterator *tbmiterator; + TBMIterator *tbmiterator = NULL; + TBMSharedIterator *shared_tbmiterator = NULL; TBMIterateResult *tbmres; - -#ifdef USE_PREFETCH - TBMIterator *prefetch_iterator; -#endif OffsetNumber targoffset; TupleTableSlot *slot; + ParallelBitmapHeapState *pstate = node->pstate; + dsa_area *dsa = node->ss.ps.state->es_query_dsa; /* * extract necessary information from index scan node @@ -81,11 +91,11 @@ BitmapHeapNext(BitmapHeapScanState *node) slot = node->ss.ss_ScanTupleSlot; scan = node->ss.ss_currentScanDesc; tbm = node->tbm; - tbmiterator = node->tbmiterator; + if (pstate == NULL) + tbmiterator = node->tbmiterator; + else + shared_tbmiterator = node->shared_tbmiterator; tbmres = node->tbmres; -#ifdef USE_PREFETCH - prefetch_iterator = node->prefetch_iterator; -#endif /* * If we haven't yet performed the underlying index scan, do it, and begin @@ -99,25 +109,82 @@ BitmapHeapNext(BitmapHeapScanState *node) * node->prefetch_maximum. This is to avoid doing a lot of prefetching in * a scan that stops after a few tuples because of a LIMIT. */ - if (tbm == NULL) + if (!node->initialized) { - tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node)); + if (!pstate) + { + tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node)); - if (!tbm || !IsA(tbm, TIDBitmap)) - elog(ERROR, "unrecognized result from subplan"); + if (!tbm || !IsA(tbm, TIDBitmap)) + elog(ERROR, "unrecognized result from subplan"); - node->tbm = tbm; - node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm); - node->tbmres = tbmres = NULL; + node->tbm = tbm; + node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm); + node->tbmres = tbmres = NULL; #ifdef USE_PREFETCH - if (node->prefetch_maximum > 0) - { - node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm); - node->prefetch_pages = 0; - node->prefetch_target = -1; + if (node->prefetch_maximum > 0) + { + node->prefetch_iterator = tbm_begin_iterate(tbm); + node->prefetch_pages = 0; + node->prefetch_target = -1; + } +#endif /* USE_PREFETCH */ } + else + { + /* + * The leader will immediately come out of the function, but + * others will be blocked until leader populates the TBM and wakes + * them up. + */ + if (BitmapShouldInitializeSharedState(pstate)) + { + tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node)); + if (!tbm || !IsA(tbm, TIDBitmap)) + elog(ERROR, "unrecognized result from subplan"); + + node->tbm = tbm; + + /* + * Prepare to iterate over the TBM. This will return the + * dsa_pointer of the iterator state which will be used by + * multiple processes to iterate jointly. + */ + pstate->tbmiterator = tbm_prepare_shared_iterate(tbm); +#ifdef USE_PREFETCH + if (node->prefetch_maximum > 0) + { + pstate->prefetch_iterator = + tbm_prepare_shared_iterate(tbm); + + /* + * We don't need the mutex here as we haven't yet woke up + * others. + */ + pstate->prefetch_pages = 0; + pstate->prefetch_target = -1; + } +#endif + + /* We have initialized the shared state so wake up others. */ + BitmapDoneInitializingSharedState(pstate); + } + + /* Allocate a private iterator and attach the shared state to it */ + node->shared_tbmiterator = shared_tbmiterator = + tbm_attach_shared_iterate(dsa, pstate->tbmiterator); + node->tbmres = tbmres = NULL; + +#ifdef USE_PREFETCH + if (node->prefetch_maximum > 0) + { + node->shared_prefetch_iterator = + tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator); + } #endif /* USE_PREFETCH */ + } + node->initialized = true; } for (;;) @@ -130,28 +197,17 @@ BitmapHeapNext(BitmapHeapScanState *node) */ if (tbmres == NULL) { - node->tbmres = tbmres = tbm_iterate(tbmiterator); + if (!pstate) + node->tbmres = tbmres = tbm_iterate(tbmiterator); + else + node->tbmres = tbmres = tbm_shared_iterate(shared_tbmiterator); if (tbmres == NULL) { /* no more entries in the bitmap */ break; } -#ifdef USE_PREFETCH - if (node->prefetch_pages > 0) - { - /* The main iterator has closed the distance by one page */ - node->prefetch_pages--; - } - else if (prefetch_iterator) - { - /* Do not let the prefetch iterator get behind the main one */ - TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); - - if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno) - elog(ERROR, "prefetch and main iterators are out of sync"); - } -#endif /* USE_PREFETCH */ + BitmapAdjustPrefetchIterator(node, tbmres); /* * Ignore any claimed entries past what we think is the end of the @@ -180,23 +236,8 @@ BitmapHeapNext(BitmapHeapScanState *node) */ scan->rs_cindex = 0; -#ifdef USE_PREFETCH - - /* - * Increase prefetch target if it's not yet at the max. Note that - * we will increase it to zero after fetching the very first - * page/tuple, then to one after the second tuple is fetched, then - * it doubles as later pages are fetched. - */ - if (node->prefetch_target >= node->prefetch_maximum) - /* don't increase any further */ ; - else if (node->prefetch_target >= node->prefetch_maximum / 2) - node->prefetch_target = node->prefetch_maximum; - else if (node->prefetch_target > 0) - node->prefetch_target *= 2; - else - node->prefetch_target++; -#endif /* USE_PREFETCH */ + /* Adjust the prefetch target */ + BitmapAdjustPrefetchTarget(node); } else { @@ -211,8 +252,19 @@ BitmapHeapNext(BitmapHeapScanState *node) * Try to prefetch at least a few pages even before we get to the * second page if we don't stop reading after the first tuple. */ - if (node->prefetch_target < node->prefetch_maximum) - node->prefetch_target++; + if (!pstate) + { + if (node->prefetch_target < node->prefetch_maximum) + node->prefetch_target++; + } + else if (pstate->prefetch_target < node->prefetch_maximum) + { + /* take spinlock while updating shared state */ + SpinLockAcquire(&pstate->mutex); + if (pstate->prefetch_target < node->prefetch_maximum) + pstate->prefetch_target++; + SpinLockRelease(&pstate->mutex); + } #endif /* USE_PREFETCH */ } @@ -225,8 +277,6 @@ BitmapHeapNext(BitmapHeapScanState *node) continue; } -#ifdef USE_PREFETCH - /* * We issue prefetch requests *after* fetching the current page to try * to avoid having prefetching interfere with the main I/O. Also, this @@ -234,24 +284,7 @@ BitmapHeapNext(BitmapHeapScanState *node) * to do on the current page, else we may uselessly prefetch the same * page we are just about to request for real. */ - if (prefetch_iterator) - { - while (node->prefetch_pages < node->prefetch_target) - { - TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); - - if (tbmpre == NULL) - { - /* No more pages to prefetch */ - tbm_end_iterate(prefetch_iterator); - node->prefetch_iterator = prefetch_iterator = NULL; - break; - } - node->prefetch_pages++; - PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno); - } - } -#endif /* USE_PREFETCH */ + BitmapPrefetch(node, scan); /* * Okay to fetch the tuple @@ -286,7 +319,7 @@ BitmapHeapNext(BitmapHeapScanState *node) econtext->ecxt_scantuple = slot; ResetExprContext(econtext); - if (!ExecQual(node->bitmapqualorig, econtext, false)) + if (!ExecQual(node->bitmapqualorig, econtext)) { /* Fails recheck, so drop it and loop back for another */ InstrCountFiltered2(node, 1); @@ -410,6 +443,201 @@ bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres) } /* + * BitmapDoneInitializingSharedState - Shared state is initialized + * + * By this time the leader has already populated the TBM and initialized the + * shared state so wake up other processes. + */ +static inline void +BitmapDoneInitializingSharedState(ParallelBitmapHeapState *pstate) +{ + SpinLockAcquire(&pstate->mutex); + pstate->state = BM_FINISHED; + SpinLockRelease(&pstate->mutex); + ConditionVariableBroadcast(&pstate->cv); +} + +/* + * BitmapAdjustPrefetchIterator - Adjust the prefetch iterator + */ +static inline void +BitmapAdjustPrefetchIterator(BitmapHeapScanState *node, + TBMIterateResult *tbmres) +{ +#ifdef USE_PREFETCH + ParallelBitmapHeapState *pstate = node->pstate; + + if (pstate == NULL) + { + TBMIterator *prefetch_iterator = node->prefetch_iterator; + + if (node->prefetch_pages > 0) + { + /* The main iterator has closed the distance by one page */ + node->prefetch_pages--; + } + else if (prefetch_iterator) + { + /* Do not let the prefetch iterator get behind the main one */ + TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); + + if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno) + elog(ERROR, "prefetch and main iterators are out of sync"); + } + return; + } + + if (node->prefetch_maximum > 0) + { + TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator; + + SpinLockAcquire(&pstate->mutex); + if (pstate->prefetch_pages > 0) + { + pstate->prefetch_pages--; + SpinLockRelease(&pstate->mutex); + } + else + { + /* Release the mutex before iterating */ + SpinLockRelease(&pstate->mutex); + + /* + * In case of shared mode, we can not ensure that the current + * blockno of the main iterator and that of the prefetch iterator + * are same. It's possible that whatever blockno we are + * prefetching will be processed by another process. Therefore, + * we don't validate the blockno here as we do in non-parallel + * case. + */ + if (prefetch_iterator) + tbm_shared_iterate(prefetch_iterator); + } + } +#endif /* USE_PREFETCH */ +} + +/* + * BitmapAdjustPrefetchTarget - Adjust the prefetch target + * + * Increase prefetch target if it's not yet at the max. Note that + * we will increase it to zero after fetching the very first + * page/tuple, then to one after the second tuple is fetched, then + * it doubles as later pages are fetched. + */ +static inline void +BitmapAdjustPrefetchTarget(BitmapHeapScanState *node) +{ +#ifdef USE_PREFETCH + ParallelBitmapHeapState *pstate = node->pstate; + + if (pstate == NULL) + { + if (node->prefetch_target >= node->prefetch_maximum) + /* don't increase any further */ ; + else if (node->prefetch_target >= node->prefetch_maximum / 2) + node->prefetch_target = node->prefetch_maximum; + else if (node->prefetch_target > 0) + node->prefetch_target *= 2; + else + node->prefetch_target++; + return; + } + + /* Do an unlocked check first to save spinlock acquisitions. */ + if (pstate->prefetch_target < node->prefetch_maximum) + { + SpinLockAcquire(&pstate->mutex); + if (pstate->prefetch_target >= node->prefetch_maximum) + /* don't increase any further */ ; + else if (pstate->prefetch_target >= node->prefetch_maximum / 2) + pstate->prefetch_target = node->prefetch_maximum; + else if (pstate->prefetch_target > 0) + pstate->prefetch_target *= 2; + else + pstate->prefetch_target++; + SpinLockRelease(&pstate->mutex); + } +#endif /* USE_PREFETCH */ +} + +/* + * BitmapPrefetch - Prefetch, if prefetch_pages are behind prefetch_target + */ +static inline void +BitmapPrefetch(BitmapHeapScanState *node, HeapScanDesc scan) +{ +#ifdef USE_PREFETCH + ParallelBitmapHeapState *pstate = node->pstate; + + if (pstate == NULL) + { + TBMIterator *prefetch_iterator = node->prefetch_iterator; + + if (prefetch_iterator) + { + while (node->prefetch_pages < node->prefetch_target) + { + TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); + + if (tbmpre == NULL) + { + /* No more pages to prefetch */ + tbm_end_iterate(prefetch_iterator); + node->prefetch_iterator = NULL; + break; + } + node->prefetch_pages++; + PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno); + } + } + + return; + } + + if (pstate->prefetch_pages < pstate->prefetch_target) + { + TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator; + + if (prefetch_iterator) + { + while (1) + { + TBMIterateResult *tbmpre; + bool do_prefetch = false; + + /* + * Recheck under the mutex. If some other process has already + * done enough prefetching then we need not to do anything. + */ + SpinLockAcquire(&pstate->mutex); + if (pstate->prefetch_pages < pstate->prefetch_target) + { + pstate->prefetch_pages++; + do_prefetch = true; + } + SpinLockRelease(&pstate->mutex); + + if (!do_prefetch) + return; + + tbmpre = tbm_shared_iterate(prefetch_iterator); + if (tbmpre == NULL) + { + /* No more pages to prefetch */ + tbm_end_shared_iterate(prefetch_iterator); + node->shared_prefetch_iterator = NULL; + break; + } + + PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno); + } + } + } +#endif /* USE_PREFETCH */ +} + +/* * BitmapHeapRecheck -- access method routine to recheck a tuple in EvalPlanQual */ static bool @@ -427,7 +655,7 @@ BitmapHeapRecheck(BitmapHeapScanState *node, TupleTableSlot *slot) ResetExprContext(econtext); - return ExecQual(node->bitmapqualorig, econtext, false); + return ExecQual(node->bitmapqualorig, econtext); } /* ---------------------------------------------------------------- @@ -458,12 +686,36 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node) tbm_end_iterate(node->tbmiterator); if (node->prefetch_iterator) tbm_end_iterate(node->prefetch_iterator); + if (node->shared_tbmiterator) + tbm_end_shared_iterate(node->shared_tbmiterator); + if (node->shared_prefetch_iterator) + tbm_end_shared_iterate(node->shared_prefetch_iterator); if (node->tbm) tbm_free(node->tbm); node->tbm = NULL; node->tbmiterator = NULL; node->tbmres = NULL; node->prefetch_iterator = NULL; + node->initialized = false; + node->shared_tbmiterator = NULL; + node->shared_prefetch_iterator = NULL; + + /* Reset parallel bitmap state, if present */ + if (node->pstate) + { + dsa_area *dsa = node->ss.ps.state->es_query_dsa; + + node->pstate->state = BM_INITIAL; + + if (DsaPointerIsValid(node->pstate->tbmiterator)) + tbm_free_shared_area(dsa, node->pstate->tbmiterator); + + if (DsaPointerIsValid(node->pstate->prefetch_iterator)) + tbm_free_shared_area(dsa, node->pstate->prefetch_iterator); + + node->pstate->tbmiterator = InvalidDsaPointer; + node->pstate->prefetch_iterator = InvalidDsaPointer; + } ExecScanReScan(&node->ss); @@ -516,6 +768,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node) tbm_end_iterate(node->prefetch_iterator); if (node->tbm) tbm_free(node->tbm); + if (node->shared_tbmiterator) + tbm_end_shared_iterate(node->shared_tbmiterator); + if (node->shared_prefetch_iterator) + tbm_end_shared_iterate(node->shared_prefetch_iterator); /* * close heap scan @@ -567,6 +823,10 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags) scanstate->prefetch_target = 0; /* may be updated below */ scanstate->prefetch_maximum = target_prefetch_pages; + scanstate->pscan_len = 0; + scanstate->initialized = false; + scanstate->shared_tbmiterator = NULL; + scanstate->pstate = NULL; /* * Miscellaneous initialization @@ -575,20 +835,13 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &scanstate->ss.ps); - scanstate->ss.ps.ps_TupFromTlist = false; - /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); - scanstate->bitmapqualorig = (List *) - ExecInitExpr((Expr *) node->bitmapqualorig, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); + scanstate->bitmapqualorig = + ExecInitQual(node->bitmapqualorig, (PlanState *) scanstate); /* * tuple table initialization @@ -653,3 +906,108 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags) */ return scanstate; } + +/*---------------- + * BitmapShouldInitializeSharedState + * + * The first process to come here and see the state to the BM_INITIAL + * will become the leader for the parallel bitmap scan and will be + * responsible for populating the TIDBitmap. The other processes will + * be blocked by the condition variable until the leader wakes them up. + * --------------- + */ +static bool +BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pstate) +{ + SharedBitmapState state; + + while (1) + { + SpinLockAcquire(&pstate->mutex); + state = pstate->state; + if (pstate->state == BM_INITIAL) + pstate->state = BM_INPROGRESS; + SpinLockRelease(&pstate->mutex); + + /* Exit if bitmap is done, or if we're the leader. */ + if (state != BM_INPROGRESS) + break; + + /* Wait for the leader to wake us up. */ + ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN); + } + + ConditionVariableCancelSleep(); + + return (state == BM_INITIAL); +} + +/* ---------------------------------------------------------------- + * ExecBitmapHeapEstimate + * + * estimates the space required to serialize bitmap scan node. + * ---------------------------------------------------------------- + */ +void +ExecBitmapHeapEstimate(BitmapHeapScanState *node, + ParallelContext *pcxt) +{ + EState *estate = node->ss.ps.state; + + node->pscan_len = add_size(offsetof(ParallelBitmapHeapState, + phs_snapshot_data), + EstimateSnapshotSpace(estate->es_snapshot)); + + shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len); + shm_toc_estimate_keys(&pcxt->estimator, 1); +} + +/* ---------------------------------------------------------------- + * ExecBitmapHeapInitializeDSM + * + * Set up a parallel bitmap heap scan descriptor. + * ---------------------------------------------------------------- + */ +void +ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node, + ParallelContext *pcxt) +{ + ParallelBitmapHeapState *pstate; + EState *estate = node->ss.ps.state; + + pstate = shm_toc_allocate(pcxt->toc, node->pscan_len); + + pstate->tbmiterator = 0; + pstate->prefetch_iterator = 0; + + /* Initialize the mutex */ + SpinLockInit(&pstate->mutex); + pstate->prefetch_pages = 0; + pstate->prefetch_target = 0; + pstate->state = BM_INITIAL; + + ConditionVariableInit(&pstate->cv); + SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data); + + shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate); + node->pstate = pstate; +} + +/* ---------------------------------------------------------------- + * ExecBitmapHeapInitializeWorker + * + * Copy relevant information from TOC into planstate. + * ---------------------------------------------------------------- + */ +void +ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc) +{ + ParallelBitmapHeapState *pstate; + Snapshot snapshot; + + pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id); + node->pstate = pstate; + + snapshot = RestoreSnapshot(pstate->phs_snapshot_data); + heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot); +} diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c index a364098e59..ce2f3210a4 100644 --- a/src/backend/executor/nodeBitmapIndexscan.c +++ b/src/backend/executor/nodeBitmapIndexscan.c @@ -3,7 +3,7 @@ * nodeBitmapIndexscan.c * Routines to support bitmapped index scans of relations * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -78,7 +78,9 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node) else { /* XXX should we use less than work_mem for this? */ - tbm = tbm_create(work_mem * 1024L); + tbm = tbm_create(work_mem * 1024L, + ((BitmapIndexScan *) node->ss.ps.plan)->isshared ? + node->ss.ps.state->es_query_dsa : NULL); } /* diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c index 7e928eb618..c0f261407b 100644 --- a/src/backend/executor/nodeBitmapOr.c +++ b/src/backend/executor/nodeBitmapOr.c @@ -3,7 +3,7 @@ * nodeBitmapOr.c * routines to handle BitmapOr nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -129,7 +129,9 @@ MultiExecBitmapOr(BitmapOrState *node) if (result == NULL) /* first subplan */ { /* XXX should we use less than work_mem for this? */ - result = tbm_create(work_mem * 1024L); + result = tbm_create(work_mem * 1024L, + ((BitmapOr *) node->ps.plan)->isshared ? + node->ps.state->es_query_dsa : NULL); } ((BitmapIndexScanState *) subnode)->biss_result = result; diff --git a/src/backend/executor/nodeCtescan.c b/src/backend/executor/nodeCtescan.c index 3c2f684a06..bed7949c5a 100644 --- a/src/backend/executor/nodeCtescan.c +++ b/src/backend/executor/nodeCtescan.c @@ -3,7 +3,7 @@ * nodeCtescan.c * routines to handle CteScan nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -210,7 +210,7 @@ ExecInitCteScan(CteScan *node, EState *estate, int eflags) prmdata = &(estate->es_param_exec_vals[node->cteParam]); Assert(prmdata->execPlan == NULL); Assert(!prmdata->isnull); - scanstate->leader = (CteScanState *) DatumGetPointer(prmdata->value); + scanstate->leader = castNode(CteScanState, DatumGetPointer(prmdata->value)); if (scanstate->leader == NULL) { /* I am the leader */ @@ -223,10 +223,13 @@ ExecInitCteScan(CteScan *node, EState *estate, int eflags) else { /* Not the leader */ - Assert(IsA(scanstate->leader, CteScanState)); + /* Create my own read pointer, and ensure it is at start */ scanstate->readptr = tuplestore_alloc_read_pointer(scanstate->leader->cte_table, scanstate->eflags); + tuplestore_select_read_pointer(scanstate->leader->cte_table, + scanstate->readptr); + tuplestore_rescan(scanstate->leader->cte_table); } /* @@ -239,12 +242,8 @@ ExecInitCteScan(CteScan *node, EState *estate, int eflags) /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); /* * tuple table initialization @@ -265,8 +264,6 @@ ExecInitCteScan(CteScan *node, EState *estate, int eflags) ExecAssignResultTypeFromTL(&scanstate->ss.ps); ExecAssignScanProjectionInfo(&scanstate->ss); - scanstate->ss.ps.ps_TupFromTlist = false; - return scanstate; } diff --git a/src/backend/executor/nodeCustom.c b/src/backend/executor/nodeCustom.c index 322abca282..5d309828ef 100644 --- a/src/backend/executor/nodeCustom.c +++ b/src/backend/executor/nodeCustom.c @@ -3,7 +3,7 @@ * nodeCustom.c * Routines to handle execution of custom scan node * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * ------------------------------------------------------------------------ @@ -35,8 +35,8 @@ ExecInitCustomScan(CustomScan *cscan, EState *estate, int eflags) * methods field correctly at this time. Other standard fields should be * set to zero. */ - css = (CustomScanState *) cscan->methods->CreateCustomScanState(cscan); - Assert(IsA(css, CustomScanState)); + css = castNode(CustomScanState, + cscan->methods->CreateCustomScanState(cscan)); /* ensure flags is filled correctly */ css->flags = cscan->flags; @@ -48,15 +48,9 @@ ExecInitCustomScan(CustomScan *cscan, EState *estate, int eflags) /* create expression context for node */ ExecAssignExprContext(estate, &css->ss.ps); - css->ss.ps.ps_TupFromTlist = false; - /* initialize child expressions */ - css->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) cscan->scan.plan.targetlist, - (PlanState *) css); - css->ss.ps.qual = (List *) - ExecInitExpr((Expr *) cscan->scan.plan.qual, - (PlanState *) css); + css->ss.ps.qual = + ExecInitQual(cscan->scan.plan.qual, (PlanState *) css); /* tuple table initialization */ ExecInitScanTupleSlot(estate, &css->ss); @@ -204,3 +198,12 @@ ExecCustomScanInitializeWorker(CustomScanState *node, shm_toc *toc) methods->InitializeWorkerCustomScan(node, toc, coordinate); } } + +void +ExecShutdownCustomScan(CustomScanState *node) +{ + const CustomExecMethods *methods = node->methods; + + if (methods->ShutdownCustomScan) + methods->ShutdownCustomScan(node); +} diff --git a/src/backend/executor/nodeForeignscan.c b/src/backend/executor/nodeForeignscan.c index 583e39021b..707db92178 100644 --- a/src/backend/executor/nodeForeignscan.c +++ b/src/backend/executor/nodeForeignscan.c @@ -3,7 +3,7 @@ * nodeForeignscan.c * Routines to support scans of foreign tables * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -109,7 +109,7 @@ ForeignRecheck(ForeignScanState *node, TupleTableSlot *slot) !fdwroutine->RecheckForeignScan(node, slot)) return false; - return ExecQual(node->fdw_recheck_quals, econtext, false); + return ExecQual(node->fdw_recheck_quals, econtext); } /* ---------------------------------------------------------------- @@ -160,20 +160,13 @@ ExecInitForeignScan(ForeignScan *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &scanstate->ss.ps); - scanstate->ss.ps.ps_TupFromTlist = false; - /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); - scanstate->fdw_recheck_quals = (List *) - ExecInitExpr((Expr *) node->fdw_recheck_quals, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); + scanstate->fdw_recheck_quals = + ExecInitQual(node->fdw_recheck_quals, (PlanState *) scanstate); /* * tuple table initialization @@ -363,3 +356,19 @@ ExecForeignScanInitializeWorker(ForeignScanState *node, shm_toc *toc) fdwroutine->InitializeWorkerForeignScan(node, toc, coordinate); } } + +/* ---------------------------------------------------------------- + * ExecShutdownForeignScan + * + * Gives FDW chance to stop asynchronous resource consumption + * and release any resources still held. + * ---------------------------------------------------------------- + */ +void +ExecShutdownForeignScan(ForeignScanState *node) +{ + FdwRoutine *fdwroutine = node->fdwroutine; + + if (fdwroutine->ShutdownForeignScan) + fdwroutine->ShutdownForeignScan(node); +} diff --git a/src/backend/executor/nodeFunctionscan.c b/src/backend/executor/nodeFunctionscan.c index a03f6e73fd..426527d2a2 100644 --- a/src/backend/executor/nodeFunctionscan.c +++ b/src/backend/executor/nodeFunctionscan.c @@ -3,7 +3,7 @@ * nodeFunctionscan.c * Support routines for scanning RangeFunctions (functions in rangetable). * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -35,7 +35,7 @@ */ typedef struct FunctionScanPerFuncState { - ExprState *funcexpr; /* state of the expression being evaluated */ + SetExprState *setexpr; /* state of the expression being evaluated */ TupleDesc tupdesc; /* desc of the function result type */ int colcount; /* expected number of result columns */ Tuplestorestate *tstore; /* holds the function result set */ @@ -92,7 +92,7 @@ FunctionNext(FunctionScanState *node) if (tstore == NULL) { node->funcstates[0].tstore = tstore = - ExecMakeTableFunctionResult(node->funcstates[0].funcexpr, + ExecMakeTableFunctionResult(node->funcstates[0].setexpr, node->ss.ps.ps_ExprContext, node->argcontext, node->funcstates[0].tupdesc, @@ -151,7 +151,7 @@ FunctionNext(FunctionScanState *node) if (fs->tstore == NULL) { fs->tstore = - ExecMakeTableFunctionResult(fs->funcexpr, + ExecMakeTableFunctionResult(fs->setexpr, node->ss.ps.ps_ExprContext, node->argcontext, fs->tupdesc, @@ -331,8 +331,6 @@ ExecInitFunctionScan(FunctionScan *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &scanstate->ss.ps); - scanstate->ss.ps.ps_TupFromTlist = false; - /* * tuple table initialization */ @@ -342,12 +340,8 @@ ExecInitFunctionScan(FunctionScan *node, EState *estate, int eflags) /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); scanstate->funcstates = palloc(nfuncs * sizeof(FunctionScanPerFuncState)); @@ -363,7 +357,10 @@ ExecInitFunctionScan(FunctionScan *node, EState *estate, int eflags) Oid funcrettype; TupleDesc tupdesc; - fs->funcexpr = ExecInitExpr((Expr *) funcexpr, (PlanState *) scanstate); + fs->setexpr = + ExecInitTableFunctionResult((Expr *) funcexpr, + scanstate->ss.ps.ps_ExprContext, + &scanstate->ss.ps); /* * Don't allocate the tuplestores; the actual calls to the functions @@ -508,9 +505,7 @@ ExecInitFunctionScan(FunctionScan *node, EState *estate, int eflags) */ scanstate->argcontext = AllocSetContextCreate(CurrentMemoryContext, "Table function arguments", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); return scanstate; } diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c index 438d1b24fc..c1db2e263b 100644 --- a/src/backend/executor/nodeGather.c +++ b/src/backend/executor/nodeGather.c @@ -3,7 +3,7 @@ * nodeGather.c * Support routines for scanning a plan via multiple workers. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * A Gather executor launches parallel workers to run multiple copies of a @@ -38,6 +38,7 @@ #include "executor/nodeSubplan.h" #include "executor/tqueue.h" #include "miscadmin.h" +#include "pgstat.h" #include "utils/memutils.h" #include "utils/rel.h" @@ -80,12 +81,8 @@ ExecInitGather(Gather *node, EState *estate, int eflags) /* * initialize child expressions */ - gatherstate->ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) gatherstate); - gatherstate->ps.qual = (List *) - ExecInitExpr((Expr *) node->plan.qual, - (PlanState *) gatherstate); + gatherstate->ps.qual = + ExecInitQual(node->plan.qual, (PlanState *) gatherstate); /* * tuple table initialization @@ -99,8 +96,6 @@ ExecInitGather(Gather *node, EState *estate, int eflags) outerNode = outerPlan(node); outerPlanState(gatherstate) = ExecInitNode(outerNode, estate, eflags); - gatherstate->ps.ps_TupFromTlist = false; - /* * Initialize result tuple type and projection info. */ @@ -131,15 +126,13 @@ ExecGather(GatherState *node) TupleTableSlot *fslot = node->funnel_slot; int i; TupleTableSlot *slot; - TupleTableSlot *resultSlot; - ExprDoneCond isDone; ExprContext *econtext; /* * Initialize the parallel context and workers on first execution. We do * this on first execution rather than during node initialization, as it - * needs to allocate large dynamic segment, so it is better to do if it is - * really needed. + * needs to allocate a large dynamic segment, so it is better to do it + * only if it is really needed. */ if (!node->initialized) { @@ -172,6 +165,7 @@ ExecGather(GatherState *node) if (pcxt->nworkers_launched > 0) { node->nreaders = 0; + node->nextreader = 0; node->reader = palloc(pcxt->nworkers_launched * sizeof(TupleQueueReader *)); @@ -198,57 +192,28 @@ ExecGather(GatherState *node) } /* - * Check to see if we're still projecting out tuples from a previous scan - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->ps.ps_TupFromTlist) - { - resultSlot = ExecProject(node->ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return resultSlot; - /* Done with that source tuple... */ - node->ps.ps_TupFromTlist = false; - } - - /* * Reset per-tuple memory context to free any expression evaluation - * storage allocated in the previous tuple cycle. Note we can't do this - * until we're done projecting. This will also clear any previous tuple - * returned by a TupleQueueReader; to make sure we don't leave a dangling - * pointer around, clear the working slot first. + * storage allocated in the previous tuple cycle. This will also clear + * any previous tuple returned by a TupleQueueReader; to make sure we + * don't leave a dangling pointer around, clear the working slot first. */ - ExecClearTuple(node->funnel_slot); + ExecClearTuple(fslot); econtext = node->ps.ps_ExprContext; ResetExprContext(econtext); - /* Get and return the next tuple, projecting if necessary. */ - for (;;) - { - /* - * Get next tuple, either from one of our workers, or by running the - * plan ourselves. - */ - slot = gather_getnext(node); - if (TupIsNull(slot)) - return NULL; - - /* - * form the result tuple using ExecProject(), and return it --- unless - * the projection produces an empty set, in which case we must loop - * back around for another tuple - */ - econtext->ecxt_outertuple = slot; - resultSlot = ExecProject(node->ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->ps.ps_TupFromTlist = (isDone == ExprMultipleResult); - return resultSlot; - } - } + /* + * Get next tuple, either from one of our workers, or by running the plan + * ourselves. + */ + slot = gather_getnext(node); + if (TupIsNull(slot)) + return NULL; - return slot; + /* + * Form the result tuple using ExecProject(), and return it. + */ + econtext->ecxt_outertuple = slot; + return ExecProject(node->ps.ps_ProjInfo); } /* ---------------------------------------------------------------- @@ -260,10 +225,10 @@ ExecGather(GatherState *node) void ExecEndGather(GatherState *node) { + ExecEndNode(outerPlanState(node)); /* let children clean up first */ ExecShutdownGather(node); ExecFreeExprContext(&node->ps); ExecClearTuple(node->ps.ps_ResultTupleSlot); - ExecEndNode(outerPlanState(node)); } /* @@ -334,6 +299,7 @@ gather_readnext(GatherState *gatherstate) CHECK_FOR_INTERRUPTS(); /* Attempt to read a tuple, but don't block if none is available. */ + Assert(gatherstate->nextreader < gatherstate->nreaders); reader = gatherstate->reader[gatherstate->nextreader]; tup = TupleQueueReaderNext(reader, true, &readerdone); @@ -387,7 +353,7 @@ gather_readnext(GatherState *gatherstate) return NULL; /* Nothing to do except wait for developments. */ - WaitLatch(MyLatch, WL_LATCH_SET, 0); + WaitLatch(MyLatch, WL_LATCH_SET, 0, WAIT_EVENT_EXECUTE_GATHER); ResetLatch(MyLatch); nvisited = 0; } diff --git a/src/backend/executor/nodeGatherMerge.c b/src/backend/executor/nodeGatherMerge.c new file mode 100644 index 0000000000..e066574836 --- /dev/null +++ b/src/backend/executor/nodeGatherMerge.c @@ -0,0 +1,681 @@ +/*------------------------------------------------------------------------- + * + * nodeGatherMerge.c + * Scan a plan in multiple workers, and do order-preserving merge. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * IDENTIFICATION + * src/backend/executor/nodeGatherMerge.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "access/relscan.h" +#include "access/xact.h" +#include "executor/execdebug.h" +#include "executor/execParallel.h" +#include "executor/nodeGatherMerge.h" +#include "executor/nodeSubplan.h" +#include "executor/tqueue.h" +#include "lib/binaryheap.h" +#include "miscadmin.h" +#include "utils/memutils.h" +#include "utils/rel.h" + +/* + * Tuple array for each worker + */ +typedef struct GMReaderTupleBuffer +{ + HeapTuple *tuple; + int readCounter; + int nTuples; + bool done; +} GMReaderTupleBuffer; + +/* + * When we read tuples from workers, it's a good idea to read several at once + * for efficiency when possible: this minimizes context-switching overhead. + * But reading too many at a time wastes memory without improving performance. + */ +#define MAX_TUPLE_STORE 10 + +static int32 heap_compare_slots(Datum a, Datum b, void *arg); +static TupleTableSlot *gather_merge_getnext(GatherMergeState *gm_state); +static HeapTuple gm_readnext_tuple(GatherMergeState *gm_state, int nreader, + bool nowait, bool *done); +static void gather_merge_init(GatherMergeState *gm_state); +static void ExecShutdownGatherMergeWorkers(GatherMergeState *node); +static bool gather_merge_readnext(GatherMergeState *gm_state, int reader, + bool nowait); +static void form_tuple_array(GatherMergeState *gm_state, int reader); + +/* ---------------------------------------------------------------- + * ExecInitGather + * ---------------------------------------------------------------- + */ +GatherMergeState * +ExecInitGatherMerge(GatherMerge *node, EState *estate, int eflags) +{ + GatherMergeState *gm_state; + Plan *outerNode; + bool hasoid; + TupleDesc tupDesc; + + /* Gather merge node doesn't have innerPlan node. */ + Assert(innerPlan(node) == NULL); + + /* + * create state structure + */ + gm_state = makeNode(GatherMergeState); + gm_state->ps.plan = (Plan *) node; + gm_state->ps.state = estate; + + /* + * Miscellaneous initialization + * + * create expression context for node + */ + ExecAssignExprContext(estate, &gm_state->ps); + + /* + * initialize child expressions + */ + gm_state->ps.qual = + ExecInitQual(node->plan.qual, &gm_state->ps); + + /* + * tuple table initialization + */ + ExecInitResultTupleSlot(estate, &gm_state->ps); + + /* + * now initialize outer plan + */ + outerNode = outerPlan(node); + outerPlanState(gm_state) = ExecInitNode(outerNode, estate, eflags); + + /* + * Initialize result tuple type and projection info. + */ + ExecAssignResultTypeFromTL(&gm_state->ps); + ExecAssignProjectionInfo(&gm_state->ps, NULL); + + gm_state->gm_initialized = false; + + /* + * initialize sort-key information + */ + if (node->numCols) + { + int i; + + gm_state->gm_nkeys = node->numCols; + gm_state->gm_sortkeys = + palloc0(sizeof(SortSupportData) * node->numCols); + + for (i = 0; i < node->numCols; i++) + { + SortSupport sortKey = gm_state->gm_sortkeys + i; + + sortKey->ssup_cxt = CurrentMemoryContext; + sortKey->ssup_collation = node->collations[i]; + sortKey->ssup_nulls_first = node->nullsFirst[i]; + sortKey->ssup_attno = node->sortColIdx[i]; + + /* + * We don't perform abbreviated key conversion here, for the same + * reasons that it isn't used in MergeAppend + */ + sortKey->abbreviate = false; + + PrepareSortSupportFromOrderingOp(node->sortOperators[i], sortKey); + } + } + + /* + * store the tuple descriptor into gather merge state, so we can use it + * later while initializing the gather merge slots. + */ + if (!ExecContextForcesOids(&gm_state->ps, &hasoid)) + hasoid = false; + tupDesc = ExecTypeFromTL(outerNode->targetlist, hasoid); + gm_state->tupDesc = tupDesc; + + return gm_state; +} + +/* ---------------------------------------------------------------- + * ExecGatherMerge(node) + * + * Scans the relation via multiple workers and returns + * the next qualifying tuple. + * ---------------------------------------------------------------- + */ +TupleTableSlot * +ExecGatherMerge(GatherMergeState *node) +{ + TupleTableSlot *slot; + ExprContext *econtext; + int i; + + /* + * As with Gather, we don't launch workers until this node is actually + * executed. + */ + if (!node->initialized) + { + EState *estate = node->ps.state; + GatherMerge *gm = (GatherMerge *) node->ps.plan; + + /* + * Sometimes we might have to run without parallelism; but if parallel + * mode is active then we can try to fire up some workers. + */ + if (gm->num_workers > 0 && IsInParallelMode()) + { + ParallelContext *pcxt; + + /* Initialize data structures for workers. */ + if (!node->pei) + node->pei = ExecInitParallelPlan(node->ps.lefttree, + estate, + gm->num_workers); + + /* Try to launch workers. */ + pcxt = node->pei->pcxt; + LaunchParallelWorkers(pcxt); + node->nworkers_launched = pcxt->nworkers_launched; + + /* Set up tuple queue readers to read the results. */ + if (pcxt->nworkers_launched > 0) + { + node->nreaders = 0; + node->reader = palloc(pcxt->nworkers_launched * + sizeof(TupleQueueReader *)); + + Assert(gm->numCols); + + for (i = 0; i < pcxt->nworkers_launched; ++i) + { + shm_mq_set_handle(node->pei->tqueue[i], + pcxt->worker[i].bgwhandle); + node->reader[node->nreaders++] = + CreateTupleQueueReader(node->pei->tqueue[i], + node->tupDesc); + } + } + else + { + /* No workers? Then never mind. */ + ExecShutdownGatherMergeWorkers(node); + } + } + + /* always allow leader to participate */ + node->need_to_scan_locally = true; + node->initialized = true; + } + + /* + * Reset per-tuple memory context to free any expression evaluation + * storage allocated in the previous tuple cycle. + */ + econtext = node->ps.ps_ExprContext; + ResetExprContext(econtext); + + /* + * Get next tuple, either from one of our workers, or by running the plan + * ourselves. + */ + slot = gather_merge_getnext(node); + if (TupIsNull(slot)) + return NULL; + + /* + * form the result tuple using ExecProject(), and return it --- unless the + * projection produces an empty set, in which case we must loop back + * around for another tuple + */ + econtext->ecxt_outertuple = slot; + return ExecProject(node->ps.ps_ProjInfo); +} + +/* ---------------------------------------------------------------- + * ExecEndGatherMerge + * + * frees any storage allocated through C routines. + * ---------------------------------------------------------------- + */ +void +ExecEndGatherMerge(GatherMergeState *node) +{ + ExecEndNode(outerPlanState(node)); /* let children clean up first */ + ExecShutdownGatherMerge(node); + ExecFreeExprContext(&node->ps); + ExecClearTuple(node->ps.ps_ResultTupleSlot); +} + +/* ---------------------------------------------------------------- + * ExecShutdownGatherMerge + * + * Destroy the setup for parallel workers including parallel context. + * Collect all the stats after workers are stopped, else some work + * done by workers won't be accounted. + * ---------------------------------------------------------------- + */ +void +ExecShutdownGatherMerge(GatherMergeState *node) +{ + ExecShutdownGatherMergeWorkers(node); + + /* Now destroy the parallel context. */ + if (node->pei != NULL) + { + ExecParallelCleanup(node->pei); + node->pei = NULL; + } +} + +/* ---------------------------------------------------------------- + * ExecShutdownGatherMergeWorkers + * + * Destroy the parallel workers. Collect all the stats after + * workers are stopped, else some work done by workers won't be + * accounted. + * ---------------------------------------------------------------- + */ +static void +ExecShutdownGatherMergeWorkers(GatherMergeState *node) +{ + /* Shut down tuple queue readers before shutting down workers. */ + if (node->reader != NULL) + { + int i; + + for (i = 0; i < node->nreaders; ++i) + if (node->reader[i]) + DestroyTupleQueueReader(node->reader[i]); + + pfree(node->reader); + node->reader = NULL; + } + + /* Now shut down the workers. */ + if (node->pei != NULL) + ExecParallelFinish(node->pei); +} + +/* ---------------------------------------------------------------- + * ExecReScanGatherMerge + * + * Re-initialize the workers and rescans a relation via them. + * ---------------------------------------------------------------- + */ +void +ExecReScanGatherMerge(GatherMergeState *node) +{ + /* + * Re-initialize the parallel workers to perform rescan of relation. We + * want to gracefully shutdown all the workers so that they should be able + * to propagate any error or other information to master backend before + * dying. Parallel context will be reused for rescan. + */ + ExecShutdownGatherMergeWorkers(node); + + node->initialized = false; + + if (node->pei) + ExecParallelReinitialize(node->pei); + + ExecReScan(node->ps.lefttree); +} + +/* + * Initialize the Gather merge tuple read. + * + * Pull at least a single tuple from each worker + leader and set up the heap. + */ +static void +gather_merge_init(GatherMergeState *gm_state) +{ + int nreaders = gm_state->nreaders; + bool initialize = true; + int i; + + /* + * Allocate gm_slots for the number of worker + one more slot for leader. + * Last slot is always for leader. Leader always calls ExecProcNode() to + * read the tuple which will return the TupleTableSlot. Later it will + * directly get assigned to gm_slot. So just initialize leader gm_slot + * with NULL. For other slots below code will call + * ExecInitExtraTupleSlot() which will do the initialization of worker + * slots. + */ + gm_state->gm_slots = + palloc((gm_state->nreaders + 1) * sizeof(TupleTableSlot *)); + gm_state->gm_slots[gm_state->nreaders] = NULL; + + /* Initialize the tuple slot and tuple array for each worker */ + gm_state->gm_tuple_buffers = + (GMReaderTupleBuffer *) palloc0(sizeof(GMReaderTupleBuffer) * + (gm_state->nreaders + 1)); + for (i = 0; i < gm_state->nreaders; i++) + { + /* Allocate the tuple array with MAX_TUPLE_STORE size */ + gm_state->gm_tuple_buffers[i].tuple = + (HeapTuple *) palloc0(sizeof(HeapTuple) * MAX_TUPLE_STORE); + + /* Initialize slot for worker */ + gm_state->gm_slots[i] = ExecInitExtraTupleSlot(gm_state->ps.state); + ExecSetSlotDescriptor(gm_state->gm_slots[i], + gm_state->tupDesc); + } + + /* Allocate the resources for the merge */ + gm_state->gm_heap = binaryheap_allocate(gm_state->nreaders + 1, + heap_compare_slots, + gm_state); + + /* + * First, try to read a tuple from each worker (including leader) in + * nowait mode, so that we initialize read from each worker as well as + * leader. After this, if all active workers are unable to produce a + * tuple, then re-read and this time use wait mode. For workers that were + * able to produce a tuple in the earlier loop and are still active, just + * try to fill the tuple array if more tuples are avaiable. + */ +reread: + for (i = 0; i < nreaders + 1; i++) + { + if (!gm_state->gm_tuple_buffers[i].done && + (TupIsNull(gm_state->gm_slots[i]) || + gm_state->gm_slots[i]->tts_isempty)) + { + if (gather_merge_readnext(gm_state, i, initialize)) + { + binaryheap_add_unordered(gm_state->gm_heap, + Int32GetDatum(i)); + } + } + else + form_tuple_array(gm_state, i); + } + initialize = false; + + for (i = 0; i < nreaders; i++) + if (!gm_state->gm_tuple_buffers[i].done && + (TupIsNull(gm_state->gm_slots[i]) || + gm_state->gm_slots[i]->tts_isempty)) + goto reread; + + binaryheap_build(gm_state->gm_heap); + gm_state->gm_initialized = true; +} + +/* + * Clear out the tuple table slots for each gather merge input. + */ +static void +gather_merge_clear_slots(GatherMergeState *gm_state) +{ + int i; + + for (i = 0; i < gm_state->nreaders; i++) + { + pfree(gm_state->gm_tuple_buffers[i].tuple); + gm_state->gm_slots[i] = ExecClearTuple(gm_state->gm_slots[i]); + } + + /* Free tuple array as we don't need it any more */ + pfree(gm_state->gm_tuple_buffers); + /* Free the binaryheap, which was created for sort */ + binaryheap_free(gm_state->gm_heap); +} + +/* + * Read the next tuple for gather merge. + * + * Fetch the sorted tuple out of the heap. + */ +static TupleTableSlot * +gather_merge_getnext(GatherMergeState *gm_state) +{ + int i; + + if (!gm_state->gm_initialized) + { + /* + * First time through: pull the first tuple from each participant, and + * set up the heap. + */ + gather_merge_init(gm_state); + } + else + { + /* + * Otherwise, pull the next tuple from whichever participant we + * returned from last time, and reinsert that participant's index into + * the heap, because it might now compare differently against the + * other elements of the heap. + */ + i = DatumGetInt32(binaryheap_first(gm_state->gm_heap)); + + if (gather_merge_readnext(gm_state, i, false)) + binaryheap_replace_first(gm_state->gm_heap, Int32GetDatum(i)); + else + (void) binaryheap_remove_first(gm_state->gm_heap); + } + + if (binaryheap_empty(gm_state->gm_heap)) + { + /* All the queues are exhausted, and so is the heap */ + gather_merge_clear_slots(gm_state); + return NULL; + } + else + { + /* Return next tuple from whichever participant has the leading one */ + i = DatumGetInt32(binaryheap_first(gm_state->gm_heap)); + return gm_state->gm_slots[i]; + } +} + +/* + * Read the tuple for given reader in nowait mode, and form the tuple array. + */ +static void +form_tuple_array(GatherMergeState *gm_state, int reader) +{ + GMReaderTupleBuffer *tuple_buffer = &gm_state->gm_tuple_buffers[reader]; + int i; + + /* Last slot is for leader and we don't build tuple array for leader */ + if (reader == gm_state->nreaders) + return; + + /* + * We here because we already read all the tuples from the tuple array, so + * initialize the counter to zero. + */ + if (tuple_buffer->nTuples == tuple_buffer->readCounter) + tuple_buffer->nTuples = tuple_buffer->readCounter = 0; + + /* Tuple array is already full? */ + if (tuple_buffer->nTuples == MAX_TUPLE_STORE) + return; + + for (i = tuple_buffer->nTuples; i < MAX_TUPLE_STORE; i++) + { + tuple_buffer->tuple[i] = heap_copytuple(gm_readnext_tuple(gm_state, + reader, + false, + &tuple_buffer->done)); + if (!HeapTupleIsValid(tuple_buffer->tuple[i])) + break; + tuple_buffer->nTuples++; + } +} + +/* + * Store the next tuple for a given reader into the appropriate slot. + * + * Returns false if the reader is exhausted, and true otherwise. + */ +static bool +gather_merge_readnext(GatherMergeState *gm_state, int reader, bool nowait) +{ + GMReaderTupleBuffer *tuple_buffer; + HeapTuple tup = NULL; + + /* + * If we're being asked to generate a tuple from the leader, then we just + * call ExecProcNode as normal to produce one. + */ + if (gm_state->nreaders == reader) + { + if (gm_state->need_to_scan_locally) + { + PlanState *outerPlan = outerPlanState(gm_state); + TupleTableSlot *outerTupleSlot; + + outerTupleSlot = ExecProcNode(outerPlan); + + if (!TupIsNull(outerTupleSlot)) + { + gm_state->gm_slots[reader] = outerTupleSlot; + return true; + } + gm_state->gm_tuple_buffers[reader].done = true; + gm_state->need_to_scan_locally = false; + } + return false; + } + + /* Otherwise, check the state of the relevant tuple buffer. */ + tuple_buffer = &gm_state->gm_tuple_buffers[reader]; + + if (tuple_buffer->nTuples > tuple_buffer->readCounter) + { + /* Return any tuple previously read that is still buffered. */ + tuple_buffer = &gm_state->gm_tuple_buffers[reader]; + tup = tuple_buffer->tuple[tuple_buffer->readCounter++]; + } + else if (tuple_buffer->done) + { + /* Reader is known to be exhausted. */ + DestroyTupleQueueReader(gm_state->reader[reader]); + gm_state->reader[reader] = NULL; + return false; + } + else + { + /* Read and buffer next tuple. */ + tup = heap_copytuple(gm_readnext_tuple(gm_state, + reader, + nowait, + &tuple_buffer->done)); + + /* + * Attempt to read more tuples in nowait mode and store them in the + * tuple array. + */ + if (HeapTupleIsValid(tup)) + form_tuple_array(gm_state, reader); + else + return false; + } + + Assert(HeapTupleIsValid(tup)); + + /* Build the TupleTableSlot for the given tuple */ + ExecStoreTuple(tup, /* tuple to store */ + gm_state->gm_slots[reader], /* slot in which to store the + * tuple */ + InvalidBuffer, /* buffer associated with this tuple */ + true); /* pfree this pointer if not from heap */ + + return true; +} + +/* + * Attempt to read a tuple from given reader. + */ +static HeapTuple +gm_readnext_tuple(GatherMergeState *gm_state, int nreader, bool nowait, + bool *done) +{ + TupleQueueReader *reader; + HeapTuple tup = NULL; + MemoryContext oldContext; + MemoryContext tupleContext; + + tupleContext = gm_state->ps.ps_ExprContext->ecxt_per_tuple_memory; + + if (done != NULL) + *done = false; + + /* Check for async events, particularly messages from workers. */ + CHECK_FOR_INTERRUPTS(); + + /* Attempt to read a tuple. */ + reader = gm_state->reader[nreader]; + + /* Run TupleQueueReaders in per-tuple context */ + oldContext = MemoryContextSwitchTo(tupleContext); + tup = TupleQueueReaderNext(reader, nowait, done); + MemoryContextSwitchTo(oldContext); + + return tup; +} + +/* + * We have one slot for each item in the heap array. We use SlotNumber + * to store slot indexes. This doesn't actually provide any formal + * type-safety, but it makes the code more self-documenting. + */ +typedef int32 SlotNumber; + +/* + * Compare the tuples in the two given slots. + */ +static int32 +heap_compare_slots(Datum a, Datum b, void *arg) +{ + GatherMergeState *node = (GatherMergeState *) arg; + SlotNumber slot1 = DatumGetInt32(a); + SlotNumber slot2 = DatumGetInt32(b); + + TupleTableSlot *s1 = node->gm_slots[slot1]; + TupleTableSlot *s2 = node->gm_slots[slot2]; + int nkey; + + Assert(!TupIsNull(s1)); + Assert(!TupIsNull(s2)); + + for (nkey = 0; nkey < node->gm_nkeys; nkey++) + { + SortSupport sortKey = node->gm_sortkeys + nkey; + AttrNumber attno = sortKey->ssup_attno; + Datum datum1, + datum2; + bool isNull1, + isNull2; + int compare; + + datum1 = slot_getattr(s1, attno, &isNull1); + datum2 = slot_getattr(s2, attno, &isNull2); + + compare = ApplySortComparator(datum1, isNull1, + datum2, isNull2, + sortKey); + if (compare != 0) + return -compare; + } + return 0; +} diff --git a/src/backend/executor/nodeGroup.c b/src/backend/executor/nodeGroup.c index dcf5175d9b..af9ba4905e 100644 --- a/src/backend/executor/nodeGroup.c +++ b/src/backend/executor/nodeGroup.c @@ -3,7 +3,7 @@ * nodeGroup.c * Routines to handle group nodes (used for queries with GROUP BY clause). * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -50,23 +50,6 @@ ExecGroup(GroupState *node) grpColIdx = ((Group *) node->ss.ps.plan)->grpColIdx; /* - * Check to see if we're still projecting out tuples from a previous group - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->ss.ps.ps_TupFromTlist) - { - TupleTableSlot *result; - ExprDoneCond isDone; - - result = ExecProject(node->ss.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return result; - /* Done with that source tuple... */ - node->ss.ps.ps_TupFromTlist = false; - } - - /* * The ScanTupleSlot holds the (copied) first tuple of each group. */ firsttupleslot = node->ss.ss_ScanTupleSlot; @@ -102,21 +85,12 @@ ExecGroup(GroupState *node) * Check the qual (HAVING clause); if the group does not match, ignore * it and fall into scan loop. */ - if (ExecQual(node->ss.ps.qual, econtext, false)) + if (ExecQual(node->ss.ps.qual, econtext)) { /* * Form and return a projection tuple using the first input tuple. */ - TupleTableSlot *result; - ExprDoneCond isDone; - - result = ExecProject(node->ss.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->ss.ps.ps_TupFromTlist = (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->ss.ps.ps_ProjInfo); } else InstrCountFiltered1(node, 1); @@ -165,21 +139,12 @@ ExecGroup(GroupState *node) * Check the qual (HAVING clause); if the group does not match, ignore * it and loop back to scan the rest of the group. */ - if (ExecQual(node->ss.ps.qual, econtext, false)) + if (ExecQual(node->ss.ps.qual, econtext)) { /* * Form and return a projection tuple using the first input tuple. */ - TupleTableSlot *result; - ExprDoneCond isDone; - - result = ExecProject(node->ss.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->ss.ps.ps_TupFromTlist = (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->ss.ps.ps_ProjInfo); } else InstrCountFiltered1(node, 1); @@ -223,12 +188,8 @@ ExecInitGroup(Group *node, EState *estate, int eflags) /* * initialize child expressions */ - grpstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) grpstate); - grpstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->plan.qual, - (PlanState *) grpstate); + grpstate->ss.ps.qual = + ExecInitQual(node->plan.qual, (PlanState *) grpstate); /* * initialize child nodes @@ -246,8 +207,6 @@ ExecInitGroup(Group *node, EState *estate, int eflags) ExecAssignResultTypeFromTL(&grpstate->ss.ps); ExecAssignProjectionInfo(&grpstate->ss.ps, NULL); - grpstate->ss.ps.ps_TupFromTlist = false; - /* * Precompute fmgr lookup data for inner loop */ @@ -283,7 +242,6 @@ ExecReScanGroup(GroupState *node) PlanState *outerPlan = outerPlanState(node); node->grp_done = FALSE; - node->ss.ps.ps_TupFromTlist = false; /* must clear first tuple */ ExecClearTuple(node->ss.ss_ScanTupleSlot); diff --git a/src/backend/executor/nodeHash.c b/src/backend/executor/nodeHash.c index 9ed09a7b0c..d9789d0719 100644 --- a/src/backend/executor/nodeHash.c +++ b/src/backend/executor/nodeHash.c @@ -3,7 +3,7 @@ * nodeHash.c * Routines to hash relations for hashjoin * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -190,12 +190,8 @@ ExecInitHash(Hash *node, EState *estate, int eflags) /* * initialize child expressions */ - hashstate->ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) hashstate); - hashstate->ps.qual = (List *) - ExecInitExpr((Expr *) node->plan.qual, - (PlanState *) hashstate); + hashstate->ps.qual = + ExecInitQual(node->plan.qual, (PlanState *) hashstate); /* * initialize child nodes @@ -344,15 +340,11 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls) */ hashtable->hashCxt = AllocSetContextCreate(CurrentMemoryContext, "HashTableContext", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); hashtable->batchCxt = AllocSetContextCreate(hashtable->hashCxt, "HashBatchContext", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* Allocate data that will live for the life of the hashjoin */ @@ -724,6 +716,9 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable) /* next tuple in this chunk */ idx += MAXALIGN(hashTupleSize); + + /* allow this loop to be cancellable */ + CHECK_FOR_INTERRUPTS(); } /* we're done with this chunk - free it and proceed to the next one */ @@ -963,7 +958,7 @@ ExecHashGetHashValue(HashJoinTable hashtable, /* * Get the join attribute value of the tuple */ - keyval = ExecEvalExpr(keyexpr, econtext, &isNull, NULL); + keyval = ExecEvalExpr(keyexpr, econtext, &isNull); /* * If the attribute is NULL, and the join operator is strict, then @@ -1064,7 +1059,7 @@ bool ExecScanHashBucket(HashJoinState *hjstate, ExprContext *econtext) { - List *hjclauses = hjstate->hashclauses; + ExprState *hjclauses = hjstate->hashclauses; HashJoinTable hashtable = hjstate->hj_HashTable; HashJoinTuple hashTuple = hjstate->hj_CurTuple; uint32 hashvalue = hjstate->hj_CurHashValue; @@ -1098,7 +1093,7 @@ ExecScanHashBucket(HashJoinState *hjstate, /* reset temp memory each time to avoid leaks from qual expr */ ResetExprContext(econtext); - if (ExecQual(hjclauses, econtext, false)) + if (ExecQual(hjclauses, econtext)) { hjstate->hj_CurTuple = hashTuple; return true; @@ -1288,10 +1283,7 @@ static void ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse) { HeapTupleData *statsTuple; - Datum *values; - int nvalues; - float4 *numbers; - int nnumbers; + AttStatsSlot sslot; /* Do nothing if planner didn't identify the outer relation's join key */ if (!OidIsValid(node->skewTable)) @@ -1310,19 +1302,17 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse) if (!HeapTupleIsValid(statsTuple)) return; - if (get_attstatsslot(statsTuple, node->skewColType, node->skewColTypmod, + if (get_attstatsslot(&sslot, statsTuple, STATISTIC_KIND_MCV, InvalidOid, - NULL, - &values, &nvalues, - &numbers, &nnumbers)) + ATTSTATSSLOT_VALUES | ATTSTATSSLOT_NUMBERS)) { double frac; int nbuckets; FmgrInfo *hashfunctions; int i; - if (mcvsToUse > nvalues) - mcvsToUse = nvalues; + if (mcvsToUse > sslot.nvalues) + mcvsToUse = sslot.nvalues; /* * Calculate the expected fraction of outer relation that will @@ -1331,11 +1321,10 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse) */ frac = 0; for (i = 0; i < mcvsToUse; i++) - frac += numbers[i]; + frac += sslot.numbers[i]; if (frac < SKEW_MIN_OUTER_FRACTION) { - free_attstatsslot(node->skewColType, - values, nvalues, numbers, nnumbers); + free_attstatsslot(&sslot); ReleaseSysCache(statsTuple); return; } @@ -1397,7 +1386,7 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse) int bucket; hashvalue = DatumGetUInt32(FunctionCall1(&hashfunctions[0], - values[i])); + sslot.values[i])); /* * While we have not hit a hole in the hashtable and have not hit @@ -1431,8 +1420,7 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse) hashtable->spacePeak = hashtable->spaceUsed; } - free_attstatsslot(node->skewColType, - values, nvalues, numbers, nnumbers); + free_attstatsslot(&sslot); } ReleaseSysCache(statsTuple); @@ -1603,6 +1591,9 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable) } hashTuple = nextHashTuple; + + /* allow this loop to be cancellable */ + CHECK_FOR_INTERRUPTS(); } /* diff --git a/src/backend/executor/nodeHashjoin.c b/src/backend/executor/nodeHashjoin.c index 369e666f88..f9ab0d6035 100644 --- a/src/backend/executor/nodeHashjoin.c +++ b/src/backend/executor/nodeHashjoin.c @@ -3,7 +3,7 @@ * nodeHashjoin.c * Routines to handle hash join nodes * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -63,10 +63,9 @@ ExecHashJoin(HashJoinState *node) { PlanState *outerNode; HashState *hashNode; - List *joinqual; - List *otherqual; + ExprState *joinqual; + ExprState *otherqual; ExprContext *econtext; - ExprDoneCond isDone; HashJoinTable hashtable; TupleTableSlot *outerTupleSlot; uint32 hashvalue; @@ -83,25 +82,8 @@ ExecHashJoin(HashJoinState *node) econtext = node->js.ps.ps_ExprContext; /* - * Check to see if we're still projecting out tuples from a previous join - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->js.ps.ps_TupFromTlist) - { - TupleTableSlot *result; - - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return result; - /* Done with that source tuple... */ - node->js.ps.ps_TupFromTlist = false; - } - - /* * Reset per-tuple memory context to free any expression evaluation - * storage allocated in the previous tuple cycle. Note this can't happen - * until we're done projecting out tuples from a join tuple. + * storage allocated in the previous tuple cycle. */ ResetExprContext(econtext); @@ -293,7 +275,7 @@ ExecHashJoin(HashJoinState *node) * Only the joinquals determine tuple match status, but all * quals must pass to actually return the tuple. */ - if (joinqual == NIL || ExecQual(joinqual, econtext, false)) + if (joinqual == NULL || ExecQual(joinqual, econtext)) { node->hj_MatchedOuter = true; HeapTupleHeaderSetMatch(HJTUPLE_MINTUPLE(node->hj_CurTuple)); @@ -306,26 +288,15 @@ ExecHashJoin(HashJoinState *node) } /* - * In a semijoin, we'll consider returning the first - * match, but after that we're done with this outer tuple. + * If we only need to join to the first matching inner + * tuple, then consider returning this one, but after that + * continue with next outer tuple. */ - if (node->js.jointype == JOIN_SEMI) + if (node->js.single_match) node->hj_JoinState = HJ_NEED_NEW_OUTER; - if (otherqual == NIL || - ExecQual(otherqual, econtext, false)) - { - TupleTableSlot *result; - - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } - } + if (otherqual == NULL || ExecQual(otherqual, econtext)) + return ExecProject(node->js.ps.ps_ProjInfo); else InstrCountFiltered2(node, 1); } @@ -351,20 +322,8 @@ ExecHashJoin(HashJoinState *node) */ econtext->ecxt_innertuple = node->hj_NullInnerTupleSlot; - if (otherqual == NIL || - ExecQual(otherqual, econtext, false)) - { - TupleTableSlot *result; - - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } - } + if (otherqual == NULL || ExecQual(otherqual, econtext)) + return ExecProject(node->js.ps.ps_ProjInfo); else InstrCountFiltered2(node, 1); } @@ -390,20 +349,8 @@ ExecHashJoin(HashJoinState *node) */ econtext->ecxt_outertuple = node->hj_NullOuterTupleSlot; - if (otherqual == NIL || - ExecQual(otherqual, econtext, false)) - { - TupleTableSlot *result; - - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } - } + if (otherqual == NULL || ExecQual(otherqual, econtext)) + return ExecProject(node->js.ps.ps_ProjInfo); else InstrCountFiltered2(node, 1); break; @@ -462,19 +409,13 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) /* * initialize child expressions */ - hjstate->js.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->join.plan.targetlist, - (PlanState *) hjstate); - hjstate->js.ps.qual = (List *) - ExecInitExpr((Expr *) node->join.plan.qual, - (PlanState *) hjstate); + hjstate->js.ps.qual = + ExecInitQual(node->join.plan.qual, (PlanState *) hjstate); hjstate->js.jointype = node->join.jointype; - hjstate->js.joinqual = (List *) - ExecInitExpr((Expr *) node->join.joinqual, - (PlanState *) hjstate); - hjstate->hashclauses = (List *) - ExecInitExpr((Expr *) node->hashclauses, - (PlanState *) hjstate); + hjstate->js.joinqual = + ExecInitQual(node->join.joinqual, (PlanState *) hjstate); + hjstate->hashclauses = + ExecInitQual(node->hashclauses, (PlanState *) hjstate); /* * initialize child nodes @@ -495,6 +436,12 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) ExecInitResultTupleSlot(estate, &hjstate->js.ps); hjstate->hj_OuterTupleSlot = ExecInitExtraTupleSlot(estate); + /* + * detect whether we need only consider the first matching inner tuple + */ + hjstate->js.single_match = (node->join.inner_unique || + node->join.jointype == JOIN_SEMI); + /* set up null tuples for outer joins, if needed */ switch (node->join.jointype) { @@ -568,16 +515,14 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) lclauses = NIL; rclauses = NIL; hoperators = NIL; - foreach(l, hjstate->hashclauses) + foreach(l, node->hashclauses) { - FuncExprState *fstate = (FuncExprState *) lfirst(l); - OpExpr *hclause; - - Assert(IsA(fstate, FuncExprState)); - hclause = (OpExpr *) fstate->xprstate.expr; - Assert(IsA(hclause, OpExpr)); - lclauses = lappend(lclauses, linitial(fstate->args)); - rclauses = lappend(rclauses, lsecond(fstate->args)); + OpExpr *hclause = lfirst_node(OpExpr, l); + + lclauses = lappend(lclauses, ExecInitExpr(linitial(hclause->args), + (PlanState *) hjstate)); + rclauses = lappend(rclauses, ExecInitExpr(lsecond(hclause->args), + (PlanState *) hjstate)); hoperators = lappend_oid(hoperators, hclause->opno); } hjstate->hj_OuterHashKeys = lclauses; @@ -586,7 +531,6 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) /* child Hash node needs to evaluate inner hash keys, too */ ((HashState *) innerPlanState(hjstate))->hashkeys = rclauses; - hjstate->js.ps.ps_TupFromTlist = false; hjstate->hj_JoinState = HJ_BUILD_HASHTABLE; hjstate->hj_MatchedOuter = false; hjstate->hj_OuterNotEmpty = false; @@ -912,6 +856,13 @@ ExecHashJoinGetSavedTuple(HashJoinState *hjstate, MinimalTuple tuple; /* + * We check for interrupts here because this is typically taken as an + * alternative code path to an ExecProcNode() call, which would include + * such a check. + */ + CHECK_FOR_INTERRUPTS(); + + /* * Since both the hash value and the MinimalTuple length word are uint32, * we can read them both in one BufFileRead() call without any type * cheating. @@ -1000,7 +951,6 @@ ExecReScanHashJoin(HashJoinState *node) node->hj_CurSkewBucketNo = INVALID_SKEW_BUCKET_NO; node->hj_CurTuple = NULL; - node->js.ps.ps_TupFromTlist = false; node->hj_MatchedOuter = false; node->hj_FirstOuterTupleSlot = NULL; diff --git a/src/backend/executor/nodeIndexonlyscan.c b/src/backend/executor/nodeIndexonlyscan.c index 4f6f91c8db..5550f6c0a4 100644 --- a/src/backend/executor/nodeIndexonlyscan.c +++ b/src/backend/executor/nodeIndexonlyscan.c @@ -3,7 +3,7 @@ * nodeIndexonlyscan.c * Routines to support index-only scans * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -21,6 +21,11 @@ * ExecEndIndexOnlyScan releases all storage. * ExecIndexOnlyMarkPos marks scan position. * ExecIndexOnlyRestrPos restores scan position. + * ExecIndexOnlyScanEstimate estimates DSM space needed for + * parallel index-only scan + * ExecIndexOnlyScanInitializeDSM initialize DSM for parallel + * index-only scan + * ExecIndexOnlyScanInitializeWorker attach to DSM info in parallel worker */ #include "postgres.h" @@ -73,6 +78,38 @@ IndexOnlyNext(IndexOnlyScanState *node) econtext = node->ss.ps.ps_ExprContext; slot = node->ss.ss_ScanTupleSlot; + if (scandesc == NULL) + { + /* + * We reach here if the index only scan is not parallel, or if we're + * executing a index only scan that was intended to be parallel + * serially. + */ + scandesc = index_beginscan(node->ss.ss_currentRelation, + node->ioss_RelationDesc, + estate->es_snapshot, + node->ioss_NumScanKeys, + node->ioss_NumOrderByKeys); + + node->ioss_ScanDesc = scandesc; + + + /* Set it up for index-only scan */ + node->ioss_ScanDesc->xs_want_itup = true; + node->ioss_VMBuffer = InvalidBuffer; + + /* + * If no run-time keys to calculate or they are ready, go ahead and + * pass the scankeys to the index AM. + */ + if (node->ioss_NumRuntimeKeys == 0 || node->ioss_RuntimeKeysReady) + index_rescan(scandesc, + node->ioss_ScanKeys, + node->ioss_NumScanKeys, + node->ioss_OrderByKeys, + node->ioss_NumOrderByKeys); + } + /* * OK, now that we have what we need, fetch the next tuple. */ @@ -144,9 +181,26 @@ IndexOnlyNext(IndexOnlyScanState *node) } /* - * Fill the scan tuple slot with data from the index. + * Fill the scan tuple slot with data from the index. This might be + * provided in either HeapTuple or IndexTuple format. Conceivably an + * index AM might fill both fields, in which case we prefer the heap + * format, since it's probably a bit cheaper to fill a slot from. */ - StoreIndexTuple(slot, scandesc->xs_itup, scandesc->xs_itupdesc); + if (scandesc->xs_hitup) + { + /* + * We don't take the trouble to verify that the provided tuple has + * exactly the slot's format, but it seems worth doing a quick + * check on the number of fields. + */ + Assert(slot->tts_tupleDescriptor->natts == + scandesc->xs_hitupdesc->natts); + ExecStoreTuple(scandesc->xs_hitup, slot, InvalidBuffer, false); + } + else if (scandesc->xs_itup) + StoreIndexTuple(slot, scandesc->xs_itup, scandesc->xs_itupdesc); + else + elog(ERROR, "no data returned for index-only scan"); /* * If the index was lossy, we have to recheck the index quals. @@ -157,7 +211,7 @@ IndexOnlyNext(IndexOnlyScanState *node) { econtext->ecxt_scantuple = slot; ResetExprContext(econtext); - if (!ExecQual(node->indexqual, econtext, false)) + if (!ExecQual(node->indexqual, econtext)) { /* Fails recheck, so drop it and loop back for another */ InstrCountFiltered2(node, 1); @@ -277,6 +331,16 @@ ExecIndexOnlyScan(IndexOnlyScanState *node) void ExecReScanIndexOnlyScan(IndexOnlyScanState *node) { + bool reset_parallel_scan = true; + + /* + * If we are here to just update the scan keys, then don't reset parallel + * scan. For detailed reason behind this look in the comments for + * ExecReScanIndexScan. + */ + if (node->ioss_NumRuntimeKeys != 0 && !node->ioss_RuntimeKeysReady) + reset_parallel_scan = false; + /* * If we are doing runtime key calculations (ie, any of the index key * values weren't simple Consts), compute the new key values. But first, @@ -296,10 +360,16 @@ ExecReScanIndexOnlyScan(IndexOnlyScanState *node) node->ioss_RuntimeKeysReady = true; /* reset index scan */ - index_rescan(node->ioss_ScanDesc, - node->ioss_ScanKeys, node->ioss_NumScanKeys, - node->ioss_OrderByKeys, node->ioss_NumOrderByKeys); + if (node->ioss_ScanDesc) + { + index_rescan(node->ioss_ScanDesc, + node->ioss_ScanKeys, node->ioss_NumScanKeys, + node->ioss_OrderByKeys, node->ioss_NumOrderByKeys); + + if (reset_parallel_scan && node->ioss_ScanDesc->parallel_scan) + index_parallelrescan(node->ioss_ScanDesc); + } ExecScanReScan(&node->ss); } @@ -412,23 +482,16 @@ ExecInitIndexOnlyScan(IndexOnlyScan *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &indexstate->ss.ps); - indexstate->ss.ps.ps_TupFromTlist = false; - /* * initialize child expressions * * Note: we don't initialize all of the indexorderby expression, only the * sub-parts corresponding to runtime keys (see below). */ - indexstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) indexstate); - indexstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) indexstate); - indexstate->indexqual = (List *) - ExecInitExpr((Expr *) node->indexqual, - (PlanState *) indexstate); + indexstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) indexstate); + indexstate->indexqual = + ExecInitQual(node->indexqual, (PlanState *) indexstate); /* * tuple table initialization @@ -536,31 +599,98 @@ ExecInitIndexOnlyScan(IndexOnlyScan *node, EState *estate, int eflags) } /* - * Initialize scan descriptor. + * all done. */ - indexstate->ioss_ScanDesc = index_beginscan(currentRelation, - indexstate->ioss_RelationDesc, - estate->es_snapshot, - indexstate->ioss_NumScanKeys, - indexstate->ioss_NumOrderByKeys); + return indexstate; +} + +/* ---------------------------------------------------------------- + * Parallel Index-only Scan Support + * ---------------------------------------------------------------- + */ - /* Set it up for index-only scan */ - indexstate->ioss_ScanDesc->xs_want_itup = true; - indexstate->ioss_VMBuffer = InvalidBuffer; +/* ---------------------------------------------------------------- + * ExecIndexOnlyScanEstimate + * + * estimates the space required to serialize index-only scan node. + * ---------------------------------------------------------------- + */ +void +ExecIndexOnlyScanEstimate(IndexOnlyScanState *node, + ParallelContext *pcxt) +{ + EState *estate = node->ss.ps.state; + + node->ioss_PscanLen = index_parallelscan_estimate(node->ioss_RelationDesc, + estate->es_snapshot); + shm_toc_estimate_chunk(&pcxt->estimator, node->ioss_PscanLen); + shm_toc_estimate_keys(&pcxt->estimator, 1); +} + +/* ---------------------------------------------------------------- + * ExecIndexOnlyScanInitializeDSM + * + * Set up a parallel index-only scan descriptor. + * ---------------------------------------------------------------- + */ +void +ExecIndexOnlyScanInitializeDSM(IndexOnlyScanState *node, + ParallelContext *pcxt) +{ + EState *estate = node->ss.ps.state; + ParallelIndexScanDesc piscan; + + piscan = shm_toc_allocate(pcxt->toc, node->ioss_PscanLen); + index_parallelscan_initialize(node->ss.ss_currentRelation, + node->ioss_RelationDesc, + estate->es_snapshot, + piscan); + shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, piscan); + node->ioss_ScanDesc = + index_beginscan_parallel(node->ss.ss_currentRelation, + node->ioss_RelationDesc, + node->ioss_NumScanKeys, + node->ioss_NumOrderByKeys, + piscan); + node->ioss_ScanDesc->xs_want_itup = true; + node->ioss_VMBuffer = InvalidBuffer; /* - * If no run-time keys to calculate, go ahead and pass the scankeys to the - * index AM. + * If no run-time keys to calculate or they are ready, go ahead and pass + * the scankeys to the index AM. */ - if (indexstate->ioss_NumRuntimeKeys == 0) - index_rescan(indexstate->ioss_ScanDesc, - indexstate->ioss_ScanKeys, - indexstate->ioss_NumScanKeys, - indexstate->ioss_OrderByKeys, - indexstate->ioss_NumOrderByKeys); + if (node->ioss_NumRuntimeKeys == 0 || node->ioss_RuntimeKeysReady) + index_rescan(node->ioss_ScanDesc, + node->ioss_ScanKeys, node->ioss_NumScanKeys, + node->ioss_OrderByKeys, node->ioss_NumOrderByKeys); +} + +/* ---------------------------------------------------------------- + * ExecIndexOnlyScanInitializeWorker + * + * Copy relevant information from TOC into planstate. + * ---------------------------------------------------------------- + */ +void +ExecIndexOnlyScanInitializeWorker(IndexOnlyScanState *node, shm_toc *toc) +{ + ParallelIndexScanDesc piscan; + + piscan = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id); + node->ioss_ScanDesc = + index_beginscan_parallel(node->ss.ss_currentRelation, + node->ioss_RelationDesc, + node->ioss_NumScanKeys, + node->ioss_NumOrderByKeys, + piscan); + node->ioss_ScanDesc->xs_want_itup = true; /* - * all done. + * If no run-time keys to calculate or they are ready, go ahead and pass + * the scankeys to the index AM. */ - return indexstate; + if (node->ioss_NumRuntimeKeys == 0 || node->ioss_RuntimeKeysReady) + index_rescan(node->ioss_ScanDesc, + node->ioss_ScanKeys, node->ioss_NumScanKeys, + node->ioss_OrderByKeys, node->ioss_NumOrderByKeys); } diff --git a/src/backend/executor/nodeIndexscan.c b/src/backend/executor/nodeIndexscan.c index 3143bd94ec..5afd02e09d 100644 --- a/src/backend/executor/nodeIndexscan.c +++ b/src/backend/executor/nodeIndexscan.c @@ -3,7 +3,7 @@ * nodeIndexscan.c * Routines to support indexed scans of relations * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -22,6 +22,9 @@ * ExecEndIndexScan releases all storage. * ExecIndexMarkPos marks scan position. * ExecIndexRestrPos restores scan position. + * ExecIndexScanEstimate estimates DSM space needed for parallel index scan + * ExecIndexScanInitializeDSM initialize DSM for parallel indexscan + * ExecIndexScanInitializeWorker attach to DSM info in parallel worker */ #include "postgres.h" @@ -99,6 +102,30 @@ IndexNext(IndexScanState *node) econtext = node->ss.ps.ps_ExprContext; slot = node->ss.ss_ScanTupleSlot; + if (scandesc == NULL) + { + /* + * We reach here if the index scan is not parallel, or if we're + * executing a index scan that was intended to be parallel serially. + */ + scandesc = index_beginscan(node->ss.ss_currentRelation, + node->iss_RelationDesc, + estate->es_snapshot, + node->iss_NumScanKeys, + node->iss_NumOrderByKeys); + + node->iss_ScanDesc = scandesc; + + /* + * If no run-time keys to calculate or they are ready, go ahead and + * pass the scankeys to the index AM. + */ + if (node->iss_NumRuntimeKeys == 0 || node->iss_RuntimeKeysReady) + index_rescan(scandesc, + node->iss_ScanKeys, node->iss_NumScanKeys, + node->iss_OrderByKeys, node->iss_NumOrderByKeys); + } + /* * ok, now that we have what we need, fetch the next tuple. */ @@ -122,7 +149,7 @@ IndexNext(IndexScanState *node) { econtext->ecxt_scantuple = slot; ResetExprContext(econtext); - if (!ExecQual(node->indexqualorig, econtext, false)) + if (!ExecQual(node->indexqualorig, econtext)) { /* Fails recheck, so drop it and loop back for another */ InstrCountFiltered2(node, 1); @@ -151,6 +178,7 @@ IndexNext(IndexScanState *node) static TupleTableSlot * IndexNextWithReorder(IndexScanState *node) { + EState *estate; ExprContext *econtext; IndexScanDesc scandesc; HeapTuple tuple; @@ -161,6 +189,8 @@ IndexNextWithReorder(IndexScanState *node) bool *lastfetched_nulls; int cmp; + estate = node->ss.ps.state; + /* * Only forward scan is supported with reordering. Note: we can get away * with just Asserting here because the system will not try to run the @@ -171,12 +201,36 @@ IndexNextWithReorder(IndexScanState *node) * explicitly. */ Assert(!ScanDirectionIsBackward(((IndexScan *) node->ss.ps.plan)->indexorderdir)); - Assert(ScanDirectionIsForward(node->ss.ps.state->es_direction)); + Assert(ScanDirectionIsForward(estate->es_direction)); scandesc = node->iss_ScanDesc; econtext = node->ss.ps.ps_ExprContext; slot = node->ss.ss_ScanTupleSlot; + if (scandesc == NULL) + { + /* + * We reach here if the index scan is not parallel, or if we're + * executing a index scan that was intended to be parallel serially. + */ + scandesc = index_beginscan(node->ss.ss_currentRelation, + node->iss_RelationDesc, + estate->es_snapshot, + node->iss_NumScanKeys, + node->iss_NumOrderByKeys); + + node->iss_ScanDesc = scandesc; + + /* + * If no run-time keys to calculate or they are ready, go ahead and + * pass the scankeys to the index AM. + */ + if (node->iss_NumRuntimeKeys == 0 || node->iss_RuntimeKeysReady) + index_rescan(scandesc, + node->iss_ScanKeys, node->iss_NumScanKeys, + node->iss_OrderByKeys, node->iss_NumOrderByKeys); + } + for (;;) { /* @@ -241,7 +295,7 @@ next_indextuple: { econtext->ecxt_scantuple = slot; ResetExprContext(econtext); - if (!ExecQual(node->indexqualorig, econtext, false)) + if (!ExecQual(node->indexqualorig, econtext)) { /* Fails recheck, so drop it and loop back for another */ InstrCountFiltered2(node, 1); @@ -336,8 +390,7 @@ EvalOrderByExpressions(IndexScanState *node, ExprContext *econtext) node->iss_OrderByValues[i] = ExecEvalExpr(orderby, econtext, - &node->iss_OrderByNulls[i], - NULL); + &node->iss_OrderByNulls[i]); i++; } @@ -362,7 +415,7 @@ IndexRecheck(IndexScanState *node, TupleTableSlot *slot) ResetExprContext(econtext); - return ExecQual(node->indexqualorig, econtext, false); + return ExecQual(node->indexqualorig, econtext); } @@ -515,6 +568,18 @@ ExecIndexScan(IndexScanState *node) void ExecReScanIndexScan(IndexScanState *node) { + bool reset_parallel_scan = true; + + /* + * If we are here to just update the scan keys, then don't reset parallel + * scan. We don't want each of the participating process in the parallel + * scan to update the shared parallel scan state at the start of the scan. + * It is quite possible that one of the participants has already begun + * scanning the index when another has yet to start it. + */ + if (node->iss_NumRuntimeKeys != 0 && !node->iss_RuntimeKeysReady) + reset_parallel_scan = false; + /* * If we are doing runtime key calculations (ie, any of the index key * values weren't simple Consts), compute the new key values. But first, @@ -540,10 +605,21 @@ ExecReScanIndexScan(IndexScanState *node) reorderqueue_pop(node); } - /* reset index scan */ - index_rescan(node->iss_ScanDesc, - node->iss_ScanKeys, node->iss_NumScanKeys, - node->iss_OrderByKeys, node->iss_NumOrderByKeys); + /* + * Reset (parallel) index scan. For parallel-aware nodes, the scan + * descriptor is initialized during actual execution of node and we can + * reach here before that (ex. during execution of nest loop join). So, + * avoid updating the scan descriptor at that time. + */ + if (node->iss_ScanDesc) + { + index_rescan(node->iss_ScanDesc, + node->iss_ScanKeys, node->iss_NumScanKeys, + node->iss_OrderByKeys, node->iss_NumOrderByKeys); + + if (reset_parallel_scan && node->iss_ScanDesc->parallel_scan) + index_parallelrescan(node->iss_ScanDesc); + } node->iss_ReachedEnd = false; ExecScanReScan(&node->ss); @@ -590,8 +666,7 @@ ExecIndexEvalRuntimeKeys(ExprContext *econtext, */ scanvalue = ExecEvalExpr(key_expr, econtext, - &isNull, - NULL); + &isNull); if (isNull) { scan_key->sk_argument = scanvalue; @@ -648,8 +723,7 @@ ExecIndexEvalArrayKeys(ExprContext *econtext, */ arraydatum = ExecEvalExpr(array_expr, econtext, - &isNull, - NULL); + &isNull); if (isNull) { result = false; @@ -837,8 +911,6 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &indexstate->ss.ps); - indexstate->ss.ps.ps_TupFromTlist = false; - /* * initialize child expressions * @@ -849,18 +921,12 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags) * would be nice to improve that. (Problem is that any SubPlans present * in the expression must be found now...) */ - indexstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) indexstate); - indexstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) indexstate); - indexstate->indexqualorig = (List *) - ExecInitExpr((Expr *) node->indexqualorig, - (PlanState *) indexstate); - indexstate->indexorderbyorig = (List *) - ExecInitExpr((Expr *) node->indexorderbyorig, - (PlanState *) indexstate); + indexstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) indexstate); + indexstate->indexqualorig = + ExecInitQual(node->indexqualorig, (PlanState *) indexstate); + indexstate->indexorderbyorig = + ExecInitExprList(node->indexorderbyorig, (PlanState *) indexstate); /* * tuple table initialization @@ -1018,24 +1084,6 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags) } /* - * Initialize scan descriptor. - */ - indexstate->iss_ScanDesc = index_beginscan(currentRelation, - indexstate->iss_RelationDesc, - estate->es_snapshot, - indexstate->iss_NumScanKeys, - indexstate->iss_NumOrderByKeys); - - /* - * If no run-time keys to calculate, go ahead and pass the scankeys to the - * index AM. - */ - if (indexstate->iss_NumRuntimeKeys == 0) - index_rescan(indexstate->iss_ScanDesc, - indexstate->iss_ScanKeys, indexstate->iss_NumScanKeys, - indexstate->iss_OrderByKeys, indexstate->iss_NumOrderByKeys); - - /* * all done. */ return indexstate; @@ -1595,3 +1643,91 @@ ExecIndexBuildScanKeys(PlanState *planstate, Relation index, else if (n_array_keys != 0) elog(ERROR, "ScalarArrayOpExpr index qual found where not allowed"); } + +/* ---------------------------------------------------------------- + * Parallel Scan Support + * ---------------------------------------------------------------- + */ + +/* ---------------------------------------------------------------- + * ExecIndexScanEstimate + * + * estimates the space required to serialize indexscan node. + * ---------------------------------------------------------------- + */ +void +ExecIndexScanEstimate(IndexScanState *node, + ParallelContext *pcxt) +{ + EState *estate = node->ss.ps.state; + + node->iss_PscanLen = index_parallelscan_estimate(node->iss_RelationDesc, + estate->es_snapshot); + shm_toc_estimate_chunk(&pcxt->estimator, node->iss_PscanLen); + shm_toc_estimate_keys(&pcxt->estimator, 1); +} + +/* ---------------------------------------------------------------- + * ExecIndexScanInitializeDSM + * + * Set up a parallel index scan descriptor. + * ---------------------------------------------------------------- + */ +void +ExecIndexScanInitializeDSM(IndexScanState *node, + ParallelContext *pcxt) +{ + EState *estate = node->ss.ps.state; + ParallelIndexScanDesc piscan; + + piscan = shm_toc_allocate(pcxt->toc, node->iss_PscanLen); + index_parallelscan_initialize(node->ss.ss_currentRelation, + node->iss_RelationDesc, + estate->es_snapshot, + piscan); + shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, piscan); + node->iss_ScanDesc = + index_beginscan_parallel(node->ss.ss_currentRelation, + node->iss_RelationDesc, + node->iss_NumScanKeys, + node->iss_NumOrderByKeys, + piscan); + + /* + * If no run-time keys to calculate or they are ready, go ahead and pass + * the scankeys to the index AM. + */ + if (node->iss_NumRuntimeKeys == 0 || node->iss_RuntimeKeysReady) + index_rescan(node->iss_ScanDesc, + node->iss_ScanKeys, node->iss_NumScanKeys, + node->iss_OrderByKeys, node->iss_NumOrderByKeys); +} + +/* ---------------------------------------------------------------- + * ExecIndexScanInitializeWorker + * + * Copy relevant information from TOC into planstate. + * ---------------------------------------------------------------- + */ +void +ExecIndexScanInitializeWorker(IndexScanState *node, shm_toc *toc) +{ + ParallelIndexScanDesc piscan; + + piscan = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id); + node->iss_ScanDesc = + index_beginscan_parallel(node->ss.ss_currentRelation, + node->iss_RelationDesc, + node->iss_NumScanKeys, + node->iss_NumOrderByKeys, + piscan); + + /* + * If no run-time keys to calculate or they are ready, go ahead and pass + * the scankeys to the index AM. + */ + if (node->iss_NumRuntimeKeys == 0 || node->iss_RuntimeKeysReady) + index_rescan(node->iss_ScanDesc, + node->iss_ScanKeys, node->iss_NumScanKeys, + node->iss_OrderByKeys, node->iss_NumOrderByKeys); +} diff --git a/src/backend/executor/nodeLimit.c b/src/backend/executor/nodeLimit.c index faf32e1aee..aaec132218 100644 --- a/src/backend/executor/nodeLimit.c +++ b/src/backend/executor/nodeLimit.c @@ -3,7 +3,7 @@ * nodeLimit.c * Routines to handle limiting of query results where appropriate * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -239,8 +239,7 @@ recompute_limits(LimitState *node) { val = ExecEvalExprSwitchContext(node->limitOffset, econtext, - &isNull, - NULL); + &isNull); /* Interpret NULL offset as no offset */ if (isNull) node->offset = 0; @@ -263,8 +262,7 @@ recompute_limits(LimitState *node) { val = ExecEvalExprSwitchContext(node->limitCount, econtext, - &isNull, - NULL); + &isNull); /* Interpret NULL count as no count (LIMIT ALL) */ if (isNull) { @@ -346,18 +344,11 @@ pass_down_bound(LimitState *node, PlanState *child_node) else if (IsA(child_node, ResultState)) { /* - * An extra consideration here is that if the Result is projecting a - * targetlist that contains any SRFs, we can't assume that every input - * tuple generates an output tuple, so a Sort underneath might need to - * return more than N tuples to satisfy LIMIT N. So we cannot use - * bounded sort. - * * If Result supported qual checking, we'd have to punt on seeing a - * qual, too. Note that having a resconstantqual is not a - * showstopper: if that fails we're not getting any rows at all. + * qual. Note that having a resconstantqual is not a showstopper: if + * that fails we're not getting any rows at all. */ - if (outerPlanState(child_node) && - !expression_returns_set((Node *) child_node->plan->targetlist)) + if (outerPlanState(child_node)) pass_down_bound(node, outerPlanState(child_node)); } } diff --git a/src/backend/executor/nodeLockRows.c b/src/backend/executor/nodeLockRows.c index 4ebcaffe69..5630eae53d 100644 --- a/src/backend/executor/nodeLockRows.c +++ b/src/backend/executor/nodeLockRows.c @@ -3,7 +3,7 @@ * nodeLockRows.c * Routines to handle FOR UPDATE/FOR SHARE row locking * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -401,12 +401,10 @@ ExecInitLockRows(LockRows *node, EState *estate, int eflags) epq_arowmarks = NIL; foreach(lc, node->rowMarks) { - PlanRowMark *rc = (PlanRowMark *) lfirst(lc); + PlanRowMark *rc = lfirst_node(PlanRowMark, lc); ExecRowMark *erm; ExecAuxRowMark *aerm; - Assert(IsA(rc, PlanRowMark)); - /* ignore "parent" rowmarks; they are irrelevant at runtime */ if (rc->isParent) continue; diff --git a/src/backend/executor/nodeMaterial.c b/src/backend/executor/nodeMaterial.c index 9ab03f3f15..aa5d2529f4 100644 --- a/src/backend/executor/nodeMaterial.c +++ b/src/backend/executor/nodeMaterial.c @@ -3,7 +3,7 @@ * nodeMaterial.c * Routines to handle materialization nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c index e271927077..fef83dbdbd 100644 --- a/src/backend/executor/nodeMergeAppend.c +++ b/src/backend/executor/nodeMergeAppend.c @@ -3,7 +3,7 @@ * nodeMergeAppend.c * routines to handle MergeAppend nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -72,6 +72,12 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags) Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK))); /* + * Lock the non-leaf tables in the partition tree controlled by this node. + * It's a no-op for non-partitioned parent tables. + */ + ExecLockNonLeafAppendTables(node->partitioned_rels, estate); + + /* * Set up empty vector of subplan states */ nplans = list_length(node->mergeplans); diff --git a/src/backend/executor/nodeMergejoin.c b/src/backend/executor/nodeMergejoin.c index 6db09b836a..572e9dce94 100644 --- a/src/backend/executor/nodeMergejoin.c +++ b/src/backend/executor/nodeMergejoin.c @@ -3,7 +3,7 @@ * nodeMergejoin.c * routines supporting merge joins * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -313,7 +313,7 @@ MJEvalOuterValues(MergeJoinState *mergestate) MergeJoinClause clause = &mergestate->mj_Clauses[i]; clause->ldatum = ExecEvalExpr(clause->lexpr, econtext, - &clause->lisnull, NULL); + &clause->lisnull); if (clause->lisnull) { /* match is impossible; can we end the join early? */ @@ -360,7 +360,7 @@ MJEvalInnerValues(MergeJoinState *mergestate, TupleTableSlot *innerslot) MergeJoinClause clause = &mergestate->mj_Clauses[i]; clause->rdatum = ExecEvalExpr(clause->rexpr, econtext, - &clause->risnull, NULL); + &clause->risnull); if (clause->risnull) { /* match is impossible; can we end the join early? */ @@ -452,32 +452,22 @@ static TupleTableSlot * MJFillOuter(MergeJoinState *node) { ExprContext *econtext = node->js.ps.ps_ExprContext; - List *otherqual = node->js.ps.qual; + ExprState *otherqual = node->js.ps.qual; ResetExprContext(econtext); econtext->ecxt_outertuple = node->mj_OuterTupleSlot; econtext->ecxt_innertuple = node->mj_NullInnerTupleSlot; - if (ExecQual(otherqual, econtext, false)) + if (ExecQual(otherqual, econtext)) { /* * qualification succeeded. now form the desired projection tuple and * return the slot containing it. */ - TupleTableSlot *result; - ExprDoneCond isDone; - MJ_printf("ExecMergeJoin: returning outer fill tuple\n"); - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->js.ps.ps_ProjInfo); } else InstrCountFiltered2(node, 1); @@ -493,32 +483,22 @@ static TupleTableSlot * MJFillInner(MergeJoinState *node) { ExprContext *econtext = node->js.ps.ps_ExprContext; - List *otherqual = node->js.ps.qual; + ExprState *otherqual = node->js.ps.qual; ResetExprContext(econtext); econtext->ecxt_outertuple = node->mj_NullOuterTupleSlot; econtext->ecxt_innertuple = node->mj_InnerTupleSlot; - if (ExecQual(otherqual, econtext, false)) + if (ExecQual(otherqual, econtext)) { /* * qualification succeeded. now form the desired projection tuple and * return the slot containing it. */ - TupleTableSlot *result; - ExprDoneCond isDone; - MJ_printf("ExecMergeJoin: returning inner fill tuple\n"); - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->js.ps.ps_ProjInfo); } else InstrCountFiltered2(node, 1); @@ -607,7 +587,7 @@ ExecMergeTupleDump(MergeJoinState *mergestate) ExecMergeTupleDumpInner(mergestate); ExecMergeTupleDumpMarked(mergestate); - printf("******** \n"); + printf("********\n"); } #endif @@ -618,8 +598,8 @@ ExecMergeTupleDump(MergeJoinState *mergestate) TupleTableSlot * ExecMergeJoin(MergeJoinState *node) { - List *joinqual; - List *otherqual; + ExprState *joinqual; + ExprState *otherqual; bool qualResult; int compareResult; PlanState *innerPlan; @@ -642,26 +622,8 @@ ExecMergeJoin(MergeJoinState *node) doFillInner = node->mj_FillInner; /* - * Check to see if we're still projecting out tuples from a previous join - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->js.ps.ps_TupFromTlist) - { - TupleTableSlot *result; - ExprDoneCond isDone; - - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return result; - /* Done with that source tuple... */ - node->js.ps.ps_TupFromTlist = false; - } - - /* * Reset per-tuple memory context to free any expression evaluation - * storage allocated in the previous tuple cycle. Note this can't happen - * until we're done projecting out tuples from a join tuple. + * storage allocated in the previous tuple cycle. */ ResetExprContext(econtext); @@ -823,8 +785,8 @@ ExecMergeJoin(MergeJoinState *node) innerTupleSlot = node->mj_InnerTupleSlot; econtext->ecxt_innertuple = innerTupleSlot; - qualResult = (joinqual == NIL || - ExecQual(joinqual, econtext, false)); + qualResult = (joinqual == NULL || + ExecQual(joinqual, econtext)); MJ_DEBUG_QUAL(joinqual, qualResult); if (qualResult) @@ -840,14 +802,15 @@ ExecMergeJoin(MergeJoinState *node) } /* - * In a semijoin, we'll consider returning the first - * match, but after that we're done with this outer tuple. + * If we only need to join to the first matching inner + * tuple, then consider returning this one, but after that + * continue with next outer tuple. */ - if (node->js.jointype == JOIN_SEMI) + if (node->js.single_match) node->mj_JoinState = EXEC_MJ_NEXTOUTER; - qualResult = (otherqual == NIL || - ExecQual(otherqual, econtext, false)); + qualResult = (otherqual == NULL || + ExecQual(otherqual, econtext)); MJ_DEBUG_QUAL(otherqual, qualResult); if (qualResult) @@ -856,20 +819,9 @@ ExecMergeJoin(MergeJoinState *node) * qualification succeeded. now form the desired * projection tuple and return the slot containing it. */ - TupleTableSlot *result; - ExprDoneCond isDone; - MJ_printf("ExecMergeJoin: returning tuple\n"); - result = ExecProject(node->js.ps.ps_ProjInfo, - &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->js.ps.ps_ProjInfo); } else InstrCountFiltered2(node, 1); @@ -1099,6 +1051,10 @@ ExecMergeJoin(MergeJoinState *node) * scan position to the first mark, and go join that tuple * (and any following ones) to the new outer. * + * If we were able to determine mark and restore are not + * needed, then we don't have to back up; the current + * inner is already the first possible match. + * * NOTE: we do not need to worry about the MatchedInner * state for the rescanned inner tuples. We know all of * them will match this new outer tuple and therefore @@ -1111,16 +1067,19 @@ ExecMergeJoin(MergeJoinState *node) * forcing the merge clause to never match, so we never * get here. */ - ExecRestrPos(innerPlan); + if (!node->mj_SkipMarkRestore) + { + ExecRestrPos(innerPlan); - /* - * ExecRestrPos probably should give us back a new Slot, - * but since it doesn't, use the marked slot. (The - * previously returned mj_InnerTupleSlot cannot be assumed - * to hold the required tuple.) - */ - node->mj_InnerTupleSlot = innerTupleSlot; - /* we need not do MJEvalInnerValues again */ + /* + * ExecRestrPos probably should give us back a new + * Slot, but since it doesn't, use the marked slot. + * (The previously returned mj_InnerTupleSlot cannot + * be assumed to hold the required tuple.) + */ + node->mj_InnerTupleSlot = innerTupleSlot; + /* we need not do MJEvalInnerValues again */ + } node->mj_JoinState = EXEC_MJ_JOINTUPLES; } @@ -1221,7 +1180,8 @@ ExecMergeJoin(MergeJoinState *node) if (compareResult == 0) { - ExecMarkPos(innerPlan); + if (!node->mj_SkipMarkRestore) + ExecMarkPos(innerPlan); MarkInnerTuple(node->mj_InnerTupleSlot, node); @@ -1504,27 +1464,29 @@ ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags) /* * initialize child expressions */ - mergestate->js.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->join.plan.targetlist, - (PlanState *) mergestate); - mergestate->js.ps.qual = (List *) - ExecInitExpr((Expr *) node->join.plan.qual, - (PlanState *) mergestate); + mergestate->js.ps.qual = + ExecInitQual(node->join.plan.qual, (PlanState *) mergestate); mergestate->js.jointype = node->join.jointype; - mergestate->js.joinqual = (List *) - ExecInitExpr((Expr *) node->join.joinqual, - (PlanState *) mergestate); + mergestate->js.joinqual = + ExecInitQual(node->join.joinqual, (PlanState *) mergestate); mergestate->mj_ConstFalseJoin = false; /* mergeclauses are handled below */ /* * initialize child nodes * - * inner child must support MARK/RESTORE. + * inner child must support MARK/RESTORE, unless we have detected that we + * don't need that. Note that skip_mark_restore must never be set if + * there are non-mergeclause joinquals, since the logic wouldn't work. */ + Assert(node->join.joinqual == NIL || !node->skip_mark_restore); + mergestate->mj_SkipMarkRestore = node->skip_mark_restore; + outerPlanState(mergestate) = ExecInitNode(outerPlan(node), estate, eflags); innerPlanState(mergestate) = ExecInitNode(innerPlan(node), estate, - eflags | EXEC_FLAG_MARK); + mergestate->mj_SkipMarkRestore ? + eflags : + (eflags | EXEC_FLAG_MARK)); /* * For certain types of inner child nodes, it is advantageous to issue @@ -1537,7 +1499,8 @@ ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags) * only if eflags doesn't specify REWIND. */ if (IsA(innerPlan(node), Material) && - (eflags & EXEC_FLAG_REWIND) == 0) + (eflags & EXEC_FLAG_REWIND) == 0 && + !mergestate->mj_SkipMarkRestore) mergestate->mj_ExtraMarks = true; else mergestate->mj_ExtraMarks = false; @@ -1551,6 +1514,13 @@ ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags) ExecSetSlotDescriptor(mergestate->mj_MarkedTupleSlot, ExecGetResultType(innerPlanState(mergestate))); + /* + * detect whether we need only consider the first matching inner tuple + */ + mergestate->js.single_match = (node->join.inner_unique || + node->join.jointype == JOIN_SEMI); + + /* set up null tuples for outer joins, if needed */ switch (node->join.jointype) { case JOIN_INNER: @@ -1629,7 +1599,6 @@ ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags) * initialize join state */ mergestate->mj_JoinState = EXEC_MJ_INITIALIZE_OUTER; - mergestate->js.ps.ps_TupFromTlist = false; mergestate->mj_MatchedOuter = false; mergestate->mj_MatchedInner = false; mergestate->mj_OuterTupleSlot = NULL; @@ -1684,7 +1653,6 @@ ExecReScanMergeJoin(MergeJoinState *node) ExecClearTuple(node->mj_MarkedTupleSlot); node->mj_JoinState = EXEC_MJ_INITIALIZE_OUTER; - node->js.ps.ps_TupFromTlist = false; node->mj_MatchedOuter = false; node->mj_MatchedInner = false; node->mj_OuterTupleSlot = NULL; diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c index 439e36ee3a..0ee82e3add 100644 --- a/src/backend/executor/nodeModifyTable.c +++ b/src/backend/executor/nodeModifyTable.c @@ -4,7 +4,7 @@ * routines to handle ModifyTable nodes. * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -46,6 +46,7 @@ #include "foreign/fdwapi.h" #include "miscadmin.h" #include "nodes/nodeFuncs.h" +#include "parser/parsetree.h" #include "storage/bufmgr.h" #include "storage/lmgr.h" #include "utils/builtins.h" @@ -176,7 +177,7 @@ ExecProcessReturning(ResultRelInfo *resultRelInfo, econtext->ecxt_outertuple = planSlot; /* Compute the RETURNING expressions */ - return ExecProject(projectReturning, NULL); + return ExecProject(projectReturning); } /* @@ -195,10 +196,25 @@ ExecCheckHeapTupleVisible(EState *estate, if (!IsolationUsesXactSnapshot()) return; + /* + * We need buffer pin and lock to call HeapTupleSatisfiesVisibility. + * Caller should be holding pin, but not lock. + */ + LockBuffer(buffer, BUFFER_LOCK_SHARE); if (!HeapTupleSatisfiesVisibility(tuple, estate->es_snapshot, buffer)) - ereport(ERROR, - (errcode(ERRCODE_T_R_SERIALIZATION_FAILURE), + { + /* + * We should not raise a serialization failure if the conflict is + * against a tuple inserted by our own transaction, even if it's not + * visible to our snapshot. (This would happen, for example, if + * conflicting keys are proposed for insertion in a single command.) + */ + if (!TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(tuple->t_data))) + ereport(ERROR, + (errcode(ERRCODE_T_R_SERIALIZATION_FAILURE), errmsg("could not serialize access due to concurrent update"))); + } + LockBuffer(buffer, BUFFER_LOCK_UNLOCK); } /* @@ -244,9 +260,11 @@ ExecInsert(ModifyTableState *mtstate, { HeapTuple tuple; ResultRelInfo *resultRelInfo; + ResultRelInfo *saved_resultRelInfo = NULL; Relation resultRelationDesc; Oid newId; List *recheckIndexes = NIL; + TupleTableSlot *result = NULL; /* * get the heap tuple out of the tuple table slot, making sure we have a @@ -258,6 +276,67 @@ ExecInsert(ModifyTableState *mtstate, * get information on the (current) result relation */ resultRelInfo = estate->es_result_relation_info; + + /* Determine the partition to heap_insert the tuple into */ + if (mtstate->mt_partition_dispatch_info) + { + int leaf_part_index; + TupleConversionMap *map; + + /* + * Away we go ... If we end up not finding a partition after all, + * ExecFindPartition() does not return and errors out instead. + * Otherwise, the returned value is to be used as an index into arrays + * mt_partitions[] and mt_partition_tupconv_maps[] that will get us + * the ResultRelInfo and TupleConversionMap for the partition, + * respectively. + */ + leaf_part_index = ExecFindPartition(resultRelInfo, + mtstate->mt_partition_dispatch_info, + slot, + estate); + Assert(leaf_part_index >= 0 && + leaf_part_index < mtstate->mt_num_partitions); + + /* + * Save the old ResultRelInfo and switch to the one corresponding to + * the selected partition. + */ + saved_resultRelInfo = resultRelInfo; + resultRelInfo = mtstate->mt_partitions + leaf_part_index; + + /* We do not yet have a way to insert into a foreign partition */ + if (resultRelInfo->ri_FdwRoutine) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot route inserted tuples to a foreign table"))); + + /* For ExecInsertIndexTuples() to work on the partition's indexes */ + estate->es_result_relation_info = resultRelInfo; + + /* + * We might need to convert from the parent rowtype to the partition + * rowtype. + */ + map = mtstate->mt_partition_tupconv_maps[leaf_part_index]; + if (map) + { + Relation partrel = resultRelInfo->ri_RelationDesc; + + tuple = do_convert_tuple(tuple, map); + + /* + * We must use the partition's tuple descriptor from this point + * on, until we're finished dealing with the partition. Use the + * dedicated slot for that. + */ + slot = mtstate->mt_partition_tuple_slot; + Assert(slot != NULL); + ExecSetSlotDescriptor(slot, RelationGetDescr(partrel)); + ExecStoreTuple(tuple, slot, InvalidBuffer, true); + } + } + resultRelationDesc = resultRelInfo->ri_RelationDesc; /* * If the result relation has OIDs, force the tuple's OID to zero so that @@ -354,7 +433,7 @@ ExecInsert(ModifyTableState *mtstate, /* * Check the constraints of the tuple */ - if (resultRelationDesc->rd_att->constr) + if (resultRelationDesc->rd_att->constr || resultRelInfo->ri_PartitionCheck) ExecConstraints(resultRelInfo, slot, estate); if (onconflict != ONCONFLICT_NONE && resultRelInfo->ri_NumIndices > 0) @@ -513,9 +592,12 @@ ExecInsert(ModifyTableState *mtstate, /* Process RETURNING if present */ if (resultRelInfo->ri_projectReturning) - return ExecProcessReturning(resultRelInfo, slot, planSlot); + result = ExecProcessReturning(resultRelInfo, slot, planSlot); - return NULL; + if (saved_resultRelInfo) + estate->es_result_relation_info = saved_resultRelInfo; + + return result; } /* ---------------------------------------------------------------- @@ -906,9 +988,11 @@ lreplace:; resultRelInfo, slot, estate); /* - * Check the constraints of the tuple + * Check the constraints of the tuple. Note that we pass the same + * slot for the orig_slot argument, because unlike ExecInsert(), no + * tuple-routing is performed here, hence the slot remains unchanged. */ - if (resultRelationDesc->rd_att->constr) + if (resultRelationDesc->rd_att->constr || resultRelInfo->ri_PartitionCheck) ExecConstraints(resultRelInfo, slot, estate); /* @@ -1068,7 +1152,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate, { ExprContext *econtext = mtstate->ps.ps_ExprContext; Relation relation = resultRelInfo->ri_RelationDesc; - List *onConflictSetWhere = resultRelInfo->ri_onConflictSetWhere; + ExprState *onConflictSetWhere = resultRelInfo->ri_onConflictSetWhere; HeapTupleData tuple; HeapUpdateFailureData hufd; LockTupleMode lockmode; @@ -1187,7 +1271,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate, econtext->ecxt_innertuple = excludedSlot; econtext->ecxt_outertuple = NULL; - if (!ExecQual(onConflictSetWhere, econtext, false)) + if (!ExecQual(onConflictSetWhere, econtext)) { ReleaseBuffer(buffer); InstrCountFiltered1(&mtstate->ps, 1); @@ -1217,7 +1301,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate, } /* Project the new tuple version */ - ExecProject(resultRelInfo->ri_onConflictSetProj, NULL); + ExecProject(resultRelInfo->ri_onConflictSetProj); /* * Note that it is possible that the target tuple has been modified in @@ -1245,19 +1329,29 @@ ExecOnConflictUpdate(ModifyTableState *mtstate, static void fireBSTriggers(ModifyTableState *node) { + ResultRelInfo *resultRelInfo = node->resultRelInfo; + + /* + * If the node modifies a partitioned table, we must fire its triggers. + * Note that in that case, node->resultRelInfo points to the first leaf + * partition, not the root table. + */ + if (node->rootResultRelInfo != NULL) + resultRelInfo = node->rootResultRelInfo; + switch (node->operation) { case CMD_INSERT: - ExecBSInsertTriggers(node->ps.state, node->resultRelInfo); + ExecBSInsertTriggers(node->ps.state, resultRelInfo); if (node->mt_onconflict == ONCONFLICT_UPDATE) ExecBSUpdateTriggers(node->ps.state, - node->resultRelInfo); + resultRelInfo); break; case CMD_UPDATE: - ExecBSUpdateTriggers(node->ps.state, node->resultRelInfo); + ExecBSUpdateTriggers(node->ps.state, resultRelInfo); break; case CMD_DELETE: - ExecBSDeleteTriggers(node->ps.state, node->resultRelInfo); + ExecBSDeleteTriggers(node->ps.state, resultRelInfo); break; default: elog(ERROR, "unknown operation"); @@ -1271,19 +1365,29 @@ fireBSTriggers(ModifyTableState *node) static void fireASTriggers(ModifyTableState *node) { + ResultRelInfo *resultRelInfo = node->resultRelInfo; + + /* + * If the node modifies a partitioned table, we must fire its triggers. + * Note that in that case, node->resultRelInfo points to the first leaf + * partition, not the root table. + */ + if (node->rootResultRelInfo != NULL) + resultRelInfo = node->rootResultRelInfo; + switch (node->operation) { case CMD_INSERT: if (node->mt_onconflict == ONCONFLICT_UPDATE) ExecASUpdateTriggers(node->ps.state, - node->resultRelInfo); - ExecASInsertTriggers(node->ps.state, node->resultRelInfo); + resultRelInfo); + ExecASInsertTriggers(node->ps.state, resultRelInfo); break; case CMD_UPDATE: - ExecASUpdateTriggers(node->ps.state, node->resultRelInfo); + ExecASUpdateTriggers(node->ps.state, resultRelInfo); break; case CMD_DELETE: - ExecASDeleteTriggers(node->ps.state, node->resultRelInfo); + ExecASDeleteTriggers(node->ps.state, resultRelInfo); break; default: elog(ERROR, "unknown operation"); @@ -1551,6 +1655,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) Plan *subplan; ListCell *l; int i; + Relation rel; /* check for unsupported flags */ Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK))); @@ -1561,7 +1666,6 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) mtstate = makeNode(ModifyTableState); mtstate->ps.plan = (Plan *) node; mtstate->ps.state = estate; - mtstate->ps.targetlist = NIL; /* not actually used */ mtstate->operation = operation; mtstate->canSetTag = node->canSetTag; @@ -1569,6 +1673,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) mtstate->mt_plans = (PlanState **) palloc0(sizeof(PlanState *) * nplans); mtstate->resultRelInfo = estate->es_result_relations + node->resultRelIndex; + + /* If modifying a partitioned table, initialize the root table info */ + if (node->rootResultRelIndex >= 0) + mtstate->rootResultRelInfo = estate->es_root_result_relations + + node->rootResultRelIndex; + mtstate->mt_arowmarks = (List **) palloc0(sizeof(List *) * nplans); mtstate->mt_nplans = nplans; mtstate->mt_onconflict = node->onConflictAction; @@ -1642,6 +1752,44 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) estate->es_result_relation_info = saved_resultRelInfo; + /* The root table RT index is at the head of the partitioned_rels list */ + if (node->partitioned_rels) + { + Index root_rti; + Oid root_oid; + + root_rti = linitial_int(node->partitioned_rels); + root_oid = getrelid(root_rti, estate->es_range_table); + rel = heap_open(root_oid, NoLock); /* locked by InitPlan */ + } + else + rel = mtstate->resultRelInfo->ri_RelationDesc; + + /* Build state for INSERT tuple routing */ + if (operation == CMD_INSERT && + rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE) + { + PartitionDispatch *partition_dispatch_info; + ResultRelInfo *partitions; + TupleConversionMap **partition_tupconv_maps; + TupleTableSlot *partition_tuple_slot; + int num_parted, + num_partitions; + + ExecSetupPartitionTupleRouting(rel, + &partition_dispatch_info, + &partitions, + &partition_tupconv_maps, + &partition_tuple_slot, + &num_parted, &num_partitions); + mtstate->mt_partition_dispatch_info = partition_dispatch_info; + mtstate->mt_num_dispatch = num_parted; + mtstate->mt_partitions = partitions; + mtstate->mt_num_partitions = num_partitions; + mtstate->mt_partition_tupconv_maps = partition_tupconv_maps; + mtstate->mt_partition_tuple_slot = partition_tuple_slot; + } + /* * Initialize any WITH CHECK OPTION constraints if needed. */ @@ -1656,7 +1804,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) foreach(ll, wcoList) { WithCheckOption *wco = (WithCheckOption *) lfirst(ll); - ExprState *wcoExpr = ExecInitExpr((Expr *) wco->qual, + ExprState *wcoExpr = ExecInitQual((List *) wco->qual, mtstate->mt_plans[i]); wcoExprs = lappend(wcoExprs, wcoExpr); @@ -1669,12 +1817,53 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) } /* + * Build WITH CHECK OPTION constraints for each leaf partition rel. Note + * that we didn't build the withCheckOptionList for each partition within + * the planner, but simple translation of the varattnos for each partition + * will suffice. This only occurs for the INSERT case; UPDATE/DELETE + * cases are handled above. + */ + if (node->withCheckOptionLists != NIL && mtstate->mt_num_partitions > 0) + { + List *wcoList; + + Assert(operation == CMD_INSERT); + resultRelInfo = mtstate->mt_partitions; + wcoList = linitial(node->withCheckOptionLists); + for (i = 0; i < mtstate->mt_num_partitions; i++) + { + Relation partrel = resultRelInfo->ri_RelationDesc; + List *mapped_wcoList; + List *wcoExprs = NIL; + ListCell *ll; + + /* varno = node->nominalRelation */ + mapped_wcoList = map_partition_varattnos(wcoList, + node->nominalRelation, + partrel, rel); + foreach(ll, mapped_wcoList) + { + WithCheckOption *wco = (WithCheckOption *) lfirst(ll); + ExprState *wcoExpr = ExecInitQual((List *) wco->qual, + mtstate->mt_plans[i]); + + wcoExprs = lappend(wcoExprs, wcoExpr); + } + + resultRelInfo->ri_WithCheckOptions = mapped_wcoList; + resultRelInfo->ri_WithCheckOptionExprs = wcoExprs; + resultRelInfo++; + } + } + + /* * Initialize RETURNING projections if needed. */ if (node->returningLists) { TupleTableSlot *slot; ExprContext *econtext; + List *returningList; /* * Initialize result tuple slot and assign its rowtype using the first @@ -1689,8 +1878,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) slot = mtstate->ps.ps_ResultTupleSlot; /* Need an econtext too */ - econtext = CreateExprContext(estate); - mtstate->ps.ps_ExprContext = econtext; + if (mtstate->ps.ps_ExprContext == NULL) + ExecAssignExprContext(estate, &mtstate->ps); + econtext = mtstate->ps.ps_ExprContext; /* * Build a projection for each result rel. @@ -1699,11 +1889,33 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) foreach(l, node->returningLists) { List *rlist = (List *) lfirst(l); - List *rliststate; - rliststate = (List *) ExecInitExpr((Expr *) rlist, &mtstate->ps); resultRelInfo->ri_projectReturning = - ExecBuildProjectionInfo(rliststate, econtext, slot, + ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps, + resultRelInfo->ri_RelationDesc->rd_att); + resultRelInfo++; + } + + /* + * Build a projection for each leaf partition rel. Note that we + * didn't build the returningList for each partition within the + * planner, but simple translation of the varattnos for each partition + * will suffice. This only occurs for the INSERT case; UPDATE/DELETE + * are handled above. + */ + resultRelInfo = mtstate->mt_partitions; + returningList = linitial(node->returningLists); + for (i = 0; i < mtstate->mt_num_partitions; i++) + { + Relation partrel = resultRelInfo->ri_RelationDesc; + List *rlist; + + /* varno = node->nominalRelation */ + rlist = map_partition_varattnos(returningList, + node->nominalRelation, + partrel, rel); + resultRelInfo->ri_projectReturning = + ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps, resultRelInfo->ri_RelationDesc->rd_att); resultRelInfo++; } @@ -1721,6 +1933,10 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) mtstate->ps.ps_ExprContext = NULL; } + /* Close the root partitioned rel if we opened it above. */ + if (rel != mtstate->resultRelInfo->ri_RelationDesc) + heap_close(rel, NoLock); + /* * If needed, Initialize target list, projection and qual for ON CONFLICT * DO UPDATE. @@ -1729,7 +1945,6 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) if (node->onConflictAction == ONCONFLICT_UPDATE) { ExprContext *econtext; - ExprState *setexpr; TupleDesc tupDesc; /* insert may only have one plan, inheritance is not expanded */ @@ -1755,11 +1970,10 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) mtstate->mt_conflproj = ExecInitExtraTupleSlot(mtstate->ps.state); ExecSetSlotDescriptor(mtstate->mt_conflproj, tupDesc); - /* build UPDATE SET expression and projection state */ - setexpr = ExecInitExpr((Expr *) node->onConflictSet, &mtstate->ps); + /* build UPDATE SET projection state */ resultRelInfo->ri_onConflictSetProj = - ExecBuildProjectionInfo((List *) setexpr, econtext, - mtstate->mt_conflproj, + ExecBuildProjectionInfo(node->onConflictSet, econtext, + mtstate->mt_conflproj, &mtstate->ps, resultRelInfo->ri_RelationDesc->rd_att); /* build DO UPDATE WHERE clause expression */ @@ -1767,10 +1981,10 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) { ExprState *qualexpr; - qualexpr = ExecInitExpr((Expr *) node->onConflictWhere, + qualexpr = ExecInitQual((List *) node->onConflictWhere, &mtstate->ps); - resultRelInfo->ri_onConflictSetWhere = (List *) qualexpr; + resultRelInfo->ri_onConflictSetWhere = qualexpr; } } @@ -1782,11 +1996,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) */ foreach(l, node->rowMarks) { - PlanRowMark *rc = (PlanRowMark *) lfirst(l); + PlanRowMark *rc = lfirst_node(PlanRowMark, l); ExecRowMark *erm; - Assert(IsA(rc, PlanRowMark)); - /* ignore "parent" rowmarks; they are irrelevant at runtime */ if (rc->isParent) continue; @@ -1873,7 +2085,8 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags) relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind; if (relkind == RELKIND_RELATION || - relkind == RELKIND_MATVIEW) + relkind == RELKIND_MATVIEW || + relkind == RELKIND_PARTITIONED_TABLE) { j->jf_junkAttNo = ExecFindJunkAttribute(j, "ctid"); if (!AttributeNumberIsValid(j->jf_junkAttNo)) @@ -1959,6 +2172,33 @@ ExecEndModifyTable(ModifyTableState *node) } /* + * Close all the partitioned tables, leaf partitions, and their indices + * + * Remember node->mt_partition_dispatch_info[0] corresponds to the root + * partitioned table, which we must not try to close, because it is the + * main target table of the query that will be closed by ExecEndPlan(). + * Also, tupslot is NULL for the root partitioned table. + */ + for (i = 1; i < node->mt_num_dispatch; i++) + { + PartitionDispatch pd = node->mt_partition_dispatch_info[i]; + + heap_close(pd->reldesc, NoLock); + ExecDropSingleTupleTableSlot(pd->tupslot); + } + for (i = 0; i < node->mt_num_partitions; i++) + { + ResultRelInfo *resultRelInfo = node->mt_partitions + i; + + ExecCloseIndices(resultRelInfo); + heap_close(resultRelInfo->ri_RelationDesc, NoLock); + } + + /* Release the standalone partition tuple descriptor, if any */ + if (node->mt_partition_tuple_slot) + ExecDropSingleTupleTableSlot(node->mt_partition_tuple_slot); + + /* * Free the exprcontext */ ExecFreeExprContext(&node->ps); diff --git a/src/backend/executor/nodeNamedtuplestorescan.c b/src/backend/executor/nodeNamedtuplestorescan.c new file mode 100644 index 0000000000..62234869ab --- /dev/null +++ b/src/backend/executor/nodeNamedtuplestorescan.c @@ -0,0 +1,199 @@ +/*------------------------------------------------------------------------- + * + * nodeNamedtuplestorescan.c + * routines to handle NamedTuplestoreScan nodes. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * + * IDENTIFICATION + * src/backend/executor/nodeNamedtuplestorescan.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "executor/execdebug.h" +#include "executor/nodeNamedtuplestorescan.h" +#include "miscadmin.h" +#include "utils/queryenvironment.h" + +static TupleTableSlot *NamedTuplestoreScanNext(NamedTuplestoreScanState *node); + +/* ---------------------------------------------------------------- + * NamedTuplestoreScanNext + * + * This is a workhorse for ExecNamedTuplestoreScan + * ---------------------------------------------------------------- + */ +static TupleTableSlot * +NamedTuplestoreScanNext(NamedTuplestoreScanState *node) +{ + TupleTableSlot *slot; + + /* We intentionally do not support backward scan. */ + Assert(ScanDirectionIsForward(node->ss.ps.state->es_direction)); + + /* + * Get the next tuple from tuplestore. Return NULL if no more tuples. + */ + slot = node->ss.ss_ScanTupleSlot; + (void) tuplestore_gettupleslot(node->relation, true, false, slot); + return slot; +} + +/* + * NamedTuplestoreScanRecheck -- access method routine to recheck a tuple in + * EvalPlanQual + */ +static bool +NamedTuplestoreScanRecheck(NamedTuplestoreScanState *node, TupleTableSlot *slot) +{ + /* nothing to check */ + return true; +} + +/* ---------------------------------------------------------------- + * ExecNamedTuplestoreScan(node) + * + * Scans the CTE sequentially and returns the next qualifying tuple. + * We call the ExecScan() routine and pass it the appropriate + * access method functions. + * ---------------------------------------------------------------- + */ +TupleTableSlot * +ExecNamedTuplestoreScan(NamedTuplestoreScanState *node) +{ + return ExecScan(&node->ss, + (ExecScanAccessMtd) NamedTuplestoreScanNext, + (ExecScanRecheckMtd) NamedTuplestoreScanRecheck); +} + + +/* ---------------------------------------------------------------- + * ExecInitNamedTuplestoreScan + * ---------------------------------------------------------------- + */ +NamedTuplestoreScanState * +ExecInitNamedTuplestoreScan(NamedTuplestoreScan *node, EState *estate, int eflags) +{ + NamedTuplestoreScanState *scanstate; + EphemeralNamedRelation enr; + + /* check for unsupported flags */ + Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK))); + + /* + * NamedTuplestoreScan should not have any children. + */ + Assert(outerPlan(node) == NULL); + Assert(innerPlan(node) == NULL); + + /* + * create new NamedTuplestoreScanState for node + */ + scanstate = makeNode(NamedTuplestoreScanState); + scanstate->ss.ps.plan = (Plan *) node; + scanstate->ss.ps.state = estate; + + enr = get_ENR(estate->es_queryEnv, node->enrname); + if (!enr) + elog(ERROR, "executor could not find named tuplestore \"%s\"", + node->enrname); + + Assert(enr->reldata); + scanstate->relation = (Tuplestorestate *) enr->reldata; + scanstate->tupdesc = ENRMetadataGetTupDesc(&(enr->md)); + scanstate->readptr = + tuplestore_alloc_read_pointer(scanstate->relation, EXEC_FLAG_REWIND); + + /* + * The new read pointer copies its position from read pointer 0, which + * could be anywhere, so explicitly rewind it. + */ + tuplestore_rescan(scanstate->relation); + + /* + * XXX: Should we add a function to free that read pointer when done? + * + * This was attempted, but it did not improve performance or memory usage + * in any tested cases. + */ + + /* + * Miscellaneous initialization + * + * create expression context for node + */ + ExecAssignExprContext(estate, &scanstate->ss.ps); + + /* + * initialize child expressions + */ + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); + + /* + * tuple table initialization + */ + ExecInitResultTupleSlot(estate, &scanstate->ss.ps); + ExecInitScanTupleSlot(estate, &scanstate->ss); + + /* + * The scan tuple type is specified for the tuplestore. + */ + ExecAssignScanType(&scanstate->ss, scanstate->tupdesc); + + /* + * Initialize result tuple type and projection info. + */ + ExecAssignResultTypeFromTL(&scanstate->ss.ps); + ExecAssignScanProjectionInfo(&scanstate->ss); + + return scanstate; +} + +/* ---------------------------------------------------------------- + * ExecEndNamedTuplestoreScan + * + * frees any storage allocated through C routines. + * ---------------------------------------------------------------- + */ +void +ExecEndNamedTuplestoreScan(NamedTuplestoreScanState *node) +{ + /* + * Free exprcontext + */ + ExecFreeExprContext(&node->ss.ps); + + /* + * clean out the tuple table + */ + ExecClearTuple(node->ss.ps.ps_ResultTupleSlot); + ExecClearTuple(node->ss.ss_ScanTupleSlot); +} + +/* ---------------------------------------------------------------- + * ExecReScanNamedTuplestoreScan + * + * Rescans the relation. + * ---------------------------------------------------------------- + */ +void +ExecReScanNamedTuplestoreScan(NamedTuplestoreScanState *node) +{ + Tuplestorestate *tuplestorestate = node->relation; + + ExecClearTuple(node->ss.ps.ps_ResultTupleSlot); + + ExecScanReScan(&node->ss); + + /* + * Rewind my own pointer. + */ + tuplestore_select_read_pointer(tuplestorestate, node->readptr); + tuplestore_rescan(tuplestorestate); +} diff --git a/src/backend/executor/nodeNestloop.c b/src/backend/executor/nodeNestloop.c index d67f2e7955..67c8269b96 100644 --- a/src/backend/executor/nodeNestloop.c +++ b/src/backend/executor/nodeNestloop.c @@ -3,7 +3,7 @@ * nodeNestloop.c * routines to support nest-loop joins * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -64,8 +64,8 @@ ExecNestLoop(NestLoopState *node) PlanState *outerPlan; TupleTableSlot *outerTupleSlot; TupleTableSlot *innerTupleSlot; - List *joinqual; - List *otherqual; + ExprState *joinqual; + ExprState *otherqual; ExprContext *econtext; ListCell *lc; @@ -82,26 +82,8 @@ ExecNestLoop(NestLoopState *node) econtext = node->js.ps.ps_ExprContext; /* - * Check to see if we're still projecting out tuples from a previous join - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->js.ps.ps_TupFromTlist) - { - TupleTableSlot *result; - ExprDoneCond isDone; - - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return result; - /* Done with that source tuple... */ - node->js.ps.ps_TupFromTlist = false; - } - - /* * Reset per-tuple memory context to free any expression evaluation - * storage allocated in the previous tuple cycle. Note this can't happen - * until we're done projecting out tuples from a join tuple. + * storage allocated in the previous tuple cycle. */ ResetExprContext(econtext); @@ -195,26 +177,16 @@ ExecNestLoop(NestLoopState *node) ENL1_printf("testing qualification for outer-join tuple"); - if (otherqual == NIL || ExecQual(otherqual, econtext, false)) + if (otherqual == NULL || ExecQual(otherqual, econtext)) { /* * qualification was satisfied so we project and return * the slot containing the result tuple using * ExecProject(). */ - TupleTableSlot *result; - ExprDoneCond isDone; - ENL1_printf("qualification succeeded, projecting tuple"); - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->js.ps.ps_ProjInfo); } else InstrCountFiltered2(node, 1); @@ -236,7 +208,7 @@ ExecNestLoop(NestLoopState *node) */ ENL1_printf("testing qualification"); - if (ExecQual(joinqual, econtext, false)) + if (ExecQual(joinqual, econtext)) { node->nl_MatchedOuter = true; @@ -248,31 +220,22 @@ ExecNestLoop(NestLoopState *node) } /* - * In a semijoin, we'll consider returning the first match, but - * after that we're done with this outer tuple. + * If we only need to join to the first matching inner tuple, then + * consider returning this one, but after that continue with next + * outer tuple. */ - if (node->js.jointype == JOIN_SEMI) + if (node->js.single_match) node->nl_NeedNewOuter = true; - if (otherqual == NIL || ExecQual(otherqual, econtext, false)) + if (otherqual == NULL || ExecQual(otherqual, econtext)) { /* * qualification was satisfied so we project and return the * slot containing the result tuple using ExecProject(). */ - TupleTableSlot *result; - ExprDoneCond isDone; - ENL1_printf("qualification succeeded, projecting tuple"); - result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->js.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; - } + return ExecProject(node->js.ps.ps_ProjInfo); } else InstrCountFiltered2(node, 1); @@ -321,16 +284,11 @@ ExecInitNestLoop(NestLoop *node, EState *estate, int eflags) /* * initialize child expressions */ - nlstate->js.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->join.plan.targetlist, - (PlanState *) nlstate); - nlstate->js.ps.qual = (List *) - ExecInitExpr((Expr *) node->join.plan.qual, - (PlanState *) nlstate); + nlstate->js.ps.qual = + ExecInitQual(node->join.plan.qual, (PlanState *) nlstate); nlstate->js.jointype = node->join.jointype; - nlstate->js.joinqual = (List *) - ExecInitExpr((Expr *) node->join.joinqual, - (PlanState *) nlstate); + nlstate->js.joinqual = + ExecInitQual(node->join.joinqual, (PlanState *) nlstate); /* * initialize child nodes @@ -353,6 +311,13 @@ ExecInitNestLoop(NestLoop *node, EState *estate, int eflags) */ ExecInitResultTupleSlot(estate, &nlstate->js.ps); + /* + * detect whether we need only consider the first matching inner tuple + */ + nlstate->js.single_match = (node->join.inner_unique || + node->join.jointype == JOIN_SEMI); + + /* set up null tuples for outer joins, if needed */ switch (node->join.jointype) { case JOIN_INNER: @@ -378,7 +343,6 @@ ExecInitNestLoop(NestLoop *node, EState *estate, int eflags) /* * finally, wipe the current outer tuple clean. */ - nlstate->js.ps.ps_TupFromTlist = false; nlstate->nl_NeedNewOuter = true; nlstate->nl_MatchedOuter = false; @@ -442,7 +406,6 @@ ExecReScanNestLoop(NestLoopState *node) * outer Vars are used as run-time keys... */ - node->js.ps.ps_TupFromTlist = false; node->nl_NeedNewOuter = true; node->nl_MatchedOuter = false; } diff --git a/src/backend/executor/nodeProjectSet.c b/src/backend/executor/nodeProjectSet.c new file mode 100644 index 0000000000..2f0a4e647b --- /dev/null +++ b/src/backend/executor/nodeProjectSet.c @@ -0,0 +1,323 @@ +/*------------------------------------------------------------------------- + * + * nodeProjectSet.c + * support for evaluating targetlists containing set-returning functions + * + * DESCRIPTION + * + * ProjectSet nodes are inserted by the planner to evaluate set-returning + * functions in the targetlist. It's guaranteed that all set-returning + * functions are directly at the top level of the targetlist, i.e. they + * can't be inside more-complex expressions. If that'd otherwise be + * the case, the planner adds additional ProjectSet nodes. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * IDENTIFICATION + * src/backend/executor/nodeProjectSet.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "executor/executor.h" +#include "executor/nodeProjectSet.h" +#include "nodes/nodeFuncs.h" +#include "utils/memutils.h" + + +static TupleTableSlot *ExecProjectSRF(ProjectSetState *node, bool continuing); + + +/* ---------------------------------------------------------------- + * ExecProjectSet(node) + * + * Return tuples after evaluating the targetlist (which contains set + * returning functions). + * ---------------------------------------------------------------- + */ +TupleTableSlot * +ExecProjectSet(ProjectSetState *node) +{ + TupleTableSlot *outerTupleSlot; + TupleTableSlot *resultSlot; + PlanState *outerPlan; + ExprContext *econtext; + + econtext = node->ps.ps_ExprContext; + + /* + * Check to see if we're still projecting out tuples from a previous scan + * tuple (because there is a function-returning-set in the projection + * expressions). If so, try to project another one. + */ + if (node->pending_srf_tuples) + { + resultSlot = ExecProjectSRF(node, true); + + if (resultSlot != NULL) + return resultSlot; + } + + /* + * Reset per-tuple memory context to free any expression evaluation + * storage allocated in the previous tuple cycle. Note this can't happen + * until we're done projecting out tuples from a scan tuple. + */ + ResetExprContext(econtext); + + /* + * Get another input tuple and project SRFs from it. + */ + for (;;) + { + /* + * Retrieve tuples from the outer plan until there are no more. + */ + outerPlan = outerPlanState(node); + outerTupleSlot = ExecProcNode(outerPlan); + + if (TupIsNull(outerTupleSlot)) + return NULL; + + /* + * Prepare to compute projection expressions, which will expect to + * access the input tuples as varno OUTER. + */ + econtext->ecxt_outertuple = outerTupleSlot; + + /* Evaluate the expressions */ + resultSlot = ExecProjectSRF(node, false); + + /* + * Return the tuple unless the projection produced no rows (due to an + * empty set), in which case we must loop back to see if there are + * more outerPlan tuples. + */ + if (resultSlot) + return resultSlot; + } + + return NULL; +} + +/* ---------------------------------------------------------------- + * ExecProjectSRF + * + * Project a targetlist containing one or more set-returning functions. + * + * 'continuing' indicates whether to continue projecting rows for the + * same input tuple; or whether a new input tuple is being projected. + * + * Returns NULL if no output tuple has been produced. + * + * ---------------------------------------------------------------- + */ +static TupleTableSlot * +ExecProjectSRF(ProjectSetState *node, bool continuing) +{ + TupleTableSlot *resultSlot = node->ps.ps_ResultTupleSlot; + ExprContext *econtext = node->ps.ps_ExprContext; + bool hassrf PG_USED_FOR_ASSERTS_ONLY; + bool hasresult; + int argno; + + ExecClearTuple(resultSlot); + + /* + * Assume no further tuples are produced unless an ExprMultipleResult is + * encountered from a set returning function. + */ + node->pending_srf_tuples = false; + + hassrf = hasresult = false; + for (argno = 0; argno < node->nelems; argno++) + { + Node *elem = node->elems[argno]; + ExprDoneCond *isdone = &node->elemdone[argno]; + Datum *result = &resultSlot->tts_values[argno]; + bool *isnull = &resultSlot->tts_isnull[argno]; + + if (continuing && *isdone == ExprEndResult) + { + /* + * If we're continuing to project output rows from a source tuple, + * return NULLs once the SRF has been exhausted. + */ + *result = (Datum) 0; + *isnull = true; + hassrf = true; + } + else if (IsA(elem, SetExprState)) + { + /* + * Evaluate SRF - possibly continuing previously started output. + */ + *result = ExecMakeFunctionResultSet((SetExprState *) elem, + econtext, isnull, isdone); + + if (*isdone != ExprEndResult) + hasresult = true; + if (*isdone == ExprMultipleResult) + node->pending_srf_tuples = true; + hassrf = true; + } + else + { + /* Non-SRF tlist expression, just evaluate normally. */ + *result = ExecEvalExpr((ExprState *) elem, econtext, isnull); + *isdone = ExprSingleResult; + } + } + + /* ProjectSet should not be used if there's no SRFs */ + Assert(hassrf); + + /* + * If all the SRFs returned EndResult, we consider that as no row being + * produced. + */ + if (hasresult) + { + ExecStoreVirtualTuple(resultSlot); + return resultSlot; + } + + return NULL; +} + +/* ---------------------------------------------------------------- + * ExecInitProjectSet + * + * Creates the run-time state information for the ProjectSet node + * produced by the planner and initializes outer relations + * (child nodes). + * ---------------------------------------------------------------- + */ +ProjectSetState * +ExecInitProjectSet(ProjectSet *node, EState *estate, int eflags) +{ + ProjectSetState *state; + ListCell *lc; + int off; + + /* check for unsupported flags */ + Assert(!(eflags & (EXEC_FLAG_MARK | EXEC_FLAG_BACKWARD))); + + /* + * create state structure + */ + state = makeNode(ProjectSetState); + state->ps.plan = (Plan *) node; + state->ps.state = estate; + + state->pending_srf_tuples = false; + + /* + * Miscellaneous initialization + * + * create expression context for node + */ + ExecAssignExprContext(estate, &state->ps); + + /* + * tuple table initialization + */ + ExecInitResultTupleSlot(estate, &state->ps); + + /* We don't support any qual on ProjectSet nodes */ + Assert(node->plan.qual == NIL); + + /* + * initialize child nodes + */ + outerPlanState(state) = ExecInitNode(outerPlan(node), estate, eflags); + + /* + * we don't use inner plan + */ + Assert(innerPlan(node) == NULL); + + /* + * initialize tuple type and projection info + */ + ExecAssignResultTypeFromTL(&state->ps); + + /* Create workspace for per-tlist-entry expr state & SRF-is-done state */ + state->nelems = list_length(node->plan.targetlist); + state->elems = (Node **) + palloc(sizeof(Node *) * state->nelems); + state->elemdone = (ExprDoneCond *) + palloc(sizeof(ExprDoneCond) * state->nelems); + + /* + * Build expressions to evaluate targetlist. We can't use + * ExecBuildProjectionInfo here, since that doesn't deal with SRFs. + * Instead compile each expression separately, using + * ExecInitFunctionResultSet where applicable. + */ + off = 0; + foreach(lc, node->plan.targetlist) + { + TargetEntry *te = (TargetEntry *) lfirst(lc); + Expr *expr = te->expr; + + if ((IsA(expr, FuncExpr) &&((FuncExpr *) expr)->funcretset) || + (IsA(expr, OpExpr) &&((OpExpr *) expr)->opretset)) + { + state->elems[off] = (Node *) + ExecInitFunctionResultSet(expr, state->ps.ps_ExprContext, + &state->ps); + } + else + { + Assert(!expression_returns_set((Node *) expr)); + state->elems[off] = (Node *) ExecInitExpr(expr, &state->ps); + } + + off++; + } + + return state; +} + +/* ---------------------------------------------------------------- + * ExecEndProjectSet + * + * frees up storage allocated through C routines + * ---------------------------------------------------------------- + */ +void +ExecEndProjectSet(ProjectSetState *node) +{ + /* + * Free the exprcontext + */ + ExecFreeExprContext(&node->ps); + + /* + * clean out the tuple table + */ + ExecClearTuple(node->ps.ps_ResultTupleSlot); + + /* + * shut down subplans + */ + ExecEndNode(outerPlanState(node)); +} + +void +ExecReScanProjectSet(ProjectSetState *node) +{ + /* Forget any incompletely-evaluated SRFs */ + node->pending_srf_tuples = false; + + /* + * If chgParam of subnode is not null then plan will be re-scanned by + * first ExecProcNode. + */ + if (node->ps.lefttree->chgParam == NULL) + ExecReScan(node->ps.lefttree); +} diff --git a/src/backend/executor/nodeRecursiveunion.c b/src/backend/executor/nodeRecursiveunion.c index e76405a56e..fc1c00d68f 100644 --- a/src/backend/executor/nodeRecursiveunion.c +++ b/src/backend/executor/nodeRecursiveunion.c @@ -3,7 +3,11 @@ * nodeRecursiveunion.c * routines to handle RecursiveUnion nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * To implement UNION (without ALL), we need a hashtable that stores tuples + * already seen. The hash key is computed from the grouping columns. + * + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -20,17 +24,6 @@ #include "utils/memutils.h" -/* - * To implement UNION (without ALL), we need a hashtable that stores tuples - * already seen. The hash key is computed from the grouping columns. - */ -typedef struct RUHashEntryData *RUHashEntry; - -typedef struct RUHashEntryData -{ - TupleHashEntryData shared; /* common header for hash table entries */ -} RUHashEntryData; - /* * Initialize the hash table to empty. @@ -48,9 +41,10 @@ build_hash_table(RecursiveUnionState *rustate) rustate->eqfunctions, rustate->hashfunctions, node->numGroups, - sizeof(RUHashEntryData), + 0, rustate->tableContext, - rustate->tempContext); + rustate->tempContext, + false); } @@ -200,15 +194,11 @@ ExecInitRecursiveUnion(RecursiveUnion *node, EState *estate, int eflags) rustate->tempContext = AllocSetContextCreate(CurrentMemoryContext, "RecursiveUnion", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); rustate->tableContext = AllocSetContextCreate(CurrentMemoryContext, "RecursiveUnion hash table", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); } /* diff --git a/src/backend/executor/nodeResult.c b/src/backend/executor/nodeResult.c index 4007b765bf..a753a53419 100644 --- a/src/backend/executor/nodeResult.c +++ b/src/backend/executor/nodeResult.c @@ -34,7 +34,7 @@ * plan normally and pass back the results. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * IDENTIFICATION @@ -67,10 +67,8 @@ TupleTableSlot * ExecResult(ResultState *node) { TupleTableSlot *outerTupleSlot; - TupleTableSlot *resultSlot; PlanState *outerPlan; ExprContext *econtext; - ExprDoneCond isDone; econtext = node->ps.ps_ExprContext; @@ -79,9 +77,7 @@ ExecResult(ResultState *node) */ if (node->rs_checkqual) { - bool qualResult = ExecQual((List *) node->resconstantqual, - econtext, - false); + bool qualResult = ExecQual(node->resconstantqual, econtext); node->rs_checkqual = false; if (!qualResult) @@ -92,23 +88,8 @@ ExecResult(ResultState *node) } /* - * Check to see if we're still projecting out tuples from a previous scan - * tuple (because there is a function-returning-set in the projection - * expressions). If so, try to project another one. - */ - if (node->ps.ps_TupFromTlist) - { - resultSlot = ExecProject(node->ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return resultSlot; - /* Done with that source tuple... */ - node->ps.ps_TupFromTlist = false; - } - - /* * Reset per-tuple memory context to free any expression evaluation - * storage allocated in the previous tuple cycle. Note this can't happen - * until we're done projecting out tuples from a scan tuple. + * storage allocated in the previous tuple cycle. */ ResetExprContext(econtext); @@ -147,18 +128,8 @@ ExecResult(ResultState *node) node->rs_done = true; } - /* - * form the result tuple using ExecProject(), and return it --- unless - * the projection produces an empty set, in which case we must loop - * back to see if there are more outerPlan tuples. - */ - resultSlot = ExecProject(node->ps.ps_ProjInfo, &isDone); - - if (isDone != ExprEndResult) - { - node->ps.ps_TupFromTlist = (isDone == ExprMultipleResult); - return resultSlot; - } + /* form the result tuple using ExecProject(), and return it */ + return ExecProject(node->ps.ps_ProjInfo); } return NULL; @@ -228,8 +199,6 @@ ExecInitResult(Result *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &resstate->ps); - resstate->ps.ps_TupFromTlist = false; - /* * tuple table initialization */ @@ -238,14 +207,10 @@ ExecInitResult(Result *node, EState *estate, int eflags) /* * initialize child expressions */ - resstate->ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) resstate); - resstate->ps.qual = (List *) - ExecInitExpr((Expr *) node->plan.qual, - (PlanState *) resstate); - resstate->resconstantqual = ExecInitExpr((Expr *) node->resconstantqual, - (PlanState *) resstate); + resstate->ps.qual = + ExecInitQual(node->plan.qual, (PlanState *) resstate); + resstate->resconstantqual = + ExecInitQual((List *) node->resconstantqual, (PlanState *) resstate); /* * initialize child nodes @@ -295,7 +260,6 @@ void ExecReScanResult(ResultState *node) { node->rs_done = false; - node->ps.ps_TupFromTlist = false; node->rs_checkqual = (node->resconstantqual == NULL) ? false : true; /* diff --git a/src/backend/executor/nodeSamplescan.c b/src/backend/executor/nodeSamplescan.c index 9ce7c02aff..0247bd2347 100644 --- a/src/backend/executor/nodeSamplescan.c +++ b/src/backend/executor/nodeSamplescan.c @@ -3,7 +3,7 @@ * nodeSamplescan.c * Support routines for sample scans of relations (table sampling). * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -22,6 +22,7 @@ #include "miscadmin.h" #include "pgstat.h" #include "storage/predicate.h" +#include "utils/builtins.h" #include "utils/rel.h" #include "utils/tqual.h" @@ -163,19 +164,12 @@ ExecInitSampleScan(SampleScan *node, EState *estate, int eflags) /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); - - scanstate->args = (List *) - ExecInitExpr((Expr *) tsc->args, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); + + scanstate->args = ExecInitExprList(tsc->args, (PlanState *) scanstate); scanstate->repeatable = - ExecInitExpr(tsc->repeatable, - (PlanState *) scanstate); + ExecInitExpr(tsc->repeatable, (PlanState *) scanstate); /* * tuple table initialization @@ -188,8 +182,6 @@ ExecInitSampleScan(SampleScan *node, EState *estate, int eflags) */ InitScanRelation(scanstate, estate, eflags); - scanstate->ss.ps.ps_TupFromTlist = false; - /* * Initialize result tuple type and projection info. */ @@ -299,8 +291,7 @@ tablesample_init(SampleScanState *scanstate) params[i] = ExecEvalExprSwitchContext(argstate, econtext, - &isnull, - NULL); + &isnull); if (isnull) ereport(ERROR, (errcode(ERRCODE_INVALID_TABLESAMPLE_ARGUMENT), @@ -312,8 +303,7 @@ tablesample_init(SampleScanState *scanstate) { datum = ExecEvalExprSwitchContext(scanstate->repeatable, econtext, - &isnull, - NULL); + &isnull); if (isnull) ereport(ERROR, (errcode(ERRCODE_INVALID_TABLESAMPLE_REPEAT), diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c index 00bf3a58b1..5680464fa2 100644 --- a/src/backend/executor/nodeSeqscan.c +++ b/src/backend/executor/nodeSeqscan.c @@ -3,7 +3,7 @@ * nodeSeqscan.c * Support routines for sequential scans of relations. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -188,12 +188,8 @@ ExecInitSeqScan(SeqScan *node, EState *estate, int eflags) /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->plan.qual, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->plan.qual, (PlanState *) scanstate); /* * tuple table initialization @@ -206,8 +202,6 @@ ExecInitSeqScan(SeqScan *node, EState *estate, int eflags) */ InitScanRelation(scanstate, estate, eflags); - scanstate->ss.ps.ps_TupFromTlist = false; - /* * Initialize result tuple type and projection info. */ diff --git a/src/backend/executor/nodeSetOp.c b/src/backend/executor/nodeSetOp.c index 2d81d46927..9ae53bb8a7 100644 --- a/src/backend/executor/nodeSetOp.c +++ b/src/backend/executor/nodeSetOp.c @@ -32,7 +32,7 @@ * input group. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -64,20 +64,7 @@ typedef struct SetOpStatePerGroupData { long numLeft; /* number of left-input dups in group */ long numRight; /* number of right-input dups in group */ -} SetOpStatePerGroupData; - -/* - * To implement hashed mode, we need a hashtable that stores a - * representative tuple and the duplicate counts for each distinct set - * of grouping columns. We compute the hash key from the grouping columns. - */ -typedef struct SetOpHashEntryData *SetOpHashEntry; - -typedef struct SetOpHashEntryData -{ - TupleHashEntryData shared; /* common header for hash table entries */ - SetOpStatePerGroupData pergroup; -} SetOpHashEntryData; +} SetOpStatePerGroupData; static TupleTableSlot *setop_retrieve_direct(SetOpState *setopstate); @@ -141,9 +128,10 @@ build_hash_table(SetOpState *setopstate) setopstate->eqfunctions, setopstate->hashfunctions, node->numGroups, - sizeof(SetOpHashEntryData), + 0, setopstate->tableContext, - setopstate->tempContext); + setopstate->tempContext, + false); } /* @@ -238,7 +226,7 @@ setop_retrieve_direct(SetOpState *setopstate) * get state info from node */ outerPlan = outerPlanState(setopstate); - pergroup = setopstate->pergroup; + pergroup = (SetOpStatePerGroup) setopstate->pergroup; resultTupleSlot = setopstate->ps.ps_ResultTupleSlot; /* @@ -367,7 +355,7 @@ setop_fill_hash_table(SetOpState *setopstate) { TupleTableSlot *outerslot; int flag; - SetOpHashEntry entry; + TupleHashEntryData *entry; bool isnew; outerslot = ExecProcNode(outerPlan); @@ -383,15 +371,20 @@ setop_fill_hash_table(SetOpState *setopstate) Assert(in_first_rel); /* Find or build hashtable entry for this tuple's group */ - entry = (SetOpHashEntry) - LookupTupleHashEntry(setopstate->hashtable, outerslot, &isnew); + entry = LookupTupleHashEntry(setopstate->hashtable, outerslot, + &isnew); /* If new tuple group, initialize counts */ if (isnew) - initialize_counts(&entry->pergroup); + { + entry->additional = (SetOpStatePerGroup) + MemoryContextAlloc(setopstate->hashtable->tablecxt, + sizeof(SetOpStatePerGroupData)); + initialize_counts((SetOpStatePerGroup) entry->additional); + } /* Advance the counts */ - advance_counts(&entry->pergroup, flag); + advance_counts((SetOpStatePerGroup) entry->additional, flag); } else { @@ -399,12 +392,12 @@ setop_fill_hash_table(SetOpState *setopstate) in_first_rel = false; /* For tuples not seen previously, do not make hashtable entry */ - entry = (SetOpHashEntry) - LookupTupleHashEntry(setopstate->hashtable, outerslot, NULL); + entry = LookupTupleHashEntry(setopstate->hashtable, outerslot, + NULL); /* Advance the counts if entry is already present */ if (entry) - advance_counts(&entry->pergroup, flag); + advance_counts((SetOpStatePerGroup) entry->additional, flag); } /* Must reset temp context after each hashtable lookup */ @@ -422,7 +415,7 @@ setop_fill_hash_table(SetOpState *setopstate) static TupleTableSlot * setop_retrieve_hash_table(SetOpState *setopstate) { - SetOpHashEntry entry; + TupleHashEntryData *entry; TupleTableSlot *resultTupleSlot; /* @@ -438,7 +431,7 @@ setop_retrieve_hash_table(SetOpState *setopstate) /* * Find the next entry in the hash table */ - entry = (SetOpHashEntry) ScanTupleHashTable(&setopstate->hashiter); + entry = ScanTupleHashTable(setopstate->hashtable, &setopstate->hashiter); if (entry == NULL) { /* No more entries in hashtable, so done */ @@ -450,12 +443,12 @@ setop_retrieve_hash_table(SetOpState *setopstate) * See if we should emit any copies of this tuple, and if so return * the first copy. */ - set_output_count(setopstate, &entry->pergroup); + set_output_count(setopstate, (SetOpStatePerGroup) entry->additional); if (setopstate->numOutput > 0) { setopstate->numOutput--; - return ExecStoreMinimalTuple(entry->shared.firstTuple, + return ExecStoreMinimalTuple(entry->firstTuple, resultTupleSlot, false); } @@ -507,9 +500,7 @@ ExecInitSetOp(SetOp *node, EState *estate, int eflags) setopstate->tempContext = AllocSetContextCreate(CurrentMemoryContext, "SetOp", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* * If hashing, we also need a longer-lived context to store the hash @@ -520,9 +511,7 @@ ExecInitSetOp(SetOp *node, EState *estate, int eflags) setopstate->tableContext = AllocSetContextCreate(CurrentMemoryContext, "SetOp hash table", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* * Tuple table initialization diff --git a/src/backend/executor/nodeSort.c b/src/backend/executor/nodeSort.c index a34dcc5135..924b458df8 100644 --- a/src/backend/executor/nodeSort.c +++ b/src/backend/executor/nodeSort.c @@ -3,7 +3,7 @@ * nodeSort.c * Routines to handle sorting of relations. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -132,12 +132,13 @@ ExecSort(SortState *node) /* * Get the first or next tuple from tuplesort. Returns NULL if no more - * tuples. + * tuples. Note that we only rely on slot tuple remaining valid until the + * next fetch from the tuplesort. */ slot = node->ss.ps.ps_ResultTupleSlot; (void) tuplesort_gettupleslot(tuplesortstate, ScanDirectionIsForward(dir), - slot, NULL); + false, slot, NULL); return slot; } diff --git a/src/backend/executor/nodeSubplan.c b/src/backend/executor/nodeSubplan.c index fb7461c96c..85e5acf75a 100644 --- a/src/backend/executor/nodeSubplan.c +++ b/src/backend/executor/nodeSubplan.c @@ -14,7 +14,7 @@ * subplans, which are re-evaluated every time their result is required. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * IDENTIFICATION @@ -42,14 +42,6 @@ #include "utils/memutils.h" -static Datum ExecSubPlan(SubPlanState *node, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone); -static Datum ExecAlternativeSubPlan(AlternativeSubPlanState *node, - ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone); static Datum ExecHashSubPlan(SubPlanState *node, ExprContext *econtext, bool *isNull); @@ -69,18 +61,15 @@ static bool slotNoNulls(TupleTableSlot *slot); * This is the main entry point for execution of a regular SubPlan. * ---------------------------------------------------------------- */ -static Datum +Datum ExecSubPlan(SubPlanState *node, ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) + bool *isNull) { - SubPlan *subplan = (SubPlan *) node->xprstate.expr; + SubPlan *subplan = node->subplan; - /* Set default values for result flags: non-null, not a set result */ + /* Set non-null as default */ *isNull = false; - if (isDone) - *isDone = ExprSingleResult; /* Sanity checks */ if (subplan->subLinkType == CTE_SUBLINK) @@ -103,7 +92,7 @@ ExecHashSubPlan(SubPlanState *node, ExprContext *econtext, bool *isNull) { - SubPlan *subplan = (SubPlan *) node->xprstate.expr; + SubPlan *subplan = node->subplan; PlanState *planstate = node->planstate; TupleTableSlot *slot; @@ -131,7 +120,7 @@ ExecHashSubPlan(SubPlanState *node, * have to set the econtext to use (hack alert!). */ node->projLeft->pi_exprContext = econtext; - slot = ExecProject(node->projLeft, NULL); + slot = ExecProject(node->projLeft); /* * Note: because we are typically called in a per-tuple context, we have @@ -225,7 +214,7 @@ ExecScanSubPlan(SubPlanState *node, ExprContext *econtext, bool *isNull) { - SubPlan *subplan = (SubPlan *) node->xprstate.expr; + SubPlan *subplan = node->subplan; PlanState *planstate = node->planstate; SubLinkType subLinkType = subplan->subLinkType; MemoryContext oldcontext; @@ -288,8 +277,7 @@ ExecScanSubPlan(SubPlanState *node, prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar), econtext, - &(prm->isnull), - NULL); + &(prm->isnull)); prm->done = true; planstate->chgParam = bms_add_member(planstate->chgParam, paramid); } @@ -407,7 +395,7 @@ ExecScanSubPlan(SubPlanState *node, } rowresult = ExecEvalExprSwitchContext(node->testexpr, econtext, - &rownull, NULL); + &rownull); if (subLinkType == ANY_SUBLINK) { @@ -472,7 +460,7 @@ ExecScanSubPlan(SubPlanState *node, static void buildSubPlanHash(SubPlanState *node, ExprContext *econtext) { - SubPlan *subplan = (SubPlan *) node->xprstate.expr; + SubPlan *subplan = node->subplan; PlanState *planstate = node->planstate; int ncols = list_length(subplan->paramIds); ExprContext *innerecontext = node->innerecontext; @@ -512,9 +500,10 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext) node->tab_eq_funcs, node->tab_hash_funcs, nbuckets, - sizeof(TupleHashEntryData), + 0, node->hashtablecxt, - node->hashtempcxt); + node->hashtempcxt, + false); if (!subplan->unknownEqFalse) { @@ -531,9 +520,10 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext) node->tab_eq_funcs, node->tab_hash_funcs, nbuckets, - sizeof(TupleHashEntryData), + 0, node->hashtablecxt, - node->hashtempcxt); + node->hashtempcxt, + false); } /* @@ -574,7 +564,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext) &(prmdata->isnull)); col++; } - slot = ExecProject(node->projRight, NULL); + slot = ExecProject(node->projRight); /* * If result contains any nulls, store separately or not at all. @@ -604,7 +594,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext) * potential for a double free attempt. (XXX possibly no longer needed, * but can't hurt.) */ - ExecClearTuple(node->projRight->pi_slot); + ExecClearTuple(node->projRight->pi_state.resultslot); MemoryContextSwitchTo(oldcontext); } @@ -630,7 +620,7 @@ findPartialMatch(TupleHashTable hashtable, TupleTableSlot *slot, TupleHashEntry entry; InitTupleHashIterator(hashtable, &hashiter); - while ((entry = ScanTupleHashTable(&hashiter)) != NULL) + while ((entry = ScanTupleHashTable(hashtable, &hashiter)) != NULL) { ExecStoreMinimalTuple(entry->firstTuple, hashtable->tableslot, false); if (!execTuplesUnequal(slot, hashtable->tableslot, @@ -702,8 +692,7 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) SubPlanState *sstate = makeNode(SubPlanState); EState *estate = parent->state; - sstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecSubPlan; - sstate->xprstate.expr = (Expr *) subplan; + sstate->subplan = subplan; /* Link the SubPlanState to already-initialized subplan */ sstate->planstate = (PlanState *) list_nth(estate->es_subplanstates, @@ -719,7 +708,7 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) /* Initialize subexpressions */ sstate->testexpr = ExecInitExpr((Expr *) subplan->testexpr, parent); - sstate->args = (List *) ExecInitExpr((Expr *) subplan->args, parent); + sstate->args = ExecInitExprList(subplan->args, parent); /* * initialize my state @@ -776,25 +765,19 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) TupleTableSlot *slot; List *oplist, *lefttlist, - *righttlist, - *leftptlist, - *rightptlist; + *righttlist; ListCell *l; /* We need a memory context to hold the hash table(s) */ sstate->hashtablecxt = AllocSetContextCreate(CurrentMemoryContext, "Subplan HashTable Context", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* and a small one for the hash tables to use as temp storage */ sstate->hashtempcxt = AllocSetContextCreate(CurrentMemoryContext, "Subplan HashTable Temp Context", - ALLOCSET_SMALL_MINSIZE, - ALLOCSET_SMALL_INITSIZE, - ALLOCSET_SMALL_MAXSIZE); + ALLOCSET_SMALL_SIZES); /* and a short-lived exprcontext for function evaluation */ sstate->innerecontext = CreateExprContext(estate); /* Silly little array of column numbers 1..n */ @@ -809,36 +792,33 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) * use the sub-select's output tuples directly, but that is not the * case if we had to insert any run-time coercions of the sub-select's * output datatypes; anyway this avoids storing any resjunk columns - * that might be in the sub-select's output.) Run through the + * that might be in the sub-select's output.) Run through the * combining expressions to build tlists for the lefthand and - * righthand sides. We need both the ExprState list (for ExecProject) - * and the underlying parse Exprs (for ExecTypeFromTL). + * righthand sides. * * We also extract the combining operators themselves to initialize * the equality and hashing functions for the hash tables. */ - if (IsA(sstate->testexpr->expr, OpExpr)) + if (IsA(subplan->testexpr, OpExpr)) { /* single combining operator */ - oplist = list_make1(sstate->testexpr); + oplist = list_make1(subplan->testexpr); } - else if (and_clause((Node *) sstate->testexpr->expr)) + else if (and_clause((Node *) subplan->testexpr)) { /* multiple combining operators */ - Assert(IsA(sstate->testexpr, BoolExprState)); - oplist = ((BoolExprState *) sstate->testexpr)->args; + oplist = castNode(BoolExpr, subplan->testexpr)->args; } else { /* shouldn't see anything else in a hashable subplan */ elog(ERROR, "unrecognized testexpr type: %d", - (int) nodeTag(sstate->testexpr->expr)); + (int) nodeTag(subplan->testexpr)); oplist = NIL; /* keep compiler quiet */ } Assert(list_length(oplist) == ncols); lefttlist = righttlist = NIL; - leftptlist = rightptlist = NIL; sstate->tab_hash_funcs = (FmgrInfo *) palloc(ncols * sizeof(FmgrInfo)); sstate->tab_eq_funcs = (FmgrInfo *) palloc(ncols * sizeof(FmgrInfo)); sstate->lhs_hash_funcs = (FmgrInfo *) palloc(ncols * sizeof(FmgrInfo)); @@ -846,47 +826,30 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) i = 1; foreach(l, oplist) { - FuncExprState *fstate = (FuncExprState *) lfirst(l); - OpExpr *opexpr = (OpExpr *) fstate->xprstate.expr; - ExprState *exstate; + OpExpr *opexpr = lfirst_node(OpExpr, l); Expr *expr; TargetEntry *tle; - GenericExprState *tlestate; Oid rhs_eq_oper; Oid left_hashfn; Oid right_hashfn; - Assert(IsA(fstate, FuncExprState)); - Assert(IsA(opexpr, OpExpr)); - Assert(list_length(fstate->args) == 2); + Assert(list_length(opexpr->args) == 2); /* Process lefthand argument */ - exstate = (ExprState *) linitial(fstate->args); - expr = exstate->expr; + expr = (Expr *) linitial(opexpr->args); tle = makeTargetEntry(expr, i, NULL, false); - tlestate = makeNode(GenericExprState); - tlestate->xprstate.expr = (Expr *) tle; - tlestate->xprstate.evalfunc = NULL; - tlestate->arg = exstate; - lefttlist = lappend(lefttlist, tlestate); - leftptlist = lappend(leftptlist, tle); + lefttlist = lappend(lefttlist, tle); /* Process righthand argument */ - exstate = (ExprState *) lsecond(fstate->args); - expr = exstate->expr; + expr = (Expr *) lsecond(opexpr->args); tle = makeTargetEntry(expr, i, NULL, false); - tlestate = makeNode(GenericExprState); - tlestate->xprstate.expr = (Expr *) tle; - tlestate->xprstate.evalfunc = NULL; - tlestate->arg = exstate; - righttlist = lappend(righttlist, tlestate); - rightptlist = lappend(rightptlist, tle); + righttlist = lappend(righttlist, tle); /* Lookup the equality function (potentially cross-type) */ fmgr_info(opexpr->opfuncid, &sstate->cur_eq_funcs[i - 1]); @@ -918,20 +881,22 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) * (hack alert!). The righthand expressions will be evaluated in our * own innerecontext. */ - tupDesc = ExecTypeFromTL(leftptlist, false); + tupDesc = ExecTypeFromTL(lefttlist, false); slot = ExecInitExtraTupleSlot(estate); ExecSetSlotDescriptor(slot, tupDesc); sstate->projLeft = ExecBuildProjectionInfo(lefttlist, NULL, slot, + parent, NULL); - tupDesc = ExecTypeFromTL(rightptlist, false); + tupDesc = ExecTypeFromTL(righttlist, false); slot = ExecInitExtraTupleSlot(estate); ExecSetSlotDescriptor(slot, tupDesc); sstate->projRight = ExecBuildProjectionInfo(righttlist, sstate->innerecontext, slot, + sstate->planstate, NULL); } @@ -954,7 +919,7 @@ ExecInitSubPlan(SubPlan *subplan, PlanState *parent) void ExecSetParamPlan(SubPlanState *node, ExprContext *econtext) { - SubPlan *subplan = (SubPlan *) node->xprstate.expr; + SubPlan *subplan = node->subplan; PlanState *planstate = node->planstate; SubLinkType subLinkType = subplan->subLinkType; MemoryContext oldcontext; @@ -996,8 +961,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext) prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar), econtext, - &(prm->isnull), - NULL); + &(prm->isnull)); prm->done = true; planstate->chgParam = bms_add_member(planstate->chgParam, paramid); } @@ -1138,7 +1102,7 @@ void ExecReScanSetParamPlan(SubPlanState *node, PlanState *parent) { PlanState *planstate = node->planstate; - SubPlan *subplan = (SubPlan *) node->xprstate.expr; + SubPlan *subplan = node->subplan; EState *estate = parent->state; ListCell *l; @@ -1189,16 +1153,22 @@ ExecInitAlternativeSubPlan(AlternativeSubPlan *asplan, PlanState *parent) SubPlan *subplan2; Cost cost1; Cost cost2; + ListCell *lc; - asstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecAlternativeSubPlan; - asstate->xprstate.expr = (Expr *) asplan; + asstate->subplan = asplan; /* * Initialize subplans. (Can we get away with only initializing the one * we're going to use?) */ - asstate->subplans = (List *) ExecInitExpr((Expr *) asplan->subplans, - parent); + foreach(lc, asplan->subplans) + { + SubPlan *sp = lfirst_node(SubPlan, lc); + SubPlanState *sps = ExecInitSubPlan(sp, parent); + + asstate->subplans = lappend(asstate->subplans, sps); + parent->subPlan = lappend(parent->subPlan, sps); + } /* * Select the one to be used. For this, we need an estimate of the number @@ -1236,20 +1206,14 @@ ExecInitAlternativeSubPlan(AlternativeSubPlan *asplan, PlanState *parent) * Note: in future we might consider changing to different subplans on the * fly, in case the original rowcount estimate turns out to be way off. */ -static Datum +Datum ExecAlternativeSubPlan(AlternativeSubPlanState *node, ExprContext *econtext, - bool *isNull, - ExprDoneCond *isDone) + bool *isNull) { /* Just pass control to the active subplan */ - SubPlanState *activesp = (SubPlanState *) list_nth(node->subplans, - node->active); - - Assert(IsA(activesp, SubPlanState)); + SubPlanState *activesp = list_nth_node(SubPlanState, + node->subplans, node->active); - return ExecSubPlan(activesp, - econtext, - isNull, - isDone); + return ExecSubPlan(activesp, econtext, isNull); } diff --git a/src/backend/executor/nodeSubqueryscan.c b/src/backend/executor/nodeSubqueryscan.c index 9bafc62677..ae184700a6 100644 --- a/src/backend/executor/nodeSubqueryscan.c +++ b/src/backend/executor/nodeSubqueryscan.c @@ -7,7 +7,7 @@ * we need two sets of code. Ought to look at trying to unify the cases. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -120,12 +120,8 @@ ExecInitSubqueryScan(SubqueryScan *node, EState *estate, int eflags) /* * initialize child expressions */ - subquerystate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) subquerystate); - subquerystate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) subquerystate); + subquerystate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) subquerystate); /* * tuple table initialization @@ -138,8 +134,6 @@ ExecInitSubqueryScan(SubqueryScan *node, EState *estate, int eflags) */ subquerystate->subplan = ExecInitNode(node->subplan, estate, eflags); - subquerystate->ss.ps.ps_TupFromTlist = false; - /* * Initialize scan tuple type (needed by ExecAssignScanProjectionInfo) */ diff --git a/src/backend/executor/nodeTableFuncscan.c b/src/backend/executor/nodeTableFuncscan.c new file mode 100644 index 0000000000..da557ceb6f --- /dev/null +++ b/src/backend/executor/nodeTableFuncscan.c @@ -0,0 +1,498 @@ +/*------------------------------------------------------------------------- + * + * nodeTableFuncscan.c + * Support routines for scanning RangeTableFunc (XMLTABLE like functions). + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * + * IDENTIFICATION + * src/backend/executor/nodeTableFuncscan.c + * + *------------------------------------------------------------------------- + */ +/* + * INTERFACE ROUTINES + * ExecTableFuncscan scans a function. + * ExecFunctionNext retrieve next tuple in sequential order. + * ExecInitTableFuncscan creates and initializes a TableFuncscan node. + * ExecEndTableFuncscan releases any storage allocated. + * ExecReScanTableFuncscan rescans the function + */ +#include "postgres.h" + +#include "nodes/execnodes.h" +#include "executor/executor.h" +#include "executor/nodeTableFuncscan.h" +#include "executor/tablefunc.h" +#include "miscadmin.h" +#include "utils/builtins.h" +#include "utils/lsyscache.h" +#include "utils/memutils.h" +#include "utils/xml.h" + + +static TupleTableSlot *TableFuncNext(TableFuncScanState *node); +static bool TableFuncRecheck(TableFuncScanState *node, TupleTableSlot *slot); + +static void tfuncFetchRows(TableFuncScanState *tstate, ExprContext *econtext); +static void tfuncInitialize(TableFuncScanState *tstate, ExprContext *econtext, Datum doc); +static void tfuncLoadRows(TableFuncScanState *tstate, ExprContext *econtext); + +/* ---------------------------------------------------------------- + * Scan Support + * ---------------------------------------------------------------- + */ +/* ---------------------------------------------------------------- + * TableFuncNext + * + * This is a workhorse for ExecTableFuncscan + * ---------------------------------------------------------------- + */ +static TupleTableSlot * +TableFuncNext(TableFuncScanState *node) +{ + TupleTableSlot *scanslot; + + scanslot = node->ss.ss_ScanTupleSlot; + + /* + * If first time through, read all tuples from function and put them in a + * tuplestore. Subsequent calls just fetch tuples from tuplestore. + */ + if (node->tupstore == NULL) + tfuncFetchRows(node, node->ss.ps.ps_ExprContext); + + /* + * Get the next tuple from tuplestore. + */ + (void) tuplestore_gettupleslot(node->tupstore, + true, + false, + scanslot); + return scanslot; +} + +/* + * TableFuncRecheck -- access method routine to recheck a tuple in EvalPlanQual + */ +static bool +TableFuncRecheck(TableFuncScanState *node, TupleTableSlot *slot) +{ + /* nothing to check */ + return true; +} + +/* ---------------------------------------------------------------- + * ExecTableFuncscan(node) + * + * Scans the function sequentially and returns the next qualifying + * tuple. + * We call the ExecScan() routine and pass it the appropriate + * access method functions. + * ---------------------------------------------------------------- + */ +TupleTableSlot * +ExecTableFuncScan(TableFuncScanState *node) +{ + return ExecScan(&node->ss, + (ExecScanAccessMtd) TableFuncNext, + (ExecScanRecheckMtd) TableFuncRecheck); +} + +/* ---------------------------------------------------------------- + * ExecInitTableFuncscan + * ---------------------------------------------------------------- + */ +TableFuncScanState * +ExecInitTableFuncScan(TableFuncScan *node, EState *estate, int eflags) +{ + TableFuncScanState *scanstate; + TableFunc *tf = node->tablefunc; + TupleDesc tupdesc; + int i; + + /* check for unsupported flags */ + Assert(!(eflags & EXEC_FLAG_MARK)); + + /* + * TableFuncscan should not have any children. + */ + Assert(outerPlan(node) == NULL); + Assert(innerPlan(node) == NULL); + + /* + * create new ScanState for node + */ + scanstate = makeNode(TableFuncScanState); + scanstate->ss.ps.plan = (Plan *) node; + scanstate->ss.ps.state = estate; + + /* + * Miscellaneous initialization + * + * create expression context for node + */ + ExecAssignExprContext(estate, &scanstate->ss.ps); + + /* + * initialize child expressions + */ + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, &scanstate->ss.ps); + + /* + * tuple table initialization + */ + ExecInitResultTupleSlot(estate, &scanstate->ss.ps); + ExecInitScanTupleSlot(estate, &scanstate->ss); + + /* + * initialize source tuple type + */ + tupdesc = BuildDescFromLists(tf->colnames, + tf->coltypes, + tf->coltypmods, + tf->colcollations); + + ExecAssignScanType(&scanstate->ss, tupdesc); + + /* + * Initialize result tuple type and projection info. + */ + ExecAssignResultTypeFromTL(&scanstate->ss.ps); + ExecAssignScanProjectionInfo(&scanstate->ss); + + /* Only XMLTABLE is supported currently */ + scanstate->routine = &XmlTableRoutine; + + scanstate->perValueCxt = + AllocSetContextCreate(CurrentMemoryContext, + "TableFunc per value context", + ALLOCSET_DEFAULT_SIZES); + scanstate->opaque = NULL; /* initialized at runtime */ + + scanstate->ns_names = tf->ns_names; + + scanstate->ns_uris = + ExecInitExprList(tf->ns_uris, (PlanState *) scanstate); + scanstate->docexpr = + ExecInitExpr((Expr *) tf->docexpr, (PlanState *) scanstate); + scanstate->rowexpr = + ExecInitExpr((Expr *) tf->rowexpr, (PlanState *) scanstate); + scanstate->colexprs = + ExecInitExprList(tf->colexprs, (PlanState *) scanstate); + scanstate->coldefexprs = + ExecInitExprList(tf->coldefexprs, (PlanState *) scanstate); + + scanstate->notnulls = tf->notnulls; + + /* these are allocated now and initialized later */ + scanstate->in_functions = palloc(sizeof(FmgrInfo) * tupdesc->natts); + scanstate->typioparams = palloc(sizeof(Oid) * tupdesc->natts); + + /* + * Fill in the necessary fmgr infos. + */ + for (i = 0; i < tupdesc->natts; i++) + { + Oid in_funcid; + + getTypeInputInfo(tupdesc->attrs[i]->atttypid, + &in_funcid, &scanstate->typioparams[i]); + fmgr_info(in_funcid, &scanstate->in_functions[i]); + } + + return scanstate; +} + +/* ---------------------------------------------------------------- + * ExecEndTableFuncscan + * + * frees any storage allocated through C routines. + * ---------------------------------------------------------------- + */ +void +ExecEndTableFuncScan(TableFuncScanState *node) +{ + /* + * Free the exprcontext + */ + ExecFreeExprContext(&node->ss.ps); + + /* + * clean out the tuple table + */ + ExecClearTuple(node->ss.ps.ps_ResultTupleSlot); + ExecClearTuple(node->ss.ss_ScanTupleSlot); + + /* + * Release tuplestore resources + */ + if (node->tupstore != NULL) + tuplestore_end(node->tupstore); + node->tupstore = NULL; +} + +/* ---------------------------------------------------------------- + * ExecReScanTableFuncscan + * + * Rescans the relation. + * ---------------------------------------------------------------- + */ +void +ExecReScanTableFuncScan(TableFuncScanState *node) +{ + Bitmapset *chgparam = node->ss.ps.chgParam; + + ExecClearTuple(node->ss.ps.ps_ResultTupleSlot); + ExecScanReScan(&node->ss); + + /* + * Recompute when parameters are changed. + */ + if (chgparam) + { + if (node->tupstore != NULL) + { + tuplestore_end(node->tupstore); + node->tupstore = NULL; + } + } + + if (node->tupstore != NULL) + tuplestore_rescan(node->tupstore); +} + +/* ---------------------------------------------------------------- + * tfuncFetchRows + * + * Read rows from a TableFunc producer + * ---------------------------------------------------------------- + */ +static void +tfuncFetchRows(TableFuncScanState *tstate, ExprContext *econtext) +{ + const TableFuncRoutine *routine = tstate->routine; + MemoryContext oldcxt; + Datum value; + bool isnull; + + Assert(tstate->opaque == NULL); + + /* build tuplestore for the result */ + oldcxt = MemoryContextSwitchTo(econtext->ecxt_per_query_memory); + tstate->tupstore = tuplestore_begin_heap(false, false, work_mem); + + PG_TRY(); + { + routine->InitOpaque(tstate, + tstate->ss.ss_ScanTupleSlot->tts_tupleDescriptor->natts); + + /* + * If evaluating the document expression returns NULL, the table + * expression is empty and we return immediately. + */ + value = ExecEvalExpr(tstate->docexpr, econtext, &isnull); + + if (!isnull) + { + /* otherwise, pass the document value to the table builder */ + tfuncInitialize(tstate, econtext, value); + + /* initialize ordinality counter */ + tstate->ordinal = 1; + + /* Load all rows into the tuplestore, and we're done */ + tfuncLoadRows(tstate, econtext); + } + } + PG_CATCH(); + { + if (tstate->opaque != NULL) + routine->DestroyOpaque(tstate); + PG_RE_THROW(); + } + PG_END_TRY(); + + /* return to original memory context, and clean up */ + MemoryContextSwitchTo(oldcxt); + + if (tstate->opaque != NULL) + { + routine->DestroyOpaque(tstate); + tstate->opaque = NULL; + } + + return; +} + +/* + * Fill in namespace declarations, the row filter, and column filters in a + * table expression builder context. + */ +static void +tfuncInitialize(TableFuncScanState *tstate, ExprContext *econtext, Datum doc) +{ + const TableFuncRoutine *routine = tstate->routine; + TupleDesc tupdesc; + ListCell *lc1, + *lc2; + bool isnull; + int colno; + Datum value; + int ordinalitycol = + ((TableFuncScan *) (tstate->ss.ps.plan))->tablefunc->ordinalitycol; + + /* + * Install the document as a possibly-toasted Datum into the tablefunc + * context. + */ + routine->SetDocument(tstate, doc); + + /* Evaluate namespace specifications */ + forboth(lc1, tstate->ns_uris, lc2, tstate->ns_names) + { + ExprState *expr = (ExprState *) lfirst(lc1); + char *ns_name = strVal(lfirst(lc2)); + char *ns_uri; + + value = ExecEvalExpr((ExprState *) expr, econtext, &isnull); + if (isnull) + ereport(ERROR, + (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("namespace URI must not be null"))); + ns_uri = TextDatumGetCString(value); + + routine->SetNamespace(tstate, ns_name, ns_uri); + } + + /* Install the row filter expression into the table builder context */ + value = ExecEvalExpr(tstate->rowexpr, econtext, &isnull); + if (isnull) + ereport(ERROR, + (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("row filter expression must not be null"))); + + routine->SetRowFilter(tstate, TextDatumGetCString(value)); + + /* + * Install the column filter expressions into the table builder context. + * If an expression is given, use that; otherwise the column name itself + * is the column filter. + */ + colno = 0; + tupdesc = tstate->ss.ss_ScanTupleSlot->tts_tupleDescriptor; + foreach(lc1, tstate->colexprs) + { + char *colfilter; + + if (colno != ordinalitycol) + { + ExprState *colexpr = lfirst(lc1); + + if (colexpr != NULL) + { + value = ExecEvalExpr(colexpr, econtext, &isnull); + if (isnull) + ereport(ERROR, + (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("column filter expression must not be null"), + errdetail("Filter for column \"%s\" is null.", + NameStr(tupdesc->attrs[colno]->attname)))); + colfilter = TextDatumGetCString(value); + } + else + colfilter = NameStr(tupdesc->attrs[colno]->attname); + + routine->SetColumnFilter(tstate, colfilter, colno); + } + + colno++; + } +} + +/* + * Load all the rows from the TableFunc table builder into a tuplestore. + */ +static void +tfuncLoadRows(TableFuncScanState *tstate, ExprContext *econtext) +{ + const TableFuncRoutine *routine = tstate->routine; + TupleTableSlot *slot = tstate->ss.ss_ScanTupleSlot; + TupleDesc tupdesc = slot->tts_tupleDescriptor; + Datum *values = slot->tts_values; + bool *nulls = slot->tts_isnull; + int natts = tupdesc->natts; + MemoryContext oldcxt; + int ordinalitycol; + + ordinalitycol = + ((TableFuncScan *) (tstate->ss.ps.plan))->tablefunc->ordinalitycol; + oldcxt = MemoryContextSwitchTo(tstate->perValueCxt); + + /* + * Keep requesting rows from the table builder until there aren't any. + */ + while (routine->FetchRow(tstate)) + { + ListCell *cell = list_head(tstate->coldefexprs); + int colno; + + ExecClearTuple(tstate->ss.ss_ScanTupleSlot); + + /* + * Obtain the value of each column for this row, installing them into + * the slot; then add the tuple to the tuplestore. + */ + for (colno = 0; colno < natts; colno++) + { + if (colno == ordinalitycol) + { + /* Fast path for ordinality column */ + values[colno] = Int32GetDatum(tstate->ordinal++); + nulls[colno] = false; + } + else + { + bool isnull; + + values[colno] = routine->GetValue(tstate, + colno, + tupdesc->attrs[colno]->atttypid, + tupdesc->attrs[colno]->atttypmod, + &isnull); + + /* No value? Evaluate and apply the default, if any */ + if (isnull && cell != NULL) + { + ExprState *coldefexpr = (ExprState *) lfirst(cell); + + if (coldefexpr != NULL) + values[colno] = ExecEvalExpr(coldefexpr, econtext, + &isnull); + } + + /* Verify a possible NOT NULL constraint */ + if (isnull && bms_is_member(colno, tstate->notnulls)) + ereport(ERROR, + (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("null is not allowed in column \"%s\"", + NameStr(tupdesc->attrs[colno]->attname)))); + + nulls[colno] = isnull; + } + + /* advance list of default expressions */ + if (cell != NULL) + cell = lnext(cell); + } + + tuplestore_putvalues(tstate->tupstore, tupdesc, values, nulls); + + MemoryContextReset(tstate->perValueCxt); + } + + MemoryContextSwitchTo(oldcxt); +} diff --git a/src/backend/executor/nodeTidscan.c b/src/backend/executor/nodeTidscan.c index 2604103352..4860ec0f4d 100644 --- a/src/backend/executor/nodeTidscan.c +++ b/src/backend/executor/nodeTidscan.c @@ -3,7 +3,7 @@ * nodeTidscan.c * Routines to support direct tid scans of relations * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -38,21 +38,94 @@ ((Var *) (node))->varattno == SelfItemPointerAttributeNumber && \ ((Var *) (node))->varlevelsup == 0) -static void TidListCreate(TidScanState *tidstate); +/* one element in tss_tidexprs */ +typedef struct TidExpr +{ + ExprState *exprstate; /* ExprState for a TID-yielding subexpr */ + bool isarray; /* if true, it yields tid[] not just tid */ + CurrentOfExpr *cexpr; /* alternatively, we can have CURRENT OF */ +} TidExpr; + +static void TidExprListCreate(TidScanState *tidstate); +static void TidListEval(TidScanState *tidstate); static int itemptr_comparator(const void *a, const void *b); static TupleTableSlot *TidNext(TidScanState *node); /* + * Extract the qual subexpressions that yield TIDs to search for, + * and compile them into ExprStates if they're ordinary expressions. + * + * CURRENT OF is a special case that we can't compile usefully; + * just drop it into the TidExpr list as-is. + */ +static void +TidExprListCreate(TidScanState *tidstate) +{ + TidScan *node = (TidScan *) tidstate->ss.ps.plan; + ListCell *l; + + tidstate->tss_tidexprs = NIL; + tidstate->tss_isCurrentOf = false; + + foreach(l, node->tidquals) + { + Expr *expr = (Expr *) lfirst(l); + TidExpr *tidexpr = (TidExpr *) palloc0(sizeof(TidExpr)); + + if (is_opclause(expr)) + { + Node *arg1; + Node *arg2; + + arg1 = get_leftop(expr); + arg2 = get_rightop(expr); + if (IsCTIDVar(arg1)) + tidexpr->exprstate = ExecInitExpr((Expr *) arg2, + &tidstate->ss.ps); + else if (IsCTIDVar(arg2)) + tidexpr->exprstate = ExecInitExpr((Expr *) arg1, + &tidstate->ss.ps); + else + elog(ERROR, "could not identify CTID variable"); + tidexpr->isarray = false; + } + else if (expr && IsA(expr, ScalarArrayOpExpr)) + { + ScalarArrayOpExpr *saex = (ScalarArrayOpExpr *) expr; + + Assert(IsCTIDVar(linitial(saex->args))); + tidexpr->exprstate = ExecInitExpr(lsecond(saex->args), + &tidstate->ss.ps); + tidexpr->isarray = true; + } + else if (expr && IsA(expr, CurrentOfExpr)) + { + CurrentOfExpr *cexpr = (CurrentOfExpr *) expr; + + tidexpr->cexpr = cexpr; + tidstate->tss_isCurrentOf = true; + } + else + elog(ERROR, "could not identify CTID expression"); + + tidstate->tss_tidexprs = lappend(tidstate->tss_tidexprs, tidexpr); + } + + /* CurrentOfExpr could never appear OR'd with something else */ + Assert(list_length(tidstate->tss_tidexprs) == 1 || + !tidstate->tss_isCurrentOf); +} + +/* * Compute the list of TIDs to be visited, by evaluating the expressions * for them. * * (The result is actually an array, not a list.) */ static void -TidListCreate(TidScanState *tidstate) +TidListEval(TidScanState *tidstate) { - List *evalList = tidstate->tss_tidquals; ExprContext *econtext = tidstate->ss.ps.ps_ExprContext; BlockNumber nblocks; ItemPointerData *tidList; @@ -73,39 +146,23 @@ TidListCreate(TidScanState *tidstate) * are simple OpExprs or CurrentOfExprs. If there are any * ScalarArrayOpExprs, we may have to enlarge the array. */ - numAllocTids = list_length(evalList); + numAllocTids = list_length(tidstate->tss_tidexprs); tidList = (ItemPointerData *) palloc(numAllocTids * sizeof(ItemPointerData)); numTids = 0; - tidstate->tss_isCurrentOf = false; - foreach(l, evalList) + foreach(l, tidstate->tss_tidexprs) { - ExprState *exstate = (ExprState *) lfirst(l); - Expr *expr = exstate->expr; + TidExpr *tidexpr = (TidExpr *) lfirst(l); ItemPointer itemptr; bool isNull; - if (is_opclause(expr)) + if (tidexpr->exprstate && !tidexpr->isarray) { - FuncExprState *fexstate = (FuncExprState *) exstate; - Node *arg1; - Node *arg2; - - arg1 = get_leftop(expr); - arg2 = get_rightop(expr); - if (IsCTIDVar(arg1)) - exstate = (ExprState *) lsecond(fexstate->args); - else if (IsCTIDVar(arg2)) - exstate = (ExprState *) linitial(fexstate->args); - else - elog(ERROR, "could not identify CTID variable"); - itemptr = (ItemPointer) - DatumGetPointer(ExecEvalExprSwitchContext(exstate, + DatumGetPointer(ExecEvalExprSwitchContext(tidexpr->exprstate, econtext, - &isNull, - NULL)); + &isNull)); if (!isNull && ItemPointerIsValid(itemptr) && ItemPointerGetBlockNumber(itemptr) < nblocks) @@ -120,9 +177,8 @@ TidListCreate(TidScanState *tidstate) tidList[numTids++] = *itemptr; } } - else if (expr && IsA(expr, ScalarArrayOpExpr)) + else if (tidexpr->exprstate && tidexpr->isarray) { - ScalarArrayOpExprState *saexstate = (ScalarArrayOpExprState *) exstate; Datum arraydatum; ArrayType *itemarray; Datum *ipdatums; @@ -130,16 +186,14 @@ TidListCreate(TidScanState *tidstate) int ndatums; int i; - exstate = (ExprState *) lsecond(saexstate->fxprstate.args); - arraydatum = ExecEvalExprSwitchContext(exstate, + arraydatum = ExecEvalExprSwitchContext(tidexpr->exprstate, econtext, - &isNull, - NULL); + &isNull); if (isNull) continue; itemarray = DatumGetArrayTypeP(arraydatum); deconstruct_array(itemarray, - TIDOID, SizeOfIptrData, false, 's', + TIDOID, sizeof(ItemPointerData), false, 's', &ipdatums, &ipnulls, &ndatums); if (numTids + ndatums > numAllocTids) { @@ -161,12 +215,12 @@ TidListCreate(TidScanState *tidstate) pfree(ipdatums); pfree(ipnulls); } - else if (expr && IsA(expr, CurrentOfExpr)) + else { - CurrentOfExpr *cexpr = (CurrentOfExpr *) expr; ItemPointerData cursor_tid; - if (execCurrentOf(cexpr, econtext, + Assert(tidexpr->cexpr); + if (execCurrentOf(tidexpr->cexpr, econtext, RelationGetRelid(tidstate->ss.ss_currentRelation), &cursor_tid)) { @@ -178,11 +232,8 @@ TidListCreate(TidScanState *tidstate) numAllocTids * sizeof(ItemPointerData)); } tidList[numTids++] = cursor_tid; - tidstate->tss_isCurrentOf = true; } } - else - elog(ERROR, "could not identify CTID expression"); } /* @@ -274,11 +325,15 @@ TidNext(TidScanState *node) * First time through, compute the list of TIDs to be visited */ if (node->tss_TidList == NULL) - TidListCreate(node); + TidListEval(node); tidList = node->tss_TidList; numTids = node->tss_NumTids; + /* + * We use node->tss_htup as the tuple pointer; note this can't just be a + * local variable here, as the scan tuple slot will keep a pointer to it. + */ tuple = &(node->tss_htup); /* @@ -469,21 +524,13 @@ ExecInitTidScan(TidScan *node, EState *estate, int eflags) */ ExecAssignExprContext(estate, &tidstate->ss.ps); - tidstate->ss.ps.ps_TupFromTlist = false; - /* * initialize child expressions */ - tidstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) tidstate); - tidstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) tidstate); - - tidstate->tss_tidquals = (List *) - ExecInitExpr((Expr *) node->tidquals, - (PlanState *) tidstate); + tidstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) tidstate); + + TidExprListCreate(tidstate); /* * tuple table initialization diff --git a/src/backend/executor/nodeUnique.c b/src/backend/executor/nodeUnique.c index 4caae34b97..28cc1e90f8 100644 --- a/src/backend/executor/nodeUnique.c +++ b/src/backend/executor/nodeUnique.c @@ -11,7 +11,7 @@ * (It's debatable whether the savings justifies carrying two plan node * types, though.) * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -133,9 +133,7 @@ ExecInitUnique(Unique *node, EState *estate, int eflags) uniquestate->tempContext = AllocSetContextCreate(CurrentMemoryContext, "Unique", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* * Tuple table initialization diff --git a/src/backend/executor/nodeValuesscan.c b/src/backend/executor/nodeValuesscan.c index 9c03f8ae16..9ee776c4c3 100644 --- a/src/backend/executor/nodeValuesscan.c +++ b/src/backend/executor/nodeValuesscan.c @@ -4,7 +4,7 @@ * Support routines for scanning Values lists * ("VALUES (...), (...), ..." in rangetable). * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -120,7 +120,7 @@ ValuesNext(ValuesScanState *node) * is a SubPlan, and there shouldn't be any (any subselects in the * VALUES list should be InitPlans). */ - exprstatelist = (List *) ExecInitExpr((Expr *) exprlist, NULL); + exprstatelist = ExecInitExprList(exprlist, NULL); /* parser should have checked all sublists are the same length */ Assert(list_length(exprstatelist) == slot->tts_tupleDescriptor->natts); @@ -140,8 +140,7 @@ ValuesNext(ValuesScanState *node) values[resind] = ExecEvalExpr(estate, econtext, - &isnull[resind], - NULL); + &isnull[resind]); /* * We must force any R/W expanded datums to read-only state, in @@ -243,12 +242,8 @@ ExecInitValuesScan(ValuesScan *node, EState *estate, int eflags) /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); /* * get info about values list @@ -272,8 +267,6 @@ ExecInitValuesScan(ValuesScan *node, EState *estate, int eflags) scanstate->exprlists[i++] = (List *) lfirst(vtl); } - scanstate->ss.ps.ps_TupFromTlist = false; - /* * Initialize result tuple type and projection info. */ diff --git a/src/backend/executor/nodeWindowAgg.c b/src/backend/executor/nodeWindowAgg.c index 0d512543d9..124db68b1f 100644 --- a/src/backend/executor/nodeWindowAgg.c +++ b/src/backend/executor/nodeWindowAgg.c @@ -24,7 +24,7 @@ * * * Portions Copyright (c) 2012-2014, TransLattice, Inc. - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * IDENTIFICATION @@ -257,7 +257,7 @@ advance_windowaggregate(WindowAggState *winstate, if (filter) { bool isnull; - Datum res = ExecEvalExpr(filter, econtext, &isnull, NULL); + Datum res = ExecEvalExpr(filter, econtext, &isnull); if (isnull || !DatumGetBool(res)) { @@ -273,7 +273,7 @@ advance_windowaggregate(WindowAggState *winstate, ExprState *argstate = (ExprState *) lfirst(arg); fcinfo->arg[i] = ExecEvalExpr(argstate, econtext, - &fcinfo->argnull[i], NULL); + &fcinfo->argnull[i]); i++; } @@ -355,7 +355,7 @@ advance_windowaggregate(WindowAggState *winstate, /* * We must track the number of rows included in transValue, since to - * remove the last input, advance_windowaggregate_base() musn't call the + * remove the last input, advance_windowaggregate_base() mustn't call the * inverse transition function, but simply reset transValue back to its * initial value. */ @@ -363,8 +363,10 @@ advance_windowaggregate(WindowAggState *winstate, /* * If pass-by-ref datatype, must copy the new value into aggcontext and - * pfree the prior transValue. But if transfn returned a pointer to its - * first input, we don't need to do anything. + * free the prior transValue. But if transfn returned a pointer to its + * first input, we don't need to do anything. Also, if transfn returned a + * pointer to a R/W expanded object that is already a child of the + * aggcontext, assume we can adopt that value without copying it. */ if (!peraggstate->transtypeByVal && DatumGetPointer(newVal) != DatumGetPointer(peraggstate->transValue)) @@ -372,12 +374,25 @@ advance_windowaggregate(WindowAggState *winstate, if (!fcinfo->isnull) { MemoryContextSwitchTo(peraggstate->aggcontext); - newVal = datumCopy(newVal, - peraggstate->transtypeByVal, - peraggstate->transtypeLen); + if (DatumIsReadWriteExpandedObject(newVal, + false, + peraggstate->transtypeLen) && + MemoryContextGetParent(DatumGetEOHP(newVal)->eoh_context) == CurrentMemoryContext) + /* do nothing */ ; + else + newVal = datumCopy(newVal, + peraggstate->transtypeByVal, + peraggstate->transtypeLen); } if (!peraggstate->transValueIsNull) - pfree(DatumGetPointer(peraggstate->transValue)); + { + if (DatumIsReadWriteExpandedObject(peraggstate->transValue, + false, + peraggstate->transtypeLen)) + DeleteExpandedObject(peraggstate->transValue); + else + pfree(DatumGetPointer(peraggstate->transValue)); + } } MemoryContextSwitchTo(oldContext); @@ -419,7 +434,7 @@ advance_windowaggregate_base(WindowAggState *winstate, if (filter) { bool isnull; - Datum res = ExecEvalExpr(filter, econtext, &isnull, NULL); + Datum res = ExecEvalExpr(filter, econtext, &isnull); if (isnull || !DatumGetBool(res)) { @@ -435,7 +450,7 @@ advance_windowaggregate_base(WindowAggState *winstate, ExprState *argstate = (ExprState *) lfirst(arg); fcinfo->arg[i] = ExecEvalExpr(argstate, econtext, - &fcinfo->argnull[i], NULL); + &fcinfo->argnull[i]); i++; } @@ -514,8 +529,10 @@ advance_windowaggregate_base(WindowAggState *winstate, /* * If pass-by-ref datatype, must copy the new value into aggcontext and - * pfree the prior transValue. But if invtransfn returned a pointer to - * its first input, we don't need to do anything. + * free the prior transValue. But if invtransfn returned a pointer to its + * first input, we don't need to do anything. Also, if invtransfn + * returned a pointer to a R/W expanded object that is already a child of + * the aggcontext, assume we can adopt that value without copying it. * * Note: the checks for null values here will never fire, but it seems * best to have this stanza look just like advance_windowaggregate. @@ -526,12 +543,25 @@ advance_windowaggregate_base(WindowAggState *winstate, if (!fcinfo->isnull) { MemoryContextSwitchTo(peraggstate->aggcontext); - newVal = datumCopy(newVal, - peraggstate->transtypeByVal, - peraggstate->transtypeLen); + if (DatumIsReadWriteExpandedObject(newVal, + false, + peraggstate->transtypeLen) && + MemoryContextGetParent(DatumGetEOHP(newVal)->eoh_context) == CurrentMemoryContext) + /* do nothing */ ; + else + newVal = datumCopy(newVal, + peraggstate->transtypeByVal, + peraggstate->transtypeLen); } if (!peraggstate->transValueIsNull) - pfree(DatumGetPointer(peraggstate->transValue)); + { + if (DatumIsReadWriteExpandedObject(peraggstate->transValue, + false, + peraggstate->transtypeLen)) + DeleteExpandedObject(peraggstate->transValue); + else + pfree(DatumGetPointer(peraggstate->transValue)); + } } MemoryContextSwitchTo(oldContext); @@ -569,7 +599,9 @@ finalize_windowaggregate(WindowAggState *winstate, numFinalArgs, perfuncstate->winCollation, (void *) winstate, NULL); - fcinfo.arg[0] = peraggstate->transValue; + fcinfo.arg[0] = MakeExpandedObjectReadOnly(peraggstate->transValue, + peraggstate->transValueIsNull, + peraggstate->transtypeLen); fcinfo.argnull[0] = peraggstate->transValueIsNull; anynull = peraggstate->transValueIsNull; @@ -597,6 +629,7 @@ finalize_windowaggregate(WindowAggState *winstate, } else { + /* Don't need MakeExpandedObjectReadOnly; datumCopy will copy it */ *result = peraggstate->transValue; *isnull = peraggstate->transValueIsNull; } @@ -1552,15 +1585,12 @@ update_frametailpos(WindowObject winobj, TupleTableSlot *slot) * ExecWindowAgg receives tuples from its outer subplan and * stores them into a tuplestore, then processes window functions. * This node doesn't reduce nor qualify any row so the number of - * returned rows is exactly the same as its outer subplan's result - * (ignoring the case of SRFs in the targetlist, that is). + * returned rows is exactly the same as its outer subplan's result. * ----------------- */ TupleTableSlot * ExecWindowAgg(WindowAggState *winstate) { - TupleTableSlot *result; - ExprDoneCond isDone; ExprContext *econtext; int i; int numfuncs; @@ -1569,23 +1599,6 @@ ExecWindowAgg(WindowAggState *winstate) return NULL; /* - * Check to see if we're still projecting out tuples from a previous - * output tuple (because there is a function-returning-set in the - * projection expressions). If so, try to project another one. - */ - if (winstate->ss.ps.ps_TupFromTlist) - { - TupleTableSlot *result; - ExprDoneCond isDone; - - result = ExecProject(winstate->ss.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) - return result; - /* Done with that source tuple... */ - winstate->ss.ps.ps_TupFromTlist = false; - } - - /* * Compute frame offset values, if any, during first call. */ if (winstate->all_first) @@ -1602,8 +1615,7 @@ ExecWindowAgg(WindowAggState *winstate) Assert(winstate->startOffset != NULL); value = ExecEvalExprSwitchContext(winstate->startOffset, econtext, - &isnull, - NULL); + &isnull); if (isnull) ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), @@ -1628,8 +1640,7 @@ ExecWindowAgg(WindowAggState *winstate) Assert(winstate->endOffset != NULL); value = ExecEvalExprSwitchContext(winstate->endOffset, econtext, - &isnull, - NULL); + &isnull); if (isnull) ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), @@ -1652,7 +1663,6 @@ ExecWindowAgg(WindowAggState *winstate) winstate->all_first = false; } -restart: if (winstate->buffer == NULL) { /* Initialize for first partition and set current row = 0 */ @@ -1744,17 +1754,8 @@ restart: * evaluated with respect to that row. */ econtext->ecxt_outertuple = winstate->ss.ss_ScanTupleSlot; - result = ExecProject(winstate->ss.ps.ps_ProjInfo, &isDone); - - if (isDone == ExprEndResult) - { - /* SRF in tlist returned no rows, so advance to next input tuple */ - goto restart; - } - winstate->ss.ps.ps_TupFromTlist = - (isDone == ExprMultipleResult); - return result; + return ExecProject(winstate->ss.ps.ps_ProjInfo); } /* ----------------- @@ -1802,10 +1803,8 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags) /* Create long-lived context for storage of partition-local memory etc */ winstate->partcontext = AllocSetContextCreate(CurrentMemoryContext, - "WindowAgg_Partition", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + "WindowAgg Partition", + ALLOCSET_DEFAULT_SIZES); /* * Create mid-lived context for aggregate trans values etc. @@ -1815,10 +1814,8 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags) */ winstate->aggcontext = AllocSetContextCreate(CurrentMemoryContext, - "WindowAgg_Aggregates", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + "WindowAgg Aggregates", + ALLOCSET_DEFAULT_SIZES); /* * tuple table initialization @@ -1830,16 +1827,12 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags) winstate->temp_slot_1 = ExecInitExtraTupleSlot(estate); winstate->temp_slot_2 = ExecInitExtraTupleSlot(estate); - winstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->plan.targetlist, - (PlanState *) winstate); - /* * WindowAgg nodes never have quals, since they can only occur at the * logical top level of a query (ie, after any WHERE or HAVING filters) */ Assert(node->plan.qual == NIL); - winstate->ss.ps.qual = NIL; + winstate->ss.ps.qual = NULL; /* * initialize child nodes @@ -1868,8 +1861,6 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags) ExecAssignResultTypeFromTL(&winstate->ss.ps); ExecAssignProjectionInfo(&winstate->ss.ps, NULL); - winstate->ss.ps.ps_TupFromTlist = false; - /* Set up data for comparing tuples */ if (node->partNumCols > 0) winstate->partEqfunctions = execTuplesMatchPrepare(node->partNumCols, @@ -1900,7 +1891,7 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags) foreach(l, winstate->funcs) { WindowFuncExprState *wfuncstate = (WindowFuncExprState *) lfirst(l); - WindowFunc *wfunc = (WindowFunc *) wfuncstate->xprstate.expr; + WindowFunc *wfunc = wfuncstate->wfunc; WindowStatePerFunc perfuncstate; AclResult aclresult; int i; @@ -2062,8 +2053,6 @@ ExecReScanWindowAgg(WindowAggState *node) ExprContext *econtext = node->ss.ps.ps_ExprContext; node->all_done = false; - - node->ss.ps.ps_TupFromTlist = false; node->all_first = true; /* release tuplestore et al */ @@ -2223,7 +2212,7 @@ initialize_peragg(WindowAggState *winstate, WindowFunc *wfunc, numArguments, 0, /* no ordered-set window functions yet */ false, /* no variadic window functions yet */ - wfunc->wintype, + aggtranstype, wfunc->inputcollid, transfn_oid, invtransfn_oid, @@ -2322,10 +2311,8 @@ initialize_peragg(WindowAggState *winstate, WindowFunc *wfunc, if (OidIsValid(invtransfn_oid)) peraggstate->aggcontext = AllocSetContextCreate(CurrentMemoryContext, - "WindowAgg_AggregatePrivate", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + "WindowAgg Per Aggregate", + ALLOCSET_DEFAULT_SIZES); else peraggstate->aggcontext = winstate->aggcontext; @@ -2686,7 +2673,7 @@ WinGetFuncArgInPartition(WindowObject winobj, int argno, } econtext->ecxt_outertuple = slot; return ExecEvalExpr((ExprState *) list_nth(winobj->argstates, argno), - econtext, isnull, NULL); + econtext, isnull); } } @@ -2785,7 +2772,7 @@ WinGetFuncArgInFrame(WindowObject winobj, int argno, } econtext->ecxt_outertuple = slot; return ExecEvalExpr((ExprState *) list_nth(winobj->argstates, argno), - econtext, isnull, NULL); + econtext, isnull); } } @@ -2815,5 +2802,5 @@ WinGetFuncArgCurrent(WindowObject winobj, int argno, bool *isnull) econtext->ecxt_outertuple = winstate->ss.ss_ScanTupleSlot; return ExecEvalExpr((ExprState *) list_nth(winobj->argstates, argno), - econtext, isnull, NULL); + econtext, isnull); } diff --git a/src/backend/executor/nodeWorktablescan.c b/src/backend/executor/nodeWorktablescan.c index cfed6e6329..d7616be065 100644 --- a/src/backend/executor/nodeWorktablescan.c +++ b/src/backend/executor/nodeWorktablescan.c @@ -3,7 +3,7 @@ * nodeWorktablescan.c * routines to handle WorkTableScan nodes. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -95,8 +95,8 @@ ExecWorkTableScan(WorkTableScanState *node) param = &(estate->es_param_exec_vals[plan->wtParam]); Assert(param->execPlan == NULL); Assert(!param->isnull); - node->rustate = (RecursiveUnionState *) DatumGetPointer(param->value); - Assert(node->rustate && IsA(node->rustate, RecursiveUnionState)); + node->rustate = castNode(RecursiveUnionState, DatumGetPointer(param->value)); + Assert(node->rustate); /* * The scan tuple type (ie, the rowtype we expect to find in the work @@ -156,12 +156,8 @@ ExecInitWorkTableScan(WorkTableScan *node, EState *estate, int eflags) /* * initialize child expressions */ - scanstate->ss.ps.targetlist = (List *) - ExecInitExpr((Expr *) node->scan.plan.targetlist, - (PlanState *) scanstate); - scanstate->ss.ps.qual = (List *) - ExecInitExpr((Expr *) node->scan.plan.qual, - (PlanState *) scanstate); + scanstate->ss.ps.qual = + ExecInitQual(node->scan.plan.qual, (PlanState *) scanstate); /* * tuple table initialization @@ -174,8 +170,6 @@ ExecInitWorkTableScan(WorkTableScan *node, EState *estate, int eflags) */ ExecAssignResultTypeFromTL(&scanstate->ss.ps); - scanstate->ss.ps.ps_TupFromTlist = false; - return scanstate; } diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c index 0a3c65e6f7..ac0870c22d 100644 --- a/src/backend/executor/spi.c +++ b/src/backend/executor/spi.c @@ -3,7 +3,7 @@ * spi.c * Server Programming Interface * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * @@ -44,8 +44,7 @@ int SPI_result; static _SPI_connection *_SPI_stack = NULL; static _SPI_connection *_SPI_current = NULL; static int _SPI_stack_depth = 0; /* allocated size of _SPI_stack */ -static int _SPI_connected = -1; -static int _SPI_curid = -1; +static int _SPI_connected = -1; /* current stack index */ static Portal SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, ParamListInfo paramLI, bool read_only); @@ -91,13 +90,7 @@ SPI_connect(void) { int newdepth; - /* - * When procedure called by Executor _SPI_curid expected to be equal to - * _SPI_connected - */ - if (_SPI_curid != _SPI_connected) - return SPI_ERROR_CONNECT; - + /* Enlarge stack if necessary */ if (_SPI_stack == NULL) { if (_SPI_connected != -1 || _SPI_stack_depth != 0) @@ -122,9 +115,7 @@ SPI_connect(void) } } - /* - * We're entering procedure where _SPI_curid == _SPI_connected - 1 - */ + /* Enter new stack level */ _SPI_connected++; Assert(_SPI_connected >= 0 && _SPI_connected < _SPI_stack_depth); @@ -136,6 +127,7 @@ SPI_connect(void) _SPI_current->procCxt = NULL; /* in case we fail to create 'em */ _SPI_current->execCxt = NULL; _SPI_current->connectSubid = GetCurrentSubTransactionId(); + _SPI_current->queryEnv = NULL; /* * Create memory contexts for this procedure @@ -147,14 +139,10 @@ SPI_connect(void) */ _SPI_current->procCxt = AllocSetContextCreate(TopTransactionContext, "SPI Proc", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); _SPI_current->execCxt = AllocSetContextCreate(TopTransactionContext, "SPI Exec", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); /* ... and switch to procedure's context */ _SPI_current->savedcxt = MemoryContextSwitchTo(_SPI_current->procCxt); @@ -187,14 +175,9 @@ SPI_finish(void) SPI_lastoid = InvalidOid; SPI_tuptable = NULL; - /* - * After _SPI_begin_call _SPI_connected == _SPI_curid. Now we are closing - * connection to SPI and returning to upper Executor and so _SPI_connected - * must be equal to _SPI_curid. - */ + /* Exit stack level */ _SPI_connected--; - _SPI_curid--; - if (_SPI_connected == -1) + if (_SPI_connected < 0) _SPI_current = NULL; else _SPI_current = &(_SPI_stack[_SPI_connected]); @@ -221,7 +204,7 @@ AtEOXact_SPI(bool isCommit) _SPI_current = _SPI_stack = NULL; _SPI_stack_depth = 0; - _SPI_connected = _SPI_curid = -1; + _SPI_connected = -1; SPI_processed = 0; SPI_lastoid = InvalidOid; SPI_tuptable = NULL; @@ -267,8 +250,7 @@ AtEOSubXact_SPI(bool isCommit, SubTransactionId mySubid) * be already gone. */ _SPI_connected--; - _SPI_curid = _SPI_connected; - if (_SPI_connected == -1) + if (_SPI_connected < 0) _SPI_current = NULL; else _SPI_current = &(_SPI_stack[_SPI_connected]); @@ -322,53 +304,6 @@ AtEOSubXact_SPI(bool isCommit, SubTransactionId mySubid) } -/* Pushes SPI stack to allow recursive SPI calls */ -void -SPI_push(void) -{ - _SPI_curid++; -} - -/* Pops SPI stack to allow recursive SPI calls */ -void -SPI_pop(void) -{ - _SPI_curid--; -} - -/* Conditional push: push only if we're inside a SPI procedure */ -bool -SPI_push_conditional(void) -{ - bool pushed = (_SPI_curid != _SPI_connected); - - if (pushed) - { - _SPI_curid++; - /* We should now be in a state where SPI_connect would succeed */ - Assert(_SPI_curid == _SPI_connected); - } - return pushed; -} - -/* Conditional pop: pop only if SPI_push_conditional pushed */ -void -SPI_pop_conditional(bool pushed) -{ - /* We should be in a state where SPI_connect would succeed */ - Assert(_SPI_curid == _SPI_connected); - if (pushed) - _SPI_curid--; -} - -/* Restore state of SPI stack after aborting a subtransaction */ -void -SPI_restore_connection(void) -{ - Assert(_SPI_connected >= 0); - _SPI_curid = _SPI_connected - 1; -} - #ifdef PGXC /* SPI_execute_direct: * Runs the 'remote_sql' query string on the node 'nodename' @@ -433,7 +368,7 @@ SPI_execute(const char *src, bool read_only, long tcount) memset(&plan, 0, sizeof(_SPI_plan)); plan.magic = _SPI_PLAN_MAGIC; - plan.cursor_options = 0; + plan.cursor_options = CURSOR_OPT_PARALLEL_OK; _SPI_prepare_oneshot_plan(src, &plan); @@ -577,7 +512,7 @@ SPI_execute_with_args(const char *src, memset(&plan, 0, sizeof(_SPI_plan)); plan.magic = _SPI_PLAN_MAGIC; - plan.cursor_options = 0; + plan.cursor_options = CURSOR_OPT_PARALLEL_OK; plan.nargs = nargs; plan.argtypes = argtypes; plan.parserSetup = NULL; @@ -748,7 +683,7 @@ SPI_freeplan(SPIPlanPtr plan) HeapTuple SPI_copytuple(HeapTuple tuple) { - MemoryContext oldcxt = NULL; + MemoryContext oldcxt; HeapTuple ctuple; if (tuple == NULL) @@ -757,17 +692,17 @@ SPI_copytuple(HeapTuple tuple) return NULL; } - if (_SPI_curid + 1 == _SPI_connected) /* connected */ + if (_SPI_current == NULL) { - if (_SPI_current != &(_SPI_stack[_SPI_curid + 1])) - elog(ERROR, "SPI stack corrupted"); - oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); + SPI_result = SPI_ERROR_UNCONNECTED; + return NULL; } + oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); + ctuple = heap_copytuple(tuple); - if (oldcxt) - MemoryContextSwitchTo(oldcxt); + MemoryContextSwitchTo(oldcxt); return ctuple; } @@ -775,7 +710,7 @@ SPI_copytuple(HeapTuple tuple) HeapTupleHeader SPI_returntuple(HeapTuple tuple, TupleDesc tupdesc) { - MemoryContext oldcxt = NULL; + MemoryContext oldcxt; HeapTupleHeader dtup; if (tuple == NULL || tupdesc == NULL) @@ -784,22 +719,22 @@ SPI_returntuple(HeapTuple tuple, TupleDesc tupdesc) return NULL; } + if (_SPI_current == NULL) + { + SPI_result = SPI_ERROR_UNCONNECTED; + return NULL; + } + /* For RECORD results, make sure a typmod has been assigned */ if (tupdesc->tdtypeid == RECORDOID && tupdesc->tdtypmod < 0) assign_record_type_typmod(tupdesc); - if (_SPI_curid + 1 == _SPI_connected) /* connected */ - { - if (_SPI_current != &(_SPI_stack[_SPI_curid + 1])) - elog(ERROR, "SPI stack corrupted"); - oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); - } + oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); dtup = DatumGetHeapTupleHeader(heap_copy_tuple_as_datum(tuple, tupdesc)); - if (oldcxt) - MemoryContextSwitchTo(oldcxt); + MemoryContextSwitchTo(oldcxt); return dtup; } @@ -808,7 +743,7 @@ HeapTuple SPI_modifytuple(Relation rel, HeapTuple tuple, int natts, int *attnum, Datum *Values, const char *Nulls) { - MemoryContext oldcxt = NULL; + MemoryContext oldcxt; HeapTuple mtuple; int numberOfAttributes; Datum *v; @@ -821,13 +756,16 @@ SPI_modifytuple(Relation rel, HeapTuple tuple, int natts, int *attnum, return NULL; } - if (_SPI_curid + 1 == _SPI_connected) /* connected */ + if (_SPI_current == NULL) { - if (_SPI_current != &(_SPI_stack[_SPI_curid + 1])) - elog(ERROR, "SPI stack corrupted"); - oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); + SPI_result = SPI_ERROR_UNCONNECTED; + return NULL; } + + oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); + SPI_result = 0; + numberOfAttributes = rel->rd_att->natts; v = (Datum *) palloc(numberOfAttributes * sizeof(Datum)); n = (bool *) palloc(numberOfAttributes * sizeof(bool)); @@ -871,8 +809,7 @@ SPI_modifytuple(Relation rel, HeapTuple tuple, int natts, int *attnum, pfree(v); pfree(n); - if (oldcxt) - MemoryContextSwitchTo(oldcxt); + MemoryContextSwitchTo(oldcxt); return mtuple; } @@ -885,7 +822,8 @@ SPI_fnumber(TupleDesc tupdesc, const char *fname) for (res = 0; res < tupdesc->natts; res++) { - if (namestrcmp(&tupdesc->attrs[res]->attname, fname) == 0) + if (namestrcmp(&tupdesc->attrs[res]->attname, fname) == 0 && + !tupdesc->attrs[res]->attisdropped) return res + 1; } @@ -1040,22 +978,10 @@ SPI_getnspname(Relation rel) void * SPI_palloc(Size size) { - MemoryContext oldcxt = NULL; - void *pointer; - - if (_SPI_curid + 1 == _SPI_connected) /* connected */ - { - if (_SPI_current != &(_SPI_stack[_SPI_curid + 1])) - elog(ERROR, "SPI stack corrupted"); - oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); - } + if (_SPI_current == NULL) + elog(ERROR, "SPI_palloc called while not connected to SPI"); - pointer = palloc(size); - - if (oldcxt) - MemoryContextSwitchTo(oldcxt); - - return pointer; + return MemoryContextAlloc(_SPI_current->savedcxt, size); } void * @@ -1075,20 +1001,17 @@ SPI_pfree(void *pointer) Datum SPI_datumTransfer(Datum value, bool typByVal, int typLen) { - MemoryContext oldcxt = NULL; + MemoryContext oldcxt; Datum result; - if (_SPI_curid + 1 == _SPI_connected) /* connected */ - { - if (_SPI_current != &(_SPI_stack[_SPI_curid + 1])) - elog(ERROR, "SPI stack corrupted"); - oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); - } + if (_SPI_current == NULL) + elog(ERROR, "SPI_datumTransfer called while not connected to SPI"); + + oldcxt = MemoryContextSwitchTo(_SPI_current->savedcxt); result = datumTransfer(value, typByVal, typLen); - if (oldcxt) - MemoryContextSwitchTo(oldcxt); + MemoryContextSwitchTo(oldcxt); return result; } @@ -1110,17 +1033,12 @@ SPI_freetuptable(SPITupleTable *tuptable) return; /* - * Since this function might be called during error recovery, it seems - * best not to insist that the caller be actively connected. We just - * search the topmost SPI context, connected or not. + * Search only the topmost SPI context for a matching tuple table. */ - if (_SPI_connected >= 0) + if (_SPI_current != NULL) { slist_mutable_iter siter; - if (_SPI_current != &(_SPI_stack[_SPI_connected])) - elog(ERROR, "SPI stack corrupted"); - /* find tuptable in active list, then remove it */ slist_foreach_modify(siter, &_SPI_current->tuptables) { @@ -1228,13 +1146,9 @@ SPI_cursor_open_with_args(const char *name, /* We needn't copy the plan; SPI_cursor_open_internal will do so */ - /* Adjust stack so that SPI_cursor_open_internal doesn't complain */ - _SPI_curid--; - result = SPI_cursor_open_internal(name, &plan, paramLI, read_only); /* And clean up */ - _SPI_curid++; _SPI_end_call(true); return result; @@ -1337,12 +1251,9 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, */ /* Replan if needed, and increment plan refcount for portal */ - cplan = GetCachedPlan(plansource, paramLI, false); + cplan = GetCachedPlan(plansource, paramLI, false, _SPI_current->queryEnv); stmt_list = cplan->stmt_list; - /* Pop the error context stack */ - error_context_stack = spierrcontext.previous; - if (!plan->saved) { /* @@ -1376,9 +1287,9 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, if (!(portal->cursorOptions & (CURSOR_OPT_SCROLL | CURSOR_OPT_NO_SCROLL))) { if (list_length(stmt_list) == 1 && - IsA((Node *) linitial(stmt_list), PlannedStmt) && - ((PlannedStmt *) linitial(stmt_list))->rowMarks == NIL && - ExecSupportsBackwardScan(((PlannedStmt *) linitial(stmt_list))->planTree)) + linitial_node(PlannedStmt, stmt_list)->commandType != CMD_UTILITY && + linitial_node(PlannedStmt, stmt_list)->rowMarks == NIL && + ExecSupportsBackwardScan(linitial_node(PlannedStmt, stmt_list)->planTree)) portal->cursorOptions |= CURSOR_OPT_SCROLL; else portal->cursorOptions |= CURSOR_OPT_NO_SCROLL; @@ -1392,14 +1303,17 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, if (portal->cursorOptions & CURSOR_OPT_SCROLL) { if (list_length(stmt_list) == 1 && - IsA((Node *) linitial(stmt_list), PlannedStmt) && - ((PlannedStmt *) linitial(stmt_list))->rowMarks != NIL) + linitial_node(PlannedStmt, stmt_list)->commandType != CMD_UTILITY && + linitial_node(PlannedStmt, stmt_list)->rowMarks != NIL) ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("DECLARE SCROLL CURSOR ... FOR UPDATE/SHARE is not supported"), errdetail("Scrollable cursors must be READ ONLY."))); } + /* Make current query environment available to portal at execution time. */ + portal->queryEnv = _SPI_current->queryEnv; + /* * If told to be read-only, or in parallel mode, verify that this query is * in fact read-only. This can't be done earlier because we need to look @@ -1414,7 +1328,7 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, foreach(lc, stmt_list) { - Node *pstmt = (Node *) lfirst(lc); + PlannedStmt *pstmt = lfirst_node(PlannedStmt, lc); if (!CommandIsReadOnly(pstmt)) { @@ -1423,9 +1337,9 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), /* translator: %s is a SQL statement name */ errmsg("%s is not allowed in a non-volatile function", - CreateCommandTag(pstmt)))); + CreateCommandTag((Node *) pstmt)))); else - PreventCommandIfParallelMode(CreateCommandTag(pstmt)); + PreventCommandIfParallelMode(CreateCommandTag((Node *) pstmt)); } } } @@ -1458,6 +1372,9 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan, Assert(portal->strategy != PORTAL_MULTI_QUERY); + /* Pop the error context stack */ + error_context_stack = spierrcontext.previous; + /* Pop the SPI stack */ _SPI_end_call(true); @@ -1676,6 +1593,10 @@ SPI_result_code_string(int code) return "SPI_ERROR_NOOUTFUNC"; case SPI_ERROR_TYPUNKNOWN: return "SPI_ERROR_TYPUNKNOWN"; + case SPI_ERROR_REL_DUPLICATE: + return "SPI_ERROR_REL_DUPLICATE"; + case SPI_ERROR_REL_NOT_FOUND: + return "SPI_ERROR_REL_NOT_FOUND"; case SPI_OK_CONNECT: return "SPI_OK_CONNECT"; case SPI_OK_FINISH: @@ -1704,6 +1625,10 @@ SPI_result_code_string(int code) return "SPI_OK_UPDATE_RETURNING"; case SPI_OK_REWRITTEN: return "SPI_OK_REWRITTEN"; + case SPI_OK_REL_REGISTER: + return "SPI_OK_REL_REGISTER"; + case SPI_OK_REL_UNREGISTER: + return "SPI_OK_REL_UNREGISTER"; } /* Unrecognized code ... return something useful ... */ sprintf(buf, "Unrecognized SPI code %d", code); @@ -1714,7 +1639,7 @@ SPI_result_code_string(int code) * SPI_plan_get_plan_sources --- get a SPI plan's underlying list of * CachedPlanSources. * - * This is exported so that pl/pgsql can use it (this beats letting pl/pgsql + * This is exported so that PL/pgSQL can use it (this beats letting PL/pgSQL * look directly into the SPIPlan for itself). It's not documented in * spi.sgml because we'd just as soon not have too many places using this. */ @@ -1730,7 +1655,7 @@ SPI_plan_get_plan_sources(SPIPlanPtr plan) * if the SPI plan contains exactly one CachedPlanSource. If not, * return NULL. Caller is responsible for doing ReleaseCachedPlan(). * - * This is exported so that pl/pgsql can use it (this beats letting pl/pgsql + * This is exported so that PL/pgSQL can use it (this beats letting PL/pgSQL * look directly into the SPIPlan for itself). It's not documented in * spi.sgml because we'd just as soon not have too many places using this. */ @@ -1759,7 +1684,8 @@ SPI_plan_get_cached_plan(SPIPlanPtr plan) error_context_stack = &spierrcontext; /* Get the generic plan for the query */ - cplan = GetCachedPlan(plansource, NULL, plan->saved); + cplan = GetCachedPlan(plansource, NULL, plan->saved, + _SPI_current->queryEnv); Assert(cplan == plansource->gplan); /* Pop the error context stack */ @@ -1783,14 +1709,8 @@ spi_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo) MemoryContext oldcxt; MemoryContext tuptabcxt; - /* - * When called by Executor _SPI_curid expected to be equal to - * _SPI_connected - */ - if (_SPI_curid != _SPI_connected || _SPI_connected < 0) - elog(ERROR, "improper call to spi_dest_startup"); - if (_SPI_current != &(_SPI_stack[_SPI_curid])) - elog(ERROR, "SPI stack corrupted"); + if (_SPI_current == NULL) + elog(ERROR, "spi_dest_startup called while not connected to SPI"); if (_SPI_current->tuptable != NULL) elog(ERROR, "improper call to spi_dest_startup"); @@ -1801,9 +1721,7 @@ spi_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo) tuptabcxt = AllocSetContextCreate(CurrentMemoryContext, "SPI TupTable", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); MemoryContextSwitchTo(tuptabcxt); _SPI_current->tuptable = tuptable = (SPITupleTable *) @@ -1837,14 +1755,8 @@ spi_printtup(TupleTableSlot *slot, DestReceiver *self) SPITupleTable *tuptable; MemoryContext oldcxt; - /* - * When called by Executor _SPI_curid expected to be equal to - * _SPI_connected - */ - if (_SPI_curid != _SPI_connected || _SPI_connected < 0) - elog(ERROR, "improper call to spi_printtup"); - if (_SPI_current != &(_SPI_stack[_SPI_curid])) - elog(ERROR, "SPI stack corrupted"); + if (_SPI_current == NULL) + elog(ERROR, "spi_printtup called while not connected to SPI"); tuptable = _SPI_current->tuptable; if (tuptable == NULL) @@ -1939,7 +1851,7 @@ _SPI_pgxc_prepare_plan(const char *src, List *src_parsetree, forboth(list_item, raw_parsetree_list, list_item2, querysource_list) { - Node *parsetree = (Node *) lfirst(list_item); + RawStmt *parsetree = lfirst_node(RawStmt, list_item); char *querysource = (char *) lfirst (list_item2); List *stmt_list; CachedPlanSource *plansource; @@ -1953,7 +1865,7 @@ _SPI_pgxc_prepare_plan(const char *src, List *src_parsetree, #ifdef PGXC NULL, #endif - CreateCommandTag(parsetree)); + CreateCommandTag(parsetree->stmt)); /* * Parameter datatypes are driven by parserSetup hook if provided, @@ -1965,14 +1877,16 @@ _SPI_pgxc_prepare_plan(const char *src, List *src_parsetree, stmt_list = pg_analyze_and_rewrite_params(parsetree, querysource, plan->parserSetup, - plan->parserSetupArg); + plan->parserSetupArg, + _SPI_current->queryEnv); } else { stmt_list = pg_analyze_and_rewrite(parsetree, querysource, plan->argtypes, - plan->nargs); + plan->nargs, + _SPI_current->queryEnv); } /* Finish filling in the CachedPlanSource */ @@ -2046,13 +1960,14 @@ _SPI_prepare_oneshot_plan(const char *src, SPIPlanPtr plan) forboth(list_item, raw_parsetree_list, list_item2, querysource_list) { - Node *parsetree = (Node *) lfirst(list_item); + RawStmt *parsetree = lfirst_node(RawStmt, list_item); char *querysource = (char *) lfirst (list_item2); CachedPlanSource *plansource; + plansource = CreateOneShotCachedPlan(parsetree, querysource, - CreateCommandTag(parsetree)); + CreateCommandTag(parsetree->stmt)); plancache_list = lappend(plancache_list, plansource); } @@ -2147,7 +2062,7 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, */ if (plan->oneshot) { - Node *parsetree = plansource->raw_parse_tree; + RawStmt *parsetree = plansource->raw_parse_tree; const char *src = plansource->query_string; List *stmt_list; @@ -2163,14 +2078,16 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, stmt_list = pg_analyze_and_rewrite_params(parsetree, src, plan->parserSetup, - plan->parserSetupArg); + plan->parserSetupArg, + _SPI_current->queryEnv); } else { stmt_list = pg_analyze_and_rewrite(parsetree, src, plan->argtypes, - plan->nargs); + plan->nargs, + _SPI_current->queryEnv); } /* Finish filling in the CachedPlanSource */ @@ -2189,7 +2106,7 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, * Replan if needed, and increment plan refcount. If it's a saved * plan, the refcount must be backed by the CurrentResourceOwner. */ - cplan = GetCachedPlan(plansource, paramLI, plan->saved); + cplan = GetCachedPlan(plansource, paramLI, plan->saved, _SPI_current->queryEnv); stmt_list = cplan->stmt_list; /* @@ -2206,26 +2123,19 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, foreach(lc2, stmt_list) { - Node *stmt = (Node *) lfirst(lc2); - bool canSetTag; + PlannedStmt *stmt = lfirst_node(PlannedStmt, lc2); + bool canSetTag = stmt->canSetTag; DestReceiver *dest; _SPI_current->processed = 0; _SPI_current->lastoid = InvalidOid; _SPI_current->tuptable = NULL; - if (IsA(stmt, PlannedStmt)) - { - canSetTag = ((PlannedStmt *) stmt)->canSetTag; - } - else + if (stmt->utilityStmt) { - /* utilities are canSetTag if only thing in list */ - canSetTag = (list_length(stmt_list) == 1); - - if (IsA(stmt, CopyStmt)) + if (IsA(stmt->utilityStmt, CopyStmt)) { - CopyStmt *cstmt = (CopyStmt *) stmt; + CopyStmt *cstmt = (CopyStmt *) stmt->utilityStmt; if (cstmt->filename == NULL) { @@ -2233,7 +2143,7 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, goto fail; } } - else if (IsA(stmt, TransactionStmt)) + else if (IsA(stmt->utilityStmt, TransactionStmt)) { my_res = SPI_ERROR_TRANSACTION; goto fail; @@ -2245,10 +2155,10 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), /* translator: %s is a SQL statement name */ errmsg("%s is not allowed in a non-volatile function", - CreateCommandTag(stmt)))); + CreateCommandTag((Node *) stmt)))); if (IsInParallelMode() && !CommandIsReadOnly(stmt)) - PreventCommandIfParallelMode(CreateCommandTag(stmt)); + PreventCommandIfParallelMode(CreateCommandTag((Node *) stmt)); /* * If not read-only mode, advance the command counter before each @@ -2262,8 +2172,7 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, dest = CreateDestReceiver(canSetTag ? DestSPI : DestNone); - if (IsA(stmt, PlannedStmt) && - ((PlannedStmt *) stmt)->utilityStmt == NULL) + if (stmt->utilityStmt == NULL) { QueryDesc *qdesc; Snapshot snap; @@ -2273,11 +2182,12 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, else snap = InvalidSnapshot; - qdesc = CreateQueryDesc((PlannedStmt *) stmt, + qdesc = CreateQueryDesc(stmt, plansource->query_string, snap, crosscheck_snapshot, dest, - paramLI, 0); + paramLI, _SPI_current->queryEnv, + 0); res = _SPI_pquery(qdesc, fire_triggers, canSetTag ? tcount : 0); FreeQueryDesc(qdesc); @@ -2290,6 +2200,7 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, plansource->query_string, PROCESS_UTILITY_QUERY, paramLI, + _SPI_current->queryEnv, dest, #ifdef PGXC false, @@ -2307,9 +2218,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, * Some utility statements return a row count, even though the * tuples are not returned to the caller. */ - if (IsA(stmt, CreateTableAsStmt)) + if (IsA(stmt->utilityStmt, CreateTableAsStmt)) { - CreateTableAsStmt *ctastmt = (CreateTableAsStmt *) stmt; + CreateTableAsStmt *ctastmt = (CreateTableAsStmt *) stmt->utilityStmt; if (strncmp(completionTag, "SELECT ", 7) == 0) _SPI_current->processed = @@ -2332,7 +2243,7 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI, if (ctastmt->is_select_into) res = SPI_OK_SELINTO; } - else if (IsA(stmt, CopyStmt)) + else if (IsA(stmt->utilityStmt, CopyStmt)) { Assert(strncmp(completionTag, "COPY ", 5) == 0); _SPI_current->processed = pg_strtouint64(completionTag + 5, @@ -2461,7 +2372,6 @@ _SPI_pquery(QueryDesc *queryDesc, bool fire_triggers, uint64 tcount) switch (operation) { case CMD_SELECT: - Assert(queryDesc->plannedstmt->utilityStmt == NULL); if (queryDesc->dest->mydest != DestSPI) { /* Don't return SPI_OK_SELECT if we're discarding result */ @@ -2505,7 +2415,7 @@ _SPI_pquery(QueryDesc *queryDesc, bool fire_triggers, uint64 tcount) ExecutorStart(queryDesc, eflags); - ExecutorRun(queryDesc, ForwardScanDirection, tcount); + ExecutorRun(queryDesc, ForwardScanDirection, tcount, true); _SPI_current->processed = queryDesc->estate->es_processed; _SPI_current->lastoid = queryDesc->estate->es_lastoid; @@ -2629,11 +2539,8 @@ _SPI_procmem(void) static int _SPI_begin_call(bool execmem) { - if (_SPI_curid + 1 != _SPI_connected) + if (_SPI_current == NULL) return SPI_ERROR_UNCONNECTED; - _SPI_curid++; - if (_SPI_current != &(_SPI_stack[_SPI_curid])) - elog(ERROR, "SPI stack corrupted"); if (execmem) /* switch to the Executor memory context */ _SPI_execmem(); @@ -2649,11 +2556,6 @@ _SPI_begin_call(bool execmem) static int _SPI_end_call(bool procmem) { - /* - * We're returning to procedure where _SPI_curid == _SPI_connected - 1 - */ - _SPI_curid--; - if (procmem) /* switch to the procedure memory context */ { _SPI_procmem(); @@ -2705,14 +2607,11 @@ _SPI_make_plan_non_temp(SPIPlanPtr plan) /* * Create a memory context for the plan, underneath the procedure context. - * We don't expect the plan to be very large, so use smaller-than-default - * alloc parameters. + * We don't expect the plan to be very large. */ plancxt = AllocSetContextCreate(parentcxt, "SPI Plan", - ALLOCSET_SMALL_MINSIZE, - ALLOCSET_SMALL_INITSIZE, - ALLOCSET_SMALL_MAXSIZE); + ALLOCSET_SMALL_SIZES); oldcxt = MemoryContextSwitchTo(plancxt); /* Copy the SPI_plan struct and subsidiary data into the new context */ @@ -2779,9 +2678,7 @@ _SPI_save_plan(SPIPlanPtr plan) */ plancxt = AllocSetContextCreate(CurrentMemoryContext, "SPI Plan", - ALLOCSET_SMALL_MINSIZE, - ALLOCSET_SMALL_INITSIZE, - ALLOCSET_SMALL_MAXSIZE); + ALLOCSET_SMALL_SIZES); oldcxt = MemoryContextSwitchTo(plancxt); /* Copy the SPI plan into its own context */ @@ -2832,3 +2729,133 @@ _SPI_save_plan(SPIPlanPtr plan) return newplan; } + +/* + * Internal lookup of ephemeral named relation by name. + */ +static EphemeralNamedRelation +_SPI_find_ENR_by_name(const char *name) +{ + /* internal static function; any error is bug in SPI itself */ + Assert(name != NULL); + + /* fast exit if no tuplestores have been added */ + if (_SPI_current->queryEnv == NULL) + return NULL; + + return get_ENR(_SPI_current->queryEnv, name); +} + +/* + * Register an ephemeral named relation for use by the planner and executor on + * subsequent calls using this SPI connection. + */ +int +SPI_register_relation(EphemeralNamedRelation enr) +{ + EphemeralNamedRelation match; + int res; + + if (enr == NULL || enr->md.name == NULL) + return SPI_ERROR_ARGUMENT; + + res = _SPI_begin_call(false); /* keep current memory context */ + if (res < 0) + return res; + + match = _SPI_find_ENR_by_name(enr->md.name); + if (match) + res = SPI_ERROR_REL_DUPLICATE; + else + { + if (_SPI_current->queryEnv == NULL) + _SPI_current->queryEnv = create_queryEnv(); + + register_ENR(_SPI_current->queryEnv, enr); + res = SPI_OK_REL_REGISTER; + } + + _SPI_end_call(false); + + return res; +} + +/* + * Unregister an ephemeral named relation by name. This will probably be a + * rarely used function, since SPI_finish will clear it automatically. + */ +int +SPI_unregister_relation(const char *name) +{ + EphemeralNamedRelation match; + int res; + + if (name == NULL) + return SPI_ERROR_ARGUMENT; + + res = _SPI_begin_call(false); /* keep current memory context */ + if (res < 0) + return res; + + match = _SPI_find_ENR_by_name(name); + if (match) + { + unregister_ENR(_SPI_current->queryEnv, match->md.name); + res = SPI_OK_REL_UNREGISTER; + } + else + res = SPI_ERROR_REL_NOT_FOUND; + + _SPI_end_call(false); + + return res; +} + +/* + * Register the transient relations from 'tdata' using this SPI connection. + * This should be called by PL implementations' trigger handlers after + * connecting, in order to make transition tables visible to any queries run + * in this connection. + */ +int +SPI_register_trigger_data(TriggerData *tdata) +{ + if (tdata == NULL) + return SPI_ERROR_ARGUMENT; + + if (tdata->tg_newtable) + { + EphemeralNamedRelation enr = + palloc(sizeof(EphemeralNamedRelationData)); + int rc; + + enr->md.name = tdata->tg_trigger->tgnewtable; + enr->md.reliddesc = tdata->tg_relation->rd_id; + enr->md.tupdesc = NULL; + enr->md.enrtype = ENR_NAMED_TUPLESTORE; + enr->md.enrtuples = tuplestore_tuple_count(tdata->tg_newtable); + enr->reldata = tdata->tg_newtable; + rc = SPI_register_relation(enr); + if (rc != SPI_OK_REL_REGISTER) + return rc; + } + + if (tdata->tg_oldtable) + { + EphemeralNamedRelation enr = + palloc(sizeof(EphemeralNamedRelationData)); + int rc; + + enr->md.name = tdata->tg_trigger->tgoldtable; + enr->md.reliddesc = tdata->tg_relation->rd_id; + enr->md.tupdesc = NULL; + enr->md.enrtype = ENR_NAMED_TUPLESTORE; + enr->md.enrtuples = tuplestore_tuple_count(tdata->tg_oldtable); + enr->reldata = tdata->tg_oldtable; + rc = SPI_register_relation(enr); + if (rc != SPI_OK_REL_REGISTER) + return rc; + } + + return SPI_OK_TD_REGISTER; +} diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c index 58d0eeaf0b..8d7e711b3b 100644 --- a/src/backend/executor/tqueue.c +++ b/src/backend/executor/tqueue.c @@ -23,7 +23,7 @@ * and rewrites the typmods sent by the remote side to the corresponding * local record typmods. * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * IDENTIFICATION @@ -281,9 +281,7 @@ tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self) tqueue->tmpcontext = AllocSetContextCreate(tqueue->mycontext, "tqueue sender temp context", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); + ALLOCSET_DEFAULT_SIZES); oldcontext = MemoryContextSwitchTo(tqueue->tmpcontext); } diff --git a/src/backend/executor/tstoreReceiver.c b/src/backend/executor/tstoreReceiver.c index 8f1e1b3f50..1e641c9837 100644 --- a/src/backend/executor/tstoreReceiver.c +++ b/src/backend/executor/tstoreReceiver.c @@ -9,7 +9,7 @@ * data even if the underlying table is dropped. * * - * Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California * * IDENTIFICATION |