diff --git a/src/gausskernel/process/stream/execStream.cpp b/src/gausskernel/process/stream/execStream.cpp index cd485f731..5ebf25835 100755 --- a/src/gausskernel/process/stream/execStream.cpp +++ b/src/gausskernel/process/stream/execStream.cpp @@ -70,6 +70,7 @@ #include "optimizer/dataskew.h" #include "instruments/instr_unique_sql.h" +static TupleTableSlot* ExecStream(PlanState* state); static void CheckStreamMatchInfo( StreamFlowCheckInfo checkInfo, int plan_node_id, List* consumerExecNode, int consumerDop, bool isLocalStream); static void InitStream(StreamFlowCtl* ctl, StreamTransType type); @@ -2132,6 +2133,7 @@ StreamState* ExecInitStream(Stream* node, EState* estate, int eflags) state->isReady = false; state->vector_output = false; state->StreamScan = ScanStreamByLibcomm; + state->ss.ps.ExecProcNode = ExecStream; if (STREAM_IS_LOCAL_NODE(node->smpDesc.distriType)) { state->StreamScan = ScanMemoryStream; @@ -2169,8 +2171,9 @@ StreamState* ExecInitStream(Stream* node, EState* estate, int eflags) return state; } -TupleTableSlot* ExecStream(StreamState* node) +static TupleTableSlot* ExecStream(PlanState* state) { + StreamState* node = castNode(StreamState, state); if (unlikely(node->isReady == false)) { StreamPrepareRequest(node); diff --git a/src/gausskernel/runtime/executor/execProcnode.cpp b/src/gausskernel/runtime/executor/execProcnode.cpp index 2b097893b..ab4342968 100755 --- a/src/gausskernel/runtime/executor/execProcnode.cpp +++ b/src/gausskernel/runtime/executor/execProcnode.cpp @@ -19,11 +19,6 @@ * ------------------------------------------------------------------------- */ /* - * INTERFACE ROUTINES - * ExecInitNode - initialize a plan node and its subplans - * ExecProcNode - get a tuple by executing the plan node - * ExecEndNode - shut down a plan node and its subplans - * * NOTES * This used to be three files. It is now all combined into * one file so that it is easier to keep ExecInitNode, ExecProcNode, @@ -164,6 +159,8 @@ #include "gstrace/executer_gstrace.h" #include "executor/node/nodeTrainModel.h" #define NODENAMELEN 64 +static TupleTableSlot *ExecProcNodeFirst(PlanState *node); +static TupleTableSlot *ExecProcNodeInstr(PlanState *node); /* * Function to determine a plannode should be processed in stub-routine when exec_nodes @@ -453,6 +450,13 @@ PlanState* ExecInitNode(Plan* node, EState* estate, int e_flags) return NULL; } + /* + * Make sure there's enough stack available. Need to check here, in + * addition to ExecProcNode() (via ExecProcNodeFirst()), to ensure the + * stack isn't overrun while initializing the node tree. + */ + check_stack_depth(); + gstrace_entry(GS_TRC_ID_ExecInitNode); if (!StreamTopConsumerAmI()) @@ -503,6 +507,13 @@ PlanState* ExecInitNode(Plan* node, EState* estate, int e_flags) /* Set the nodeContext */ result->nodeContext = node_context; + + /* + * Add a wrapper around the ExecProcNode callback that checks stack depth + * during the first execution. + */ + result->ExecProcNodeReal = result->ExecProcNode; + result->ExecProcNode = ExecProcNodeFirst; /* * Initialize any initPlans present in this node. The planner put them in @@ -601,112 +612,12 @@ PlanState* ExecInitNode(Plan* node, EState* estate, int e_flags) return result; } -TupleTableSlot* ExecProcNodeByType(PlanState* node) +static TupleTableSlot* ExecProcNodeInstr(PlanState* node) { - TupleTableSlot* result = NULL; - switch (nodeTag(node)) { - case T_ResultState: - return ExecResult((ResultState*)node); - case T_ModifyTableState: - case T_DistInsertSelectState: - return ExecModifyTable((ModifyTableState*)node); - case T_AppendState: - return ExecAppend((AppendState*)node); - case T_MergeAppendState: - return ExecMergeAppend((MergeAppendState*)node); - case T_RecursiveUnionState: - return ExecRecursiveUnion((RecursiveUnionState*)node); - case T_StartWithOpState: - return ExecStartWithOp((StartWithOpState*)node); - case T_SeqScanState: - return ExecSeqScan((SeqScanState*)node); - case T_IndexScanState: - return ExecIndexScan((IndexScanState*)node); - case T_IndexOnlyScanState: - return ExecIndexOnlyScan((IndexOnlyScanState*)node); - case T_BitmapHeapScanState: - return ExecBitmapHeapScan((BitmapHeapScanState*)node); - case T_TidScanState: - return ExecTidScan((TidScanState*)node); - case T_SubqueryScanState: - return ExecSubqueryScan((SubqueryScanState*)node); - case T_FunctionScanState: - return ExecFunctionScan((FunctionScanState*)node); - case T_ValuesScanState: - return ExecValuesScan((ValuesScanState*)node); - case T_CteScanState: - return ExecCteScan((CteScanState*)node); - case T_WorkTableScanState: - return ExecWorkTableScan((WorkTableScanState*)node); - case T_ForeignScanState: - return ExecForeignScan((ForeignScanState*)node); - case T_ExtensiblePlanState: - return ExecExtensiblePlan((ExtensiblePlanState*)node); - /* - * join nodes - */ - case T_NestLoopState: - return ExecNestLoop((NestLoopState*)node); - case T_MergeJoinState: - return ExecMergeJoin((MergeJoinState*)node); - case T_HashJoinState: - return ExecHashJoin((HashJoinState*)node); + TupleTableSlot *result; + InstrStartNode(node->instrument); + result = node->ExecProcNodeReal(node); - /* - * partition iterator node - */ - case T_PartIteratorState: - return ExecPartIterator((PartIteratorState*)node); - /* - * materialization nodes - */ - case T_MaterialState: - return ExecMaterial((MaterialState*)node); - case T_SortState: - return ExecSort((SortState*)node); - case T_GroupState: - return ExecGroup((GroupState*)node); - case T_AggState: - return ExecAgg((AggState*)node); - case T_WindowAggState: - return ExecWindowAgg((WindowAggState*)node); - case T_UniqueState: - return ExecUnique((UniqueState*)node); - case T_HashState: - return ExecHash(); - case T_SetOpState: - return ExecSetOp((SetOpState*)node); - case T_LockRowsState: - return ExecLockRows((LockRowsState*)node); - case T_LimitState: - return ExecLimit((LimitState*)node); - case T_VecToRowState: - return ExecVecToRow((VecToRowState*)node); -#ifdef PGXC - case T_RemoteQueryState: - t_thrd.pgxc_cxt.GlobalNetInstr = node->instrument; - result = ExecRemoteQuery((RemoteQueryState*)node); - t_thrd.pgxc_cxt.GlobalNetInstr = NULL; - return result; -#endif - case T_StreamState: - t_thrd.pgxc_cxt.GlobalNetInstr = node->instrument; - result = ExecStream((StreamState*)node); - t_thrd.pgxc_cxt.GlobalNetInstr = NULL; - return result; - case T_TrainModelState: - return ExecTrainModel((TrainModelState*)node); - default: - ereport(ERROR, - (errmodule(MOD_EXECUTOR), - errcode(ERRCODE_UNRECOGNIZED_NODE_TYPE), - errmsg("unrecognized node type: %d when executing executor node.", (int)nodeTag(node)))); - return NULL; - } -} - -void ExecProcNodeInstr(PlanState* node, TupleTableSlot* result) -{ switch (nodeTag(node)) { case T_ModifyTableState: case T_DistInsertSelectState: @@ -751,298 +662,38 @@ void ExecProcNodeInstr(PlanState* node, TupleTableSlot* result) if (TupIsNull(result)) node->instrument->status = true; -} - -typedef TupleTableSlot* (*ExecProcFuncType)(PlanState* node); - -static inline TupleTableSlot *DefaultExecProc(PlanState *node) -{ - ereport(ERROR, - (errmodule(MOD_EXECUTOR), - errcode(ERRCODE_UNRECOGNIZED_NODE_TYPE), - errmsg("unrecognized node type: %d when executing executor node.", (int)nodeTag(node)))); - return NULL; -} - -static inline TupleTableSlot *ExecResultWrap(PlanState *node) -{ - return ExecResult((ResultState*)node); -}; - -static inline TupleTableSlot *ExecVecToRowWrap(PlanState *node) -{ - return ExecVecToRow((VecToRowState*)node); -} - -static inline TupleTableSlot *ExecModifyTableWrap(PlanState *node) -{ - return ExecModifyTable((ModifyTableState*)node); -}; - -static inline TupleTableSlot *ExecAppendWrap(PlanState *node) -{ - return ExecAppend((AppendState*)node); -}; - -static inline TupleTableSlot *ExecPartIteratorWrap(PlanState *node) -{ - return ExecPartIterator((PartIteratorState*)node); -}; - -static inline TupleTableSlot *ExecMergeAppendWrap(PlanState *node) -{ - return ExecMergeAppend((MergeAppendState*)node); -}; - -static inline TupleTableSlot *ExecRecursiveUnionWrap(PlanState *node) -{ - return ExecRecursiveUnion((RecursiveUnionState*)node); -}; - -static inline TupleTableSlot *ExecStartWithOpWrap(PlanState *node) -{ - return ExecStartWithOp((StartWithOpState*)node); -}; - -static inline TupleTableSlot *ExecSeqScanWrap(PlanState *node) -{ - return ExecSeqScan((SeqScanState *)node); -}; - -static inline TupleTableSlot *ExecIndexScanWrap(PlanState *node) -{ - return ExecIndexScan((IndexScanState *)node); -}; - -static inline TupleTableSlot *ExecIndexOnlyScanWrap(PlanState *node) -{ - return ExecIndexOnlyScan((IndexOnlyScanState *)node); -}; - -static inline TupleTableSlot *ExecBitmapHeapScanWrap(PlanState *node) -{ - return ExecBitmapHeapScan((BitmapHeapScanState *)node); -}; - -static inline TupleTableSlot *ExecTidScanWrap(PlanState *node) -{ - return ExecTidScan((TidScanState *)node); -}; - -static inline TupleTableSlot *ExecSubqueryScanWrap(PlanState *node) -{ - return ExecSubqueryScan((SubqueryScanState *)node); -}; - -static inline TupleTableSlot *ExecFunctionScanWrap(PlanState *node) -{ - return ExecFunctionScan((FunctionScanState *)node); -}; - -static inline TupleTableSlot *ExecValuesScanWrap(PlanState *node) -{ - return ExecValuesScan((ValuesScanState *)node); -}; - -static inline TupleTableSlot *ExecCteScanWrap(PlanState *node) -{ - return ExecCteScan((CteScanState *)node); -}; - -static inline TupleTableSlot *ExecWorkTableScanWrap(PlanState *node) -{ - return ExecWorkTableScan((WorkTableScanState *)node); -}; - -static inline TupleTableSlot *ExecForeignScanWrap(PlanState *node) -{ - return ExecForeignScan((ForeignScanState *)node); -}; - -static inline TupleTableSlot *ExecExtensiblePlanWrap(PlanState *node) -{ - return ExecExtensiblePlan((ExtensiblePlanState *)node); -}; - -static inline TupleTableSlot *ExecNestLoopWrap(PlanState *node) -{ - return ExecNestLoop((NestLoopState *)node); -}; - -static inline TupleTableSlot *ExecMergeJoinWrap(PlanState *node) -{ - return ExecMergeJoin((MergeJoinState *)node); -}; - -static inline TupleTableSlot *ExecHashJoinWrap(PlanState *node) -{ - return ExecHashJoin((HashJoinState *)node); -}; - -static inline TupleTableSlot *ExecMaterialWrap(PlanState *node) -{ - return ExecMaterial((MaterialState *)node); -}; - -static inline TupleTableSlot *ExecSortWrap(PlanState *node) -{ - return ExecSort((SortState *)node); -}; - -static inline TupleTableSlot *ExecGroupWrap(PlanState *node) -{ - return ExecGroup((GroupState *)node); -}; - -static inline TupleTableSlot *ExecAggWrap(PlanState *node) -{ - return ExecAgg((AggState *)node); -}; - -static inline TupleTableSlot *ExecWindowAggWrap(PlanState *node) -{ - return ExecWindowAgg((WindowAggState *)node); -}; - -static inline TupleTableSlot *ExecUniqueWrap(PlanState *node) -{ - return ExecUnique((UniqueState *)node); -}; - -static inline TupleTableSlot *ExecHashWrap(PlanState *node) -{ - return ExecHash(); -}; - -static inline TupleTableSlot *ExecSetOpWrap(PlanState *node) -{ - return ExecSetOp((SetOpState *)node); -}; - -static TupleTableSlot *ExecLockRowsWrap(PlanState *node) -{ - return ExecLockRows((LockRowsState *)node); -}; - -static inline TupleTableSlot *ExecLimitWrap(PlanState *node) -{ - return ExecLimit((LimitState *)node); -}; - -static inline TupleTableSlot *ExecRemoteQueryWrap(PlanState *node) -{ - return ExecRemoteQuery((RemoteQueryState *)node); -}; - -static inline TupleTableSlot *ExecTrainModelWrap(PlanState *node) -{ - return ExecTrainModel((TrainModelState*)node); -} - -static inline TupleTableSlot *ExecStreamWrap(PlanState *node) -{ - return ExecStream((StreamState *)node); -}; - -ExecProcFuncType g_execProcFuncTable[] = { - ExecResultWrap, - ExecVecToRowWrap, - DefaultExecProc, - ExecModifyTableWrap, - ExecModifyTableWrap, - ExecAppendWrap, - ExecPartIteratorWrap, - ExecMergeAppendWrap, - ExecRecursiveUnionWrap, - ExecStartWithOpWrap, - DefaultExecProc, - DefaultExecProc, - DefaultExecProc, - ExecSeqScanWrap, - ExecIndexScanWrap, - ExecIndexOnlyScanWrap, - DefaultExecProc, - ExecBitmapHeapScanWrap, - ExecTidScanWrap, - ExecSubqueryScanWrap, - ExecFunctionScanWrap, - ExecValuesScanWrap, - ExecCteScanWrap, - ExecWorkTableScanWrap, - ExecForeignScanWrap, - ExecExtensiblePlanWrap, - DefaultExecProc, - ExecNestLoopWrap, - ExecMergeJoinWrap, - ExecHashJoinWrap, - ExecMaterialWrap, - ExecSortWrap, - ExecGroupWrap, - ExecAggWrap, - ExecWindowAggWrap, - ExecUniqueWrap, - ExecHashWrap, - ExecSetOpWrap, - ExecLockRowsWrap, - ExecLimitWrap, - ExecRemoteQueryWrap, - ExecTrainModelWrap, - ExecStreamWrap -}; - -/* ---------------------------------------------------------------- - * ExecProcNode - * - * Execute the given node to return a(nother) tuple. - * ---------------------------------------------------------------- - */ -TupleTableSlot* ExecProcNode(PlanState* node) -{ - TupleTableSlot* result = NULL; - - CHECK_FOR_INTERRUPTS(); - MemoryContext old_context; - - /* Response to stop or cancel signal. */ -#ifdef ENABLE_MULTIPLE_NODES - if (unlikely(executorEarlyStop())) { - return NULL; - } -#endif - - /* Switch to Node Level Memory Context */ - old_context = MemoryContextSwitchTo(node->nodeContext); - - if (node->chgParam != NULL) { /* something changed */ - ExecReScan(node); /* let ReScan handle this */ - } - - if (node->instrument != NULL) { - InstrStartNode(node->instrument); - } - -#ifdef ENABLE_MULTIPLE_NODES - if (unlikely(planstate_need_stub(node))) { - result = ExecProcNodeStub(node); - } else -#endif - { - int index = (int)(nodeTag(node))-T_ResultState; - Assert(index >= 0 && index <= T_StreamState - T_ResultState); - result = g_execProcFuncTable[index](node); - } - - if (node->instrument != NULL) { - ExecProcNodeInstr(node, result); - } - - MemoryContextSwitchTo(old_context); - - node->ps_rownum++; return result; } +/* + * ExecProcNode wrapper that performs some one-time checks, before calling + * the relevant node method (possibly via an instrumentation wrapper). + */ +static TupleTableSlot *ExecProcNodeFirst(PlanState *node) +{ + /* + * Perform stack depth check during the first execution of the node. We + * only do so the first time round because it turns out to not be cheap on + * some common architectures (eg. x86). This relies on the assumption that + * ExecProcNode calls for a given plan node will always be made at roughly + * the same stack depth. + */ + check_stack_depth(); + + /* + * If instrumentation is required, change the wrapper to one that just + * does instrumentation. Otherwise we can dispense with all wrappers and + * have ExecProcNode() directly call the relevant function from now on. + */ + if (node->instrument) + node->ExecProcNode = ExecProcNodeInstr; + else + node->ExecProcNode = node->ExecProcNodeReal; + + return node->ExecProcNode(node); +} + /* ---------------------------------------------------------------- * MultiExecProcNode * @@ -1059,12 +710,16 @@ TupleTableSlot* ExecProcNode(PlanState* node) Node* MultiExecProcNode(PlanState* node) { Node* result = NULL; - MemoryContext old_context; + MemoryContext old_context = NULL; + + check_stack_depth(); CHECK_FOR_INTERRUPTS(); - /* Switch to Node Level Memory Context */ - old_context = MemoryContextSwitchTo(node->nodeContext); + if (unlikely(node->nodeContext)) { + /* Switch to Node Level Memory Context */ + old_context = MemoryContextSwitchTo(node->nodeContext); + } if (node->chgParam != NULL) { /* something changed */ ExecReScan(node); /* let ReScan handle this */ @@ -1104,7 +759,9 @@ Node* MultiExecProcNode(PlanState* node) node->instrument->memoryinfo.operatorMemory = node->plan->operatorMemKB[0]; } - MemoryContextSwitchTo(old_context); + if (unlikely(old_context)) { + MemoryContextSwitchTo(old_context); + } return result; } @@ -1620,6 +1277,7 @@ void ExecEndNode(PlanState* node) if (node == NULL) { return; } + check_stack_depth(); cleanup_sensitive_information(); if (node->chgParam != NULL) { bms_free_ext(node->chgParam); diff --git a/src/gausskernel/runtime/executor/execScan.cpp b/src/gausskernel/runtime/executor/execScan.cpp index bf4ebf240..73e6e95db 100644 --- a/src/gausskernel/runtime/executor/execScan.cpp +++ b/src/gausskernel/runtime/executor/execScan.cpp @@ -170,8 +170,6 @@ TupleTableSlot* ExecScan(ScanState* node, ExecScanAccessMtd access_mtd, /* funct for (;;) { TupleTableSlot* slot = NULL; - CHECK_FOR_INTERRUPTS(); - slot = ExecScanFetch(node, access_mtd, recheck_mtd); /* refresh qual every loop */ qual = node->ps.qual; diff --git a/src/gausskernel/runtime/executor/nodeAgg.cpp b/src/gausskernel/runtime/executor/nodeAgg.cpp index 2f0487fd9..9632afa27 100644 --- a/src/gausskernel/runtime/executor/nodeAgg.cpp +++ b/src/gausskernel/runtime/executor/nodeAgg.cpp @@ -147,6 +147,7 @@ #include "utils/memprot.h" #include "workload/workload.h" +static TupleTableSlot* ExecAgg(PlanState* state); static void initialize_aggregates( AggState* aggstate, AggStatePerAgg peragg, AggStatePerGroup pergroup, int numReset = 0); static void advance_transition_function( @@ -248,6 +249,8 @@ static TupleTableSlot* fetch_input_tuple(AggState* aggstate) TupleTableSlot* slot = NULL; if (aggstate->sort_in) { + /* make sure we check for interrupts in either path through here */ + CHECK_FOR_INTERRUPTS(); if (!tuplesort_gettupleslot(aggstate->sort_in, true, aggstate->sort_slot, NULL)) return NULL; slot = aggstate->sort_slot; @@ -1461,8 +1464,9 @@ static TupleTableSlot* agg_retrieve(AggState* node) * stored in the expression context to be used when ExecProject evaluates * the result tuple. */ -TupleTableSlot* ExecAgg(AggState* node) +static TupleTableSlot* ExecAgg(PlanState* state) { + AggState* node = castNode(AggState, state); /* * just for cooperation analysis. do nothing if is_dummy is true. * is_dummy is true that means Agg node is deparsed to remote sql in ForeignScan node. @@ -1987,6 +1991,7 @@ AggState* ExecInitAgg(Agg* node, EState* estate, int eflags) aggstate->sort_in = NULL; aggstate->sort_out = NULL; aggstate->is_final = node->is_final; + aggstate->ss.ps.ExecProcNode = ExecAgg; /* * Calculate the maximum number of grouping sets in any phase; this diff --git a/src/gausskernel/runtime/executor/nodeAppend.cpp b/src/gausskernel/runtime/executor/nodeAppend.cpp index 6f38c5b01..f5cd48631 100644 --- a/src/gausskernel/runtime/executor/nodeAppend.cpp +++ b/src/gausskernel/runtime/executor/nodeAppend.cpp @@ -60,6 +60,8 @@ #include "executor/exec/execdebug.h" #include "executor/node/nodeAppend.h" +static TupleTableSlot* ExecAppend(PlanState* state); + /* ---------------------------------------------------------------- * exec_append_initialize_next * @@ -132,6 +134,7 @@ AppendState* ExecInitAppend(Append* node, EState* estate, int eflags) appendstate->ps.state = estate; appendstate->appendplans = appendplanstates; appendstate->as_nplans = nplans; + appendstate->ps.ExecProcNode = ExecAppend; /* * Miscellaneous initialization @@ -179,12 +182,15 @@ AppendState* ExecInitAppend(Append* node, EState* estate, int eflags) * Handles iteration over multiple subplans. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecAppend(AppendState* node) +static TupleTableSlot* ExecAppend(PlanState* state) { + AppendState* node = castNode(AppendState, state); for (;;) { PlanState* subnode = NULL; TupleTableSlot* result = NULL; + CHECK_FOR_INTERRUPTS(); + /* * figure out which subplan we are currently processing */ diff --git a/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp b/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp index 3526a600e..c911bb683 100644 --- a/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp +++ b/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp @@ -63,6 +63,7 @@ #include "nodes/makefuncs.h" #include "optimizer/pruning.h" +static TupleTableSlot* ExecBitmapHeapScan(PlanState* state); static TupleTableSlot* BitmapHbucketTblNext(BitmapHeapScanState* node); static TupleTableSlot* BitmapHeapTblNext(BitmapHeapScanState* node); bool heapam_scan_bitmap_next_block(TableScanDesc scan, TBMIterateResult* tbmres, @@ -610,8 +611,9 @@ static bool BitmapHeapRecheck(BitmapHeapScanState* node, TupleTableSlot* slot) * ExecBitmapHeapScan(node) * ---------------------------------------------------------------- */ -TupleTableSlot* ExecBitmapHeapScan(BitmapHeapScanState* node) +static TupleTableSlot* ExecBitmapHeapScan(PlanState* state) { + BitmapHeapScanState* node = castNode(BitmapHeapScanState, state); return ExecScan(&node->ss, node->ss.ScanNextMtd, (ExecScanRecheckMtd)BitmapHeapRecheck); } @@ -768,6 +770,7 @@ BitmapHeapScanState* ExecInitBitmapHeapScan(BitmapHeapScan* node, EState* estate scanstate->ss.isPartTbl = node->scan.isPartTbl; scanstate->ss.currentSlot = 0; scanstate->ss.partScanDirection = node->scan.partScanDirection; + scanstate->ss.ps.ExecProcNode = ExecBitmapHeapScan; /* initialize Global partition index scan information */ GPIScanInit(&scanstate->gpi_scan); diff --git a/src/gausskernel/runtime/executor/nodeCtescan.cpp b/src/gausskernel/runtime/executor/nodeCtescan.cpp index 3adae70f2..c84d95490 100644 --- a/src/gausskernel/runtime/executor/nodeCtescan.cpp +++ b/src/gausskernel/runtime/executor/nodeCtescan.cpp @@ -20,6 +20,8 @@ #include "executor/node/nodeCtescan.h" #include "miscadmin.h" +static TupleTableSlot* ExecCteScan(PlanState* state); + static TupleTableSlot* CteScanNext(CteScanState* node); /* ---------------------------------------------------------------- @@ -151,8 +153,9 @@ static bool CteScanRecheck(CteScanState* node, TupleTableSlot* slot) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecCteScan(CteScanState* node) +static TupleTableSlot* ExecCteScan(PlanState* state) { + CteScanState* node = castNode(CteScanState, state); return ExecScan(&node->ss, (ExecScanAccessMtd)CteScanNext, (ExecScanRecheckMtd)CteScanRecheck); } @@ -197,6 +200,7 @@ CteScanState* ExecInitCteScan(CteScan* node, EState* estate, int eflags) scanstate->eflags = eflags; scanstate->cte_table = NULL; scanstate->eof_cte = false; + scanstate->ss.ps.ExecProcNode = ExecCteScan; /* * Find the already-initialized plan for the CTE query. diff --git a/src/gausskernel/runtime/executor/nodeExtensible.cpp b/src/gausskernel/runtime/executor/nodeExtensible.cpp index 4d385089e..7f83626da 100644 --- a/src/gausskernel/runtime/executor/nodeExtensible.cpp +++ b/src/gausskernel/runtime/executor/nodeExtensible.cpp @@ -44,6 +44,8 @@ static HTAB* g_extensible_plan_methods = NULL; const int HASHTABLE_LENGTH = 100; const char* EXTENSIBLE_PLAN_METHODS_LABEL = "Extensible Plan Methods"; +static TupleTableSlot* ExecExtensiblePlan(PlanState* state); + typedef struct { char extnodename[EXTNODENAME_MAX_LEN]; void* extnodemethods; @@ -109,6 +111,7 @@ ExtensiblePlanState* ExecInitExtensiblePlan(ExtensiblePlan* eplan, EState* estat /* fill up fields of ScanState */ extensionPlanState->ss.ps.plan = &eplan->scan.plan; extensionPlanState->ss.ps.state = estate; + extensionPlanState->ss.ps.ExecProcNode = ExecExtensiblePlan; /* create expression context for node */ ExecAssignExprContext(estate, &extensionPlanState->ss.ps); @@ -168,8 +171,9 @@ ExtensiblePlanState* ExecInitExtensiblePlan(ExtensiblePlan* eplan, EState* estat return extensionPlanState; } -TupleTableSlot* ExecExtensiblePlan(ExtensiblePlanState* node) +static TupleTableSlot* ExecExtensiblePlan(PlanState* state) { + ExtensiblePlanState* node = castNode(ExtensiblePlanState, state); Assert(node->methods->ExecExtensiblePlan != NULL); return node->methods->ExecExtensiblePlan(node); } diff --git a/src/gausskernel/runtime/executor/nodeForeignscan.cpp b/src/gausskernel/runtime/executor/nodeForeignscan.cpp index b04b858ec..3ca0060b7 100644 --- a/src/gausskernel/runtime/executor/nodeForeignscan.cpp +++ b/src/gausskernel/runtime/executor/nodeForeignscan.cpp @@ -41,6 +41,7 @@ #include "utils/knl_relcache.h" +static TupleTableSlot* ExecForeignScan(PlanState* state); static TupleTableSlot* ForeignNext(ForeignScanState* node); static bool ForeignRecheck(ForeignScanState* node, TupleTableSlot* slot); @@ -99,8 +100,9 @@ static bool ForeignRecheck(ForeignScanState* node, TupleTableSlot* slot) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecForeignScan(ForeignScanState* node) +static TupleTableSlot* ExecForeignScan(PlanState* state) { + ForeignScanState* node = castNode(ForeignScanState, state); return ExecScan((ScanState*)node, (ExecScanAccessMtd)ForeignNext, (ExecScanRecheckMtd)ForeignRecheck); } @@ -126,6 +128,7 @@ ForeignScanState* ExecInitForeignScan(ForeignScan* node, EState* estate, int efl scanstate = makeNode(ForeignScanState); scanstate->ss.ps.plan = (Plan*)node; scanstate->ss.ps.state = estate; + scanstate->ss.ps.ExecProcNode = ExecForeignScan; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeFunctionscan.cpp b/src/gausskernel/runtime/executor/nodeFunctionscan.cpp index fcf91f206..11122c254 100644 --- a/src/gausskernel/runtime/executor/nodeFunctionscan.cpp +++ b/src/gausskernel/runtime/executor/nodeFunctionscan.cpp @@ -27,6 +27,7 @@ #include "funcapi.h" #include "nodes/nodeFuncs.h" +static TupleTableSlot* ExecFunctionScan(PlanState* node); static TupleTableSlot* FunctionNext(FunctionScanState* node); /* ---------------------------------------------------------------- @@ -89,8 +90,9 @@ static bool FunctionRecheck(FunctionScanState* node, TupleTableSlot* slot) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecFunctionScan(FunctionScanState* node) +static TupleTableSlot* ExecFunctionScan(PlanState* state) { + FunctionScanState* node = castNode(FunctionScanState, state); return ExecScan(&node->ss, (ExecScanAccessMtd)FunctionNext, (ExecScanRecheckMtd)FunctionRecheck); } @@ -121,6 +123,7 @@ FunctionScanState* ExecInitFunctionScan(FunctionScan* node, EState* estate, int scanstate->ss.ps.plan = (Plan*)node; scanstate->ss.ps.state = estate; scanstate->eflags = eflags; + scanstate->ss.ps.ExecProcNode = ExecFunctionScan; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeGroup.cpp b/src/gausskernel/runtime/executor/nodeGroup.cpp index 05ba5c29d..499225edb 100644 --- a/src/gausskernel/runtime/executor/nodeGroup.cpp +++ b/src/gausskernel/runtime/executor/nodeGroup.cpp @@ -26,19 +26,24 @@ #include "executor/executor.h" #include "executor/node/nodeGroup.h" +static TupleTableSlot* ExecGroup(PlanState* state); + /* * ExecGroup - * * Return one tuple for each group of matching input tuples. */ -TupleTableSlot* ExecGroup(GroupState* node) +static TupleTableSlot* ExecGroup(PlanState* state) { + GroupState* node = castNode(GroupState, state); ExprContext* econtext = NULL; int numCols; AttrNumber* grpColIdx = NULL; TupleTableSlot* firsttupleslot = NULL; TupleTableSlot* outerslot = NULL; + CHECK_FOR_INTERRUPTS(); + /* * get state info from node */ @@ -194,6 +199,7 @@ GroupState* ExecInitGroup(Group* node, EState* estate, int eflags) grpstate->ss.ps.plan = (Plan*)node; grpstate->ss.ps.state = estate; grpstate->grp_done = FALSE; + grpstate->ss.ps.ExecProcNode = ExecGroup; /* * create expression context diff --git a/src/gausskernel/runtime/executor/nodeHash.cpp b/src/gausskernel/runtime/executor/nodeHash.cpp index a802e5ba6..f7da17b40 100644 --- a/src/gausskernel/runtime/executor/nodeHash.cpp +++ b/src/gausskernel/runtime/executor/nodeHash.cpp @@ -62,7 +62,7 @@ static void* dense_alloc(HashJoinTable hashtable, Size size); * stub for pro forma compliance * ---------------------------------------------------------------- */ -TupleTableSlot* ExecHash(void) +static TupleTableSlot* ExecHash(PlanState* state) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), @@ -116,6 +116,8 @@ Node* MultiExecHash(HashState* node) */ WaitState oldStatus = pgstat_report_waitstatus(STATE_EXEC_HASHJOIN_BUILD_HASH); for (;;) { + /* allow this loop to be cancellable */ + CHECK_FOR_INTERRUPTS(); slot = ExecProcNode(outerNode); if (TupIsNull(slot)) break; @@ -197,6 +199,7 @@ HashState* ExecInitHash(Hash* node, EState* estate, int eflags) hashstate->ps.state = estate; hashstate->hashtable = NULL; hashstate->hashkeys = NIL; /* will be set by parent HashJoin */ + hashstate->ps.ExecProcNode = ExecHash; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeHashjoin.cpp b/src/gausskernel/runtime/executor/nodeHashjoin.cpp index 0d7272f27..829b1f280 100755 --- a/src/gausskernel/runtime/executor/nodeHashjoin.cpp +++ b/src/gausskernel/runtime/executor/nodeHashjoin.cpp @@ -40,6 +40,7 @@ /* Returns true if doing null-fill on inner relation */ #define HJ_FILL_INNER(hjstate) ((hjstate)->hj_NullOuterTupleSlot != NULL) +static TupleTableSlot* ExecHashJoin(PlanState* state); static TupleTableSlot* ExecHashJoinOuterGetTuple(PlanState* outerNode, HashJoinState* hjstate, uint32* hashvalue); static TupleTableSlot* ExecHashJoinGetSavedTuple( HashJoinState* hjstate, BufFile* file, uint32* hashvalue, TupleTableSlot* tupleSlot); @@ -55,8 +56,9 @@ static bool ExecHashJoinNewBatch(HashJoinState* hjstate); * ---------------------------------------------------------------- */ /* return: a tuple or NULL */ -TupleTableSlot* ExecHashJoin(HashJoinState* node) +static TupleTableSlot* ExecHashJoin(PlanState* state) { + HashJoinState* node = castNode(HashJoinState, state); PlanState* outerNode = NULL; HashState* hashNode = NULL; List* joinqual = NIL; @@ -67,7 +69,7 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) TupleTableSlot* outerTupleSlot = NULL; uint32 hashvalue; int batchno; - MemoryContext oldcxt; + MemoryContext oldcxt = NULL; JoinType jointype; /* @@ -107,6 +109,14 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) * run the hash join state machine */ for (;;) { + /* + * It's possible to iterate this loop many times before returning a + * tuple, in some pathological cases such as needing to move much of + * the current batch to a later batch. So let's check for interrupts + * each time through. + */ + CHECK_FOR_INTERRUPTS(); + switch (node->hj_JoinState) { case HJ_BUILD_HASHTABLE: { /* @@ -170,10 +180,19 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) /* * create the hash table, sometimes we should keep nulls */ - oldcxt = MemoryContextSwitchTo(hashNode->ps.nodeContext); + if (hashNode->ps.nodeContext) { + /*enable_memory_limit*/ + oldcxt = MemoryContextSwitchTo(hashNode->ps.nodeContext); + } + hashtable = ExecHashTableCreate((Hash*)hashNode->ps.plan, node->hj_HashOperators, HJ_FILL_INNER(node) || node->js.nulleqqual != NIL); - MemoryContextSwitchTo(oldcxt); + + if (oldcxt) { + /*enable_memory_limit*/ + MemoryContextSwitchTo(oldcxt); + } + node->hj_HashTable = hashtable; /* @@ -528,6 +547,7 @@ HashJoinState* ExecInitHashJoin(HashJoin* node, EState* estate, int eflags) hjstate->js.ps.state = estate; hjstate->hj_streamBothSides = node->streamBothSides; hjstate->hj_rebuildHashtable = node->rebuildHashTable; + hjstate->js.ps.ExecProcNode = ExecHashJoin; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp b/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp index 0b800657c..5dd07ffed 100644 --- a/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp +++ b/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp @@ -44,6 +44,7 @@ #include "optimizer/pruning.h" +static TupleTableSlot* ExecIndexOnlyScan(PlanState* state); static TupleTableSlot* IndexOnlyNext(IndexOnlyScanState* node); static void ExecInitNextIndexPartitionForIndexScanOnly(IndexOnlyScanState* node); @@ -147,6 +148,8 @@ static TupleTableSlot* IndexOnlyNext(IndexOnlyScanState* node) HeapTuple tuple = NULL; IndexScanDesc indexScan = GetIndexScanDesc(scandesc); + CHECK_FOR_INTERRUPTS(); + /* * We can skip the heap fetch if the TID references a heap page on * which all tuples are known visible to everybody. In any case, @@ -319,8 +322,9 @@ static bool IndexOnlyRecheck(IndexOnlyScanState* node, TupleTableSlot* slot) * ExecIndexOnlyScan(node) * ---------------------------------------------------------------- */ -TupleTableSlot* ExecIndexOnlyScan(IndexOnlyScanState* node) +static TupleTableSlot* ExecIndexOnlyScan(PlanState* state) { + IndexOnlyScanState* node = castNode(IndexOnlyScanState, state); /* * If we have runtime keys and they've not already been set up, do it now. */ @@ -552,6 +556,7 @@ IndexOnlyScanState* ExecInitIndexOnlyScan(IndexOnlyScan* node, EState* estate, i indexstate->ss.isPartTbl = node->scan.isPartTbl; indexstate->ss.partScanDirection = node->indexorderdir; indexstate->ss.currentSlot = 0; + indexstate->ss.ps.ExecProcNode = ExecIndexOnlyScan; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeIndexscan.cpp b/src/gausskernel/runtime/executor/nodeIndexscan.cpp index 528aac02c..37df68c3d 100644 --- a/src/gausskernel/runtime/executor/nodeIndexscan.cpp +++ b/src/gausskernel/runtime/executor/nodeIndexscan.cpp @@ -44,7 +44,7 @@ #include "nodes/makefuncs.h" #include "optimizer/pruning.h" - +static TupleTableSlot* ExecIndexScan(PlanState* state); static TupleTableSlot* IndexNext(IndexScanState* node); static void ExecInitNextPartitionForIndexScan(IndexScanState* node); @@ -89,6 +89,9 @@ static TupleTableSlot* IndexNext(IndexScanState* node) // we should change abs_idx_getnext to call IdxScanAm(scan)->idx_getnext and channge .idx_getnext in g_HeapIdxAm to // IndexGetnextSlot while (true) { + + CHECK_FOR_INTERRUPTS(); + IndexScanDesc indexScan = GetIndexScanDesc(scandesc); if (isUstore) { if (!IndexGetnextSlot(scandesc, direction, slot, &node->ss.ps.state->have_current_xact_date)) { @@ -160,8 +163,9 @@ static bool IndexRecheck(IndexScanState* node, TupleTableSlot* slot) * ExecIndexScan(node) * ---------------------------------------------------------------- */ -TupleTableSlot* ExecIndexScan(IndexScanState* node) +TupleTableSlot* ExecIndexScan(PlanState* state) { + IndexScanState* node = castNode(IndexScanState, state); /* * If we have runtime keys and they've not already been set up, do it now. */ @@ -654,6 +658,7 @@ IndexScanState* ExecInitIndexScan(IndexScan* node, EState* estate, int eflags) index_state->ss.isPartTbl = node->scan.isPartTbl; index_state->ss.currentSlot = 0; index_state->ss.partScanDirection = node->indexorderdir; + index_state->ss.ps.ExecProcNode = ExecIndexScan; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeLimit.cpp b/src/gausskernel/runtime/executor/nodeLimit.cpp index 5780635e5..b402b697f 100644 --- a/src/gausskernel/runtime/executor/nodeLimit.cpp +++ b/src/gausskernel/runtime/executor/nodeLimit.cpp @@ -25,6 +25,7 @@ #include "executor/node/nodeLimit.h" #include "nodes/nodeFuncs.h" +static TupleTableSlot* ExecLimit(PlanState* state); static void pass_down_bound(LimitState* node, PlanState* child_node); /* ---------------------------------------------------------------- @@ -34,12 +35,15 @@ static void pass_down_bound(LimitState* node, PlanState* child_node); * filtering on the stream of tuples returned by a subplan. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecLimit(LimitState* node) /* return: a tuple or NULL */ +static TupleTableSlot* ExecLimit(PlanState* state) /* return: a tuple or NULL */ { + LimitState* node = castNode(LimitState, state); ScanDirection direction; TupleTableSlot* slot = NULL; PlanState* outer_plan = NULL; + CHECK_FOR_INTERRUPTS(); + /* * get information from the node */ @@ -353,6 +357,7 @@ LimitState* ExecInitLimit(Limit* node, EState* estate, int eflags) limit_state->ps.state = estate; limit_state->lstate = LIMIT_INITIAL; + limit_state->ps.ExecProcNode = ExecLimit; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeLockRows.cpp b/src/gausskernel/runtime/executor/nodeLockRows.cpp index 91981972e..62e7aa4d3 100755 --- a/src/gausskernel/runtime/executor/nodeLockRows.cpp +++ b/src/gausskernel/runtime/executor/nodeLockRows.cpp @@ -34,13 +34,16 @@ #include "utils/snapmgr.h" #include "access/tableam.h" +static TupleTableSlot* ExecLockRows(PlanState* state); + /* ---------------------------------------------------------------- * ExecLockRows * return: a tuple or NULL * ---------------------------------------------------------------- */ -TupleTableSlot* ExecLockRows(LockRowsState* node) +static TupleTableSlot* ExecLockRows(PlanState* state) { + LockRowsState* node = castNode(LockRowsState, state); TupleTableSlot* slot = NULL; EState* estate = NULL; PlanState* outer_plan = NULL; @@ -53,6 +56,8 @@ TupleTableSlot* ExecLockRows(LockRowsState* node) bool orig_early_free = false; bool orig_early_deinit = false; + CHECK_FOR_INTERRUPTS(); + /* * get information from the node */ @@ -438,6 +443,7 @@ LockRowsState* ExecInitLockRows(LockRows* node, EState* estate, int eflags) lrstate->ps.plan = (Plan*)node; lrstate->ps.state = estate; + lrstate->ps.ExecProcNode = ExecLockRows; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeMaterial.cpp b/src/gausskernel/runtime/executor/nodeMaterial.cpp index e9507f938..7173cd572 100644 --- a/src/gausskernel/runtime/executor/nodeMaterial.cpp +++ b/src/gausskernel/runtime/executor/nodeMaterial.cpp @@ -29,6 +29,8 @@ #include "pgstat.h" #include "pgxc/pgxc.h" +static TupleTableSlot* ExecMaterial(PlanState* state); + /* * Material all the tuples first, and then return the tuple needed. */ @@ -261,8 +263,12 @@ static TupleTableSlot* ExecMaterialOne(MaterialState* node) /* result tuple from * * ---------------------------------------------------------------- */ -TupleTableSlot* ExecMaterial(MaterialState* node) /* result tuple from subplan */ +static TupleTableSlot* ExecMaterial(PlanState* state) /* result tuple from subplan */ { + MaterialState* node = castNode(MaterialState, state); + + CHECK_FOR_INTERRUPTS(); + if (node->materalAll) return ExecMaterialAll(node); else @@ -283,6 +289,7 @@ MaterialState* ExecInitMaterial(Material* node, EState* estate, int eflags) MaterialState* mat_state = makeNode(MaterialState); mat_state->ss.ps.plan = (Plan*)node; mat_state->ss.ps.state = estate; + mat_state->ss.ps.ExecProcNode = ExecMaterial; int64 operator_mem = SET_NODEMEM(((Plan*)node)->operatorMemKB[0], ((Plan*)node)->dop); AllocSetContext* set = (AllocSetContext*)(estate->es_query_cxt); diff --git a/src/gausskernel/runtime/executor/nodeMergeAppend.cpp b/src/gausskernel/runtime/executor/nodeMergeAppend.cpp index 2a76e8ae8..c6da728bb 100644 --- a/src/gausskernel/runtime/executor/nodeMergeAppend.cpp +++ b/src/gausskernel/runtime/executor/nodeMergeAppend.cpp @@ -51,6 +51,7 @@ typedef int SlotNumber; typedef int HeapPosition; +static TupleTableSlot* ExecMergeAppend(PlanState* state); static void heap_insert_slot(MergeAppendState* node, SlotNumber new_slot); static void heap_siftup_slot(MergeAppendState* node); static int32 heap_compare_slots(MergeAppendState* node, SlotNumber slot1, SlotNumber slot2); @@ -86,6 +87,7 @@ MergeAppendState* ExecInitMergeAppend(MergeAppend* node, EState* estate, int efl merge_state->ps.state = estate; merge_state->mergeplans = merge_plan_states; merge_state->ms_nplans = nplans; + merge_state->ps.ExecProcNode = ExecMergeAppend; merge_state->ms_slots = (TupleTableSlot**)palloc0(sizeof(TupleTableSlot*) * nplans); merge_state->ms_heap = (int*)palloc0(sizeof(int) * nplans); @@ -163,11 +165,14 @@ MergeAppendState* ExecInitMergeAppend(MergeAppend* node, EState* estate, int efl * Handles iteration over multiple subplans. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecMergeAppend(MergeAppendState* node) +static TupleTableSlot* ExecMergeAppend(PlanState* state) { + MergeAppendState* node = castNode(MergeAppendState, state); TupleTableSlot* result = NULL; SlotNumber i; + CHECK_FOR_INTERRUPTS(); + if (!node->ms_initialized) { /* * First time through: pull the first tuple from each subplan, and set diff --git a/src/gausskernel/runtime/executor/nodeMergejoin.cpp b/src/gausskernel/runtime/executor/nodeMergejoin.cpp index e6623f74f..281ffe912 100644 --- a/src/gausskernel/runtime/executor/nodeMergejoin.cpp +++ b/src/gausskernel/runtime/executor/nodeMergejoin.cpp @@ -134,6 +134,7 @@ typedef enum { #define MarkInnerTuple(inner_tuple_slot, merge_state) ExecCopySlot((merge_state)->mj_MarkedTupleSlot, (inner_tuple_slot)) +static TupleTableSlot* ExecMergeJoin(PlanState* state); /* * MJExamineQuals * @@ -549,14 +550,17 @@ static void ExecMergeTupleDump(MergeJoinState* merge_state) * ExecMergeJoin * ---------------------------------------------------------------- */ -TupleTableSlot* ExecMergeJoin(MergeJoinState* node) +static TupleTableSlot* ExecMergeJoin(PlanState* state) { + MergeJoinState* node = castNode(MergeJoinState, state); bool qual_result = false; int compare_result; TupleTableSlot* inner_tuple_slot = NULL; TupleTableSlot* outer_tuple_slot = NULL; ExprDoneCond isDone; + CHECK_FOR_INTERRUPTS(); + /* * get information from node */ @@ -1406,6 +1410,7 @@ MergeJoinState* ExecInitMergeJoin(MergeJoin* node, EState* estate, int eflags) MergeJoinState* merge_state = makeNode(MergeJoinState); merge_state->js.ps.plan = (Plan*)node; merge_state->js.ps.state = estate; + merge_state->js.ps.ExecProcNode = ExecMergeJoin; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeModifyTable.cpp b/src/gausskernel/runtime/executor/nodeModifyTable.cpp index 95979e9a7..74f0f1e79 100644 --- a/src/gausskernel/runtime/executor/nodeModifyTable.cpp +++ b/src/gausskernel/runtime/executor/nodeModifyTable.cpp @@ -100,6 +100,7 @@ static void RecoredGeneratedExpr(ResultRelInfo *resultRelInfo, EState *estate, C #define DatumGetItemPointer(X) ((ItemPointer)DatumGetPointer(X)) #endif +static TupleTableSlot* ExecModifyTable(PlanState* state); extern CopyFromManager initCopyFromManager(MemoryContext parent, Relation heapRel, bool isInsertSelect); extern void deinitCopyFromManager(CopyFromManager mgr); extern void FlushInsertSelectBulk( @@ -3310,8 +3311,9 @@ static TupleTableSlot* ExecReplace(EState* estate, ModifyTableState* node, Tuple * if needed. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecModifyTable(ModifyTableState* node) +static TupleTableSlot* ExecModifyTable(PlanState* state) { + ModifyTableState* node = castNode(ModifyTableState, state); EState* estate = node->ps.state; CmdType operation = node->operation; ResultRelInfo* saved_result_rel_info = NULL; @@ -3341,6 +3343,9 @@ TupleTableSlot* ExecModifyTable(ModifyTableState* node) int2 bucketid = InvalidBktId; List *partition_list = NIL; int resultRelationNum = node->mt_ResultTupleSlots ? list_length(node->mt_ResultTupleSlots) : 1; + + CHECK_FOR_INTERRUPTS(); + /* * This should NOT get called during EvalPlanQual; we should have passed a * subplan tree to EvalPlanQual, instead. Use a runtime test not just @@ -3814,6 +3819,7 @@ ModifyTableState* ExecInitModifyTable(ModifyTable* node, EState* estate, int efl mt_state->ps.plan = (Plan*)node; mt_state->ps.state = estate; mt_state->ps.targetlist = NIL; /* not actually used */ + mt_state->ps.ExecProcNode = ExecModifyTable; mt_state->operation = operation; mt_state->canSetTag = node->canSetTag; diff --git a/src/gausskernel/runtime/executor/nodeNestloop.cpp b/src/gausskernel/runtime/executor/nodeNestloop.cpp index fc85f0574..dfe3e12f0 100644 --- a/src/gausskernel/runtime/executor/nodeNestloop.cpp +++ b/src/gausskernel/runtime/executor/nodeNestloop.cpp @@ -28,6 +28,7 @@ #include "utils/memutils.h" #include "executor/node/nodeHashjoin.h" +static TupleTableSlot* ExecNestLoop(PlanState* state); static void MaterialAll(PlanState* node) { if (IsA(node, MaterialState)) { @@ -81,8 +82,9 @@ static void MaterialAll(PlanState* node) * are prepared to return the first tuple. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecNestLoop(NestLoopState* node) +static TupleTableSlot* ExecNestLoop(PlanState* state) { + NestLoopState* node = castNode(NestLoopState, state); TupleTableSlot* outer_tuple_slot = NULL; TupleTableSlot* inner_tuple_slot = NULL; ListCell* lc = NULL; @@ -99,6 +101,8 @@ TupleTableSlot* ExecNestLoop(NestLoopState* node) PlanState* inner_plan = innerPlanState(node); ExprContext* econtext = node->js.ps.ps_ExprContext; + CHECK_FOR_INTERRUPTS(); + /* * 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 @@ -340,6 +344,7 @@ NestLoopState* ExecInitNestLoop(NestLoop* node, EState* estate, int eflags) nlstate->js.ps.plan = (Plan*)node; nlstate->js.ps.state = estate; nlstate->nl_MaterialAll = node->materialAll; + nlstate->js.ps.ExecProcNode = ExecNestLoop; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodePartIterator.cpp b/src/gausskernel/runtime/executor/nodePartIterator.cpp index fcd1e2136..89a239f21 100755 --- a/src/gausskernel/runtime/executor/nodePartIterator.cpp +++ b/src/gausskernel/runtime/executor/nodePartIterator.cpp @@ -33,6 +33,7 @@ #include "nodes/plannodes.h" #include "vecexecutor/vecnodes.h" +static TupleTableSlot* ExecPartIterator(PlanState* state); /* * @@GaussDB@@ * Target : data partition @@ -47,6 +48,7 @@ PartIteratorState* ExecInitPartIterator(PartIterator* node, EState* estate, int state = makeNode(PartIteratorState); state->ps.plan = (Plan*)node; state->ps.state = estate; + state->ps.ExecProcNode = ExecPartIterator; /* initiate sub node */ state->ps.lefttree = ExecInitNode(node->plan.lefttree, estate, eflags); @@ -150,8 +152,9 @@ static void InitScanPartition(PartIteratorState* node, int partitionScan, PlanSt * : table. it is like a monitor. The real job is done by SeqScan .e.g * Notes : */ -TupleTableSlot* ExecPartIterator(PartIteratorState* node) +static TupleTableSlot* ExecPartIterator(PlanState* planState) { + PartIteratorState* node = castNode(PartIteratorState, planState); TupleTableSlot* slot = NULL; EState* state = node->ps.lefttree->state; node->ps.lefttree->do_not_reset_rownum = true; @@ -167,6 +170,8 @@ TupleTableSlot* ExecPartIterator(PartIteratorState* node) return NULL; } + CHECK_FOR_INTERRUPTS(); + /* init first scanned partition */ if (node->currentItr == -1) InitScanPartition(node, partitionScan, noden); @@ -206,6 +211,8 @@ TupleTableSlot* ExecPartIterator(PartIteratorState* node) /* switch to next partiiton */ InitScanPartition(node, partitionScan, noden); + CHECK_FOR_INTERRUPTS(); + /* For partition wise join, can not early free left tree's caching memory */ orig_early_free = state->es_skip_early_free; state->es_skip_early_free = true; diff --git a/src/gausskernel/runtime/executor/nodeRecursiveunion.cpp b/src/gausskernel/runtime/executor/nodeRecursiveunion.cpp index 3201c60cc..3a7e66d10 100644 --- a/src/gausskernel/runtime/executor/nodeRecursiveunion.cpp +++ b/src/gausskernel/runtime/executor/nodeRecursiveunion.cpp @@ -55,6 +55,7 @@ THR_LOCAL int global_iteration = 0; +static TupleTableSlot* ExecRecursiveUnion(PlanState* state); static SyncController* create_stream_synccontroller(Stream* stream_node); static SyncController* create_recursiveunion_synccontroller(RecursiveUnion* ru_node); static List* getSpecialSubPlanStateNodes(const PlanState* node); @@ -165,8 +166,9 @@ static void markIterationStats(RecursiveUnionState* node, bool isSW) * 2.6 go back to 2.2 * ---------------------------------------------------------------- */ -TupleTableSlot* ExecRecursiveUnion(RecursiveUnionState* node) +static TupleTableSlot* ExecRecursiveUnion(PlanState* state) { + RecursiveUnionState* node = castNode(RecursiveUnionState, state); PlanState* outer_plan = outerPlanState(node); PlanState* inner_plan = innerPlanState(node); RecursiveUnion* plan = (RecursiveUnion*)node->ps.plan; @@ -182,6 +184,8 @@ TupleTableSlot* ExecRecursiveUnion(RecursiveUnionState* node) } } + CHECK_FOR_INTERRUPTS(); + /* 1. Evaluate non-recursive term */ if (!node->recursing) { for (;;) { @@ -504,6 +508,7 @@ RecursiveUnionState* ExecInitRecursiveUnion(RecursiveUnion* node, EState* estate rustate->hashtable = NULL; rustate->tempContext = NULL; rustate->tableContext = NULL; + rustate->ps.ExecProcNode = ExecRecursiveUnion; /* initialize processing state */ rustate->recursing = false; diff --git a/src/gausskernel/runtime/executor/nodeResult.cpp b/src/gausskernel/runtime/executor/nodeResult.cpp index a71daa744..db482c403 100644 --- a/src/gausskernel/runtime/executor/nodeResult.cpp +++ b/src/gausskernel/runtime/executor/nodeResult.cpp @@ -62,14 +62,17 @@ * 'nil' if the constant qualification is not satisfied. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecResult(ResultState* node) +static TupleTableSlot* ExecResult(PlanState* state) { + ResultState* node = castNode(ResultState, state); TupleTableSlot* outer_tuple_slot = NULL; TupleTableSlot* result_slot = NULL; PlanState* outer_plan = NULL; ExprDoneCond is_done; ExprContext* econtext = node->ps.ps_ExprContext; + CHECK_FOR_INTERRUPTS(); + /* * check constant qualifications like (2 > 1), if not already done */ @@ -223,6 +226,7 @@ ResultState* ExecInitResult(BaseResult* node, EState* estate, int eflags) ResultState* resstate = makeNode(ResultState); resstate->ps.plan = (Plan*)node; resstate->ps.state = estate; + resstate->ps.ExecProcNode = ExecResult; resstate->rs_done = false; resstate->rs_checkqual = (node->resconstantqual == NULL) ? false : true; diff --git a/src/gausskernel/runtime/executor/nodeSeqscan.cpp b/src/gausskernel/runtime/executor/nodeSeqscan.cpp index e95999450..edef9820e 100644 --- a/src/gausskernel/runtime/executor/nodeSeqscan.cpp +++ b/src/gausskernel/runtime/executor/nodeSeqscan.cpp @@ -51,6 +51,7 @@ #include "optimizer/var.h" #include "optimizer/tlist.h" +static TupleTableSlot* ExecSeqScan(PlanState* state); extern void StrategyGetRingPrefetchQuantityAndTrigger(BufferAccessStrategy strategy, int* quantity, int* trigger); /* ---------------------------------------------------------------- * prefetch_pages @@ -374,9 +375,10 @@ static ScanBatchResult *SeqNextBatchMode(SeqScanState *node) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecSeqScan(SeqScanState* node) +static TupleTableSlot* ExecSeqScan(PlanState* state) { - if (node->scanBatchMode) { + SeqScanState* node = castNode(SeqScanState, state); + if (unlikely(node->scanBatchMode)) { return (TupleTableSlot *)SeqNextBatchMode(node); } else { return ExecScan((ScanState *) node, node->ScanNextMtd, (ExecScanRecheckMtd) SeqRecheck); @@ -948,6 +950,7 @@ SeqScanState* ExecInitSeqScan(SeqScan* node, EState* estate, int eflags) scanstate->currentSlot = 0; scanstate->partScanDirection = node->partScanDirection; scanstate->rangeScanInRedis = {false,0,0}; + scanstate->ps.ExecProcNode = ExecSeqScan; if (!node->tablesample) { scanstate->isSampleScan = false; diff --git a/src/gausskernel/runtime/executor/nodeSetOp.cpp b/src/gausskernel/runtime/executor/nodeSetOp.cpp index c3bec9a5d..6a50ae45b 100644 --- a/src/gausskernel/runtime/executor/nodeSetOp.cpp +++ b/src/gausskernel/runtime/executor/nodeSetOp.cpp @@ -80,6 +80,7 @@ typedef struct SetOpHashEntryData { SetOpStatePerGroupData pergroup; } SetOpHashEntryData; +static TupleTableSlot* ExecSetOp(PlanState* state); static TupleTableSlot* setop_retrieve_direct(SetOpState* setopstate); static void setop_fill_hash_table(SetOpState* setopstate); static TupleTableSlot* setop_retrieve_hash_table(SetOpState* setopstate); @@ -188,11 +189,14 @@ static void set_output_count(SetOpState* setopstate, SetOpStatePerGroup pergroup * ---------------------------------------------------------------- */ /* return: a tuple or NULL */ -TupleTableSlot* ExecSetOp(SetOpState* node) +static TupleTableSlot* ExecSetOp(PlanState* state) { + SetOpState* node = castNode(SetOpState, state); SetOp* plan_node = (SetOp*)node->ps.plan; TupleTableSlot* result_tuple_slot = node->ps.ps_ResultTupleSlot; + CHECK_FOR_INTERRUPTS(); + /* * If the previously-returned tuple needs to be returned more than once, * keep returning it. @@ -583,6 +587,7 @@ SetOpState* ExecInitSetOp(SetOp* node, EState* estate, int eflags) SetOpState* setopstate = makeNode(SetOpState); setopstate->ps.plan = (Plan*)node; setopstate->ps.state = estate; + setopstate->ps.ExecProcNode = ExecSetOp; setopstate->eqfunctions = NULL; setopstate->hashfunctions = NULL; diff --git a/src/gausskernel/runtime/executor/nodeSort.cpp b/src/gausskernel/runtime/executor/nodeSort.cpp index e2cb8b8d2..3a1a502da 100644 --- a/src/gausskernel/runtime/executor/nodeSort.cpp +++ b/src/gausskernel/runtime/executor/nodeSort.cpp @@ -32,6 +32,7 @@ #include "pgxc/pgxc.h" #endif +static TupleTableSlot* ExecSort(PlanState* state); /* ---------------------------------------------------------------- * ExecSort * @@ -46,10 +47,13 @@ * -- the outer child is prepared to return the first tuple. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecSort(SortState* node) +static TupleTableSlot* ExecSort(PlanState* state) { + SortState* node = castNode(SortState, state); TupleTableSlot* slot = NULL; + CHECK_FOR_INTERRUPTS(); + /* * get state info from node */ @@ -226,6 +230,7 @@ SortState* ExecInitSort(Sort* node, EState* estate, int eflags) SortState* sortstate = makeNode(SortState); sortstate->ss.ps.plan = (Plan*)node; sortstate->ss.ps.state = estate; + sortstate->ss.ps.ExecProcNode = ExecSort; /* * We must have random access to the sort output to do backward scan or diff --git a/src/gausskernel/runtime/executor/nodeStartWithOp.cpp b/src/gausskernel/runtime/executor/nodeStartWithOp.cpp index 3489f89d5..c5c86726e 100644 --- a/src/gausskernel/runtime/executor/nodeStartWithOp.cpp +++ b/src/gausskernel/runtime/executor/nodeStartWithOp.cpp @@ -46,6 +46,7 @@ typedef enum StartWithOpExecStatus { char* get_typename(Oid typid); +static TupleTableSlot* ExecStartWithOp(PlanState* state); static void ProcessPseudoReturnColumns(StartWithOpState *state); static AttrNumber FetchRUItrTargetEntryResno(StartWithOpState *state); @@ -252,6 +253,8 @@ StartWithOpState* ExecInitStartWithOp(StartWithOp* node, EState* estate, int efl ALLOCSET_DEFAULT_INITSIZE, ALLOCSET_DEFAULT_MAXSIZE); + state->ps.ExecProcNode = ExecStartWithOp; + /* * Miscellaneous initialization * @@ -605,8 +608,9 @@ TupleTableSlot* GetStartWithSlot(RecursiveUnionState* node, TupleTableSlot* slot return dstSlot; } -TupleTableSlot* ExecStartWithOp(StartWithOpState *node) +static TupleTableSlot* ExecStartWithOp(PlanState* state) { + StartWithOpState *node = castNode(StartWithOpState, state); TupleTableSlot *dstSlot = node->ps.ps_ResultTupleSlot; PlanState *outerNode = outerPlanState(node); StartWithOp *swplan = (StartWithOp *)node->ps.plan; diff --git a/src/gausskernel/runtime/executor/nodeSubplan.cpp b/src/gausskernel/runtime/executor/nodeSubplan.cpp index 87ad1dc7b..3824854d9 100644 --- a/src/gausskernel/runtime/executor/nodeSubplan.cpp +++ b/src/gausskernel/runtime/executor/nodeSubplan.cpp @@ -47,6 +47,8 @@ static Datum ExecSubPlan(SubPlanState* node, ExprContext* econtext, bool* isNull ScanDirection direction; Datum retval; + CHECK_FOR_INTERRUPTS(); + /* Set default values for result flags: non-null, not a set result */ *isNull = false; if (isDone != NULL) { @@ -571,6 +573,9 @@ bool findPartialMatch(TupleHashTable hashtable, TupleTableSlot* slot, FmgrInfo* InitTupleHashIterator(hashtable, &hashiter); while ((entry = ScanTupleHashTable(&hashiter)) != NULL) { + + CHECK_FOR_INTERRUPTS(); + ExecStoreMinimalTuple(entry->firstTuple, hashtable->tableslot, false); if (!execTuplesUnequal(slot, hashtable->tableslot, num_cols, key_col_idx, eqfunctions, hashtable->tempcxt)) { TermTupleHashIterator(&hashiter); diff --git a/src/gausskernel/runtime/executor/nodeSubqueryscan.cpp b/src/gausskernel/runtime/executor/nodeSubqueryscan.cpp index 4fbf0e567..fb4c9c041 100644 --- a/src/gausskernel/runtime/executor/nodeSubqueryscan.cpp +++ b/src/gausskernel/runtime/executor/nodeSubqueryscan.cpp @@ -30,6 +30,7 @@ #include "executor/exec/execdebug.h" #include "executor/node/nodeSubqueryscan.h" +static TupleTableSlot* ExecSubqueryScan(PlanState* state); static TupleTableSlot* SubqueryNext(SubqueryScanState* node); /* ---------------------------------------------------------------- @@ -77,8 +78,9 @@ static bool SubqueryRecheck(SubqueryScanState* node, TupleTableSlot* slot) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecSubqueryScan(SubqueryScanState* node) +static TupleTableSlot* ExecSubqueryScan(PlanState* state) { + SubqueryScanState* node = castNode(SubqueryScanState, state); return ExecScan(&node->ss, (ExecScanAccessMtd)SubqueryNext, (ExecScanRecheckMtd)SubqueryRecheck); } @@ -101,6 +103,8 @@ SubqueryScanState* ExecInitSubqueryScan(SubqueryScan* node, EState* estate, int SubqueryScanState* sub_query_state = makeNode(SubqueryScanState); sub_query_state->ss.ps.plan = (Plan*)node; sub_query_state->ss.ps.state = estate; + sub_query_state->ss.ps.ExecProcNode = ExecSubqueryScan; + /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeTidscan.cpp b/src/gausskernel/runtime/executor/nodeTidscan.cpp index e68796a53..5444b6d39 100644 --- a/src/gausskernel/runtime/executor/nodeTidscan.cpp +++ b/src/gausskernel/runtime/executor/nodeTidscan.cpp @@ -44,6 +44,7 @@ ((node) != NULL && IsA((node), Var) && ((Var*)(node))->varattno == SelfItemPointerAttributeNumber && \ ((Var*)(node))->varlevelsup == 0) +static TupleTableSlot* ExecTidScan(PlanState* state); static void TidListCreate(TidScanState* tidstate, bool isBucket); static int ItemptrComparator(const void* a, const void* b); static TupleTableSlot* TidNext(TidScanState* node); @@ -293,6 +294,8 @@ static TupleTableSlot* HbktTidFetchTuple(TidScanState* node, bool bBackward) /* Reset the tidPtr */ node->tss_TidPtr = -1; + + CHECK_FOR_INTERRUPTS(); } return ExecClearTuple(slot); @@ -466,8 +469,9 @@ static bool TidRecheck(TidScanState* node, TupleTableSlot* slot) * -- tidPtr is -1. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecTidScan(TidScanState* node) +static TupleTableSlot* ExecTidScan(PlanState* state) { + TidScanState* node = castNode(TidScanState, state); return ExecScan(&node->ss, (ExecScanAccessMtd)TidNext, (ExecScanRecheckMtd)TidRecheck); } @@ -583,6 +587,7 @@ TidScanState* ExecInitTidScan(TidScan* node, EState* estate, int eflags) tidstate->ss.isPartTbl = node->scan.isPartTbl; tidstate->ss.currentSlot = 0; tidstate->ss.partScanDirection = node->scan.partScanDirection; + tidstate->ss.ps.ExecProcNode = ExecTidScan; tidstate->tss_htup.tupTableType = HEAP_TUPLE; /* diff --git a/src/gausskernel/runtime/executor/nodeTrainModel.cpp b/src/gausskernel/runtime/executor/nodeTrainModel.cpp index 9c9e1af4a..d1d8b3c8c 100644 --- a/src/gausskernel/runtime/executor/nodeTrainModel.cpp +++ b/src/gausskernel/runtime/executor/nodeTrainModel.cpp @@ -29,6 +29,8 @@ #include "executor/node/nodeTrainModel.h" #include "db4ai/db4ai_api.h" +static TupleTableSlot* ExecTrainModel(PlanState* state); + static bool ExecFetchTrainModel(void *callback_data, ModelTuple * tuple) { TrainModelState *pstate = (TrainModelState*)callback_data; @@ -92,6 +94,7 @@ TrainModelState* ExecInitTrainModel(TrainModel* pnode, EState* estate, int eflag pstate->config = pnode; pstate->algorithm = palgo; pstate->finished = 0; + pstate->ss.ps.ExecProcNode = ExecTrainModel; // Tuple table initialization ExecInitScanTupleSlot(estate, &pstate->ss); @@ -140,8 +143,9 @@ TrainModelState* ExecInitTrainModel(TrainModel* pnode, EState* estate, int eflag return pstate; } -TupleTableSlot* ExecTrainModel(TrainModelState* pstate) +static TupleTableSlot* ExecTrainModel(PlanState* state) { + TrainModelState* pstate = castNode(TrainModelState, state); // check if already finished if (pstate->finished == pstate->config->configurations) return NULL; diff --git a/src/gausskernel/runtime/executor/nodeUnique.cpp b/src/gausskernel/runtime/executor/nodeUnique.cpp index 4c3bd519b..ad1bf1fd8 100644 --- a/src/gausskernel/runtime/executor/nodeUnique.cpp +++ b/src/gausskernel/runtime/executor/nodeUnique.cpp @@ -41,8 +41,9 @@ * ExecUnique * ---------------------------------------------------------------- */ -TupleTableSlot* ExecUnique(UniqueState* node) /* return: a tuple or NULL */ +static TupleTableSlot* ExecUnique(PlanState* state) /* return: a tuple or NULL */ { + UniqueState* node = castNode(UniqueState, state); Unique* plan_node = (Unique*)node->ps.plan; TupleTableSlot* slot = NULL; @@ -58,6 +59,7 @@ TupleTableSlot* ExecUnique(UniqueState* node) /* return: a tuple or NULL */ * first tuple of each group is returned. */ for (;;) { + CHECK_FOR_INTERRUPTS(); /* * fetch a tuple from the outer subplan */ @@ -112,6 +114,7 @@ UniqueState* ExecInitUnique(Unique* node, EState* estate, int eflags) unique_state->ps.plan = (Plan*)node; unique_state->ps.state = estate; + unique_state->ps.ExecProcNode = ExecUnique; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeValuesscan.cpp b/src/gausskernel/runtime/executor/nodeValuesscan.cpp index 6b25c686b..3b80328c1 100644 --- a/src/gausskernel/runtime/executor/nodeValuesscan.cpp +++ b/src/gausskernel/runtime/executor/nodeValuesscan.cpp @@ -28,6 +28,7 @@ #include "executor/node/nodeValuesscan.h" #include "parser/parsetree.h" +static TupleTableSlot* ExecValuesScan(PlanState* state); static TupleTableSlot* ValuesNext(ValuesScanState* node); /* ---------------------------------------------------------------- @@ -177,8 +178,9 @@ static bool ValuesRecheck(ValuesScanState* node, TupleTableSlot* slot) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecValuesScan(ValuesScanState* node) +static TupleTableSlot* ExecValuesScan(PlanState* state) { + ValuesScanState* node = castNode(ValuesScanState, state); return ExecScan(&node->ss, (ExecScanAccessMtd)ValuesNext, (ExecScanRecheckMtd)ValuesRecheck); } @@ -206,6 +208,7 @@ ValuesScanState* ExecInitValuesScan(ValuesScan* node, EState* estate, int eflags scan_state->ss.ps.plan = (Plan*)node; scan_state->ss.ps.state = estate; + scan_state->ss.ps.ExecProcNode = ExecValuesScan; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/executor/nodeWindowAgg.cpp b/src/gausskernel/runtime/executor/nodeWindowAgg.cpp index d40c7879c..0694a1a2b 100644 --- a/src/gausskernel/runtime/executor/nodeWindowAgg.cpp +++ b/src/gausskernel/runtime/executor/nodeWindowAgg.cpp @@ -52,6 +52,7 @@ #include "utils/syscache.h" #include "windowapi.h" +static TupleTableSlot* ExecWindowAgg(PlanState* state); static void initialize_windowaggregate( WindowAggState* winstate, WindowStatePerFunc perfuncstate, WindowStatePerAgg peraggstate); static void advance_windowaggregate( @@ -951,14 +952,17 @@ static void update_frametailpos(WindowObject winobj, TupleTableSlot* slot) * (ignoring the case of SRFs in the targetlist, that is). * ----------------- */ -TupleTableSlot* ExecWindowAgg(WindowAggState* winstate) +static TupleTableSlot* ExecWindowAgg(PlanState* state) { + WindowAggState* winstate = castNode(WindowAggState, state); TupleTableSlot* result = NULL; ExprDoneCond is_done; ExprContext* econtext = NULL; int i; int numfuncs; + CHECK_FOR_INTERRUPTS(); + if (winstate->all_done) return NULL; @@ -1157,6 +1161,7 @@ WindowAggState* ExecInitWindowAgg(WindowAgg* node, EState* estate, int eflags) WindowAggState* winstate = makeNode(WindowAggState); winstate->ss.ps.plan = (Plan*)node; winstate->ss.ps.state = estate; + winstate->ss.ps.ExecProcNode = ExecWindowAgg; int64 operator_mem = SET_NODEMEM(((Plan*)node)->operatorMemKB[0], ((Plan*)node)->dop); AllocSetContext* set = (AllocSetContext*)(estate->es_query_cxt); diff --git a/src/gausskernel/runtime/executor/nodeWorktablescan.cpp b/src/gausskernel/runtime/executor/nodeWorktablescan.cpp index 7469b0b42..49cbeeec7 100755 --- a/src/gausskernel/runtime/executor/nodeWorktablescan.cpp +++ b/src/gausskernel/runtime/executor/nodeWorktablescan.cpp @@ -20,6 +20,7 @@ #include "executor/node/nodeRecursiveunion.h" #include "executor/node/nodeWorktablescan.h" +static TupleTableSlot* ExecWorkTableScan(PlanState* node); static TupleTableSlot* WorkTableScanNext(WorkTableScanState* node); /* ---------------------------------------------------------------- @@ -127,8 +128,9 @@ static bool WorkTableScanRecheck(WorkTableScanState* node, TupleTableSlot* slot) * access method functions. * ---------------------------------------------------------------- */ -TupleTableSlot* ExecWorkTableScan(WorkTableScanState* node) +static TupleTableSlot* ExecWorkTableScan(PlanState* state) { + WorkTableScanState* node = castNode(WorkTableScanState, state); /* * On the first call, find the ancestor RecursiveUnion's state via the * Param slot reserved for it. (We can't do this during node init because @@ -234,6 +236,7 @@ WorkTableScanState* ExecInitWorkTableScan(WorkTableScan* node, EState* estate, i scan_state->ss.ps.plan = (Plan*)node; scan_state->ss.ps.state = estate; scan_state->rustate = NULL; /* we'll set this later */ + scan_state->ss.ps.ExecProcNode = ExecWorkTableScan; /* * Miscellaneous initialization diff --git a/src/gausskernel/runtime/vecexecutor/vecexecutor.cpp b/src/gausskernel/runtime/vecexecutor/vecexecutor.cpp index 1759c6412..eb3f5e56a 100644 --- a/src/gausskernel/runtime/vecexecutor/vecexecutor.cpp +++ b/src/gausskernel/runtime/vecexecutor/vecexecutor.cpp @@ -142,7 +142,7 @@ static bool NeedStub(const Plan* node) VectorBatch* VectorEngine(PlanState* node) { VectorBatch* result = NULL; - MemoryContext old_context; + MemoryContext old_context = NULL; CHECK_FOR_INTERRUPTS(); @@ -156,7 +156,9 @@ VectorBatch* VectorEngine(PlanState* node) #endif Assert(node->vectorized); - old_context = MemoryContextSwitchTo(node->nodeContext); + if (node->nodeContext) { + old_context = MemoryContextSwitchTo(node->nodeContext); + } if (node->chgParam != NULL) /* something changed */ VecExecReScan(node); /* let ReScan handle this */ @@ -204,8 +206,9 @@ VectorBatch* VectorEngine(PlanState* node) node->instrument->status = true; } - (void)MemoryContextSwitchTo(old_context); - + if (old_context) { + (void)MemoryContextSwitchTo(old_context); + } return result; } diff --git a/src/gausskernel/runtime/vecexecutor/vecnode/vectortorow.cpp b/src/gausskernel/runtime/vecexecutor/vecnode/vectortorow.cpp index ffb4e00cf..96d43c574 100644 --- a/src/gausskernel/runtime/vecexecutor/vecnode/vectortorow.cpp +++ b/src/gausskernel/runtime/vecexecutor/vecnode/vectortorow.cpp @@ -39,6 +39,9 @@ #include "vecexecutor/vecexecutor.h" #include "storage/item/itemptr.h" + +static TupleTableSlot* ExecVecToRow(PlanState* pstate); + /* Convert one column of the entire batch from vector store to row store. * typid in template is the OID of the column data type. */ template @@ -140,8 +143,9 @@ void DevectorizeOneBatch(VecToRowState* state) return; } -TupleTableSlot* ExecVecToRow(VecToRowState* state) /* return: a tuple or NULL */ +static TupleTableSlot* ExecVecToRow(PlanState* pstate) /* return: a tuple or NULL */ { + VecToRowState* state = castNode(VecToRowState, pstate); PlanState* outer_plan = NULL; TupleTableSlot* tuple = state->tts; VectorBatch* current_batch = NULL; @@ -220,6 +224,7 @@ VecToRowState* ExecInitVecToRow(VecToRow* node, EState* estate, int eflags) state->ps.plan = (Plan*)node; state->ps.state = estate; state->ps.vectorized = false; + state->ps.ExecProcNode = ExecVecToRow; /* * tuple table initialization diff --git a/src/include/executor/exec/execStream.h b/src/include/executor/exec/execStream.h index 165b407ae..af2131941 100644 --- a/src/include/executor/exec/execStream.h +++ b/src/include/executor/exec/execStream.h @@ -118,7 +118,6 @@ typedef struct StreamState { } StreamState; extern StreamState* ExecInitStream(Stream* node, EState* estate, int eflags); -extern TupleTableSlot* ExecStream(StreamState* node); extern void ExecEndStream(StreamState* node); extern int HandleStreamResponse(PGXCNodeHandle* conn, StreamState* node); extern void StreamPrepareRequest(StreamState* node); diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h index 1d39bfc87..beb38e97e 100644 --- a/src/include/executor/executor.h +++ b/src/include/executor/executor.h @@ -252,10 +252,9 @@ extern void EvalPlanQualBegin(EPQState* epqstate, EState* parentestate, bool isU extern void EvalPlanQualEnd(EPQState* epqstate); /* - * prototypes from functions in execProcnode.c + * functions in execProcnode.c */ extern PlanState* ExecInitNode(Plan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecProcNode(PlanState* node); extern Node* MultiExecProcNode(PlanState* node); extern void ExecEndNode(PlanState* node); extern bool NeedStubExecution(Plan* plan); @@ -263,6 +262,46 @@ extern TupleTableSlot* FetchPlanSlot(PlanState* subPlanState, ProjectionInfo** p extern long ExecGetPlanMemCost(Plan* node); +/* ---------------------------------------------------------------- + * ExecProcNode + * + * Execute the given node to return a(nother) tuple. + * ---------------------------------------------------------------- + */ +#ifndef FRONTEND +#ifndef ENABLE_MULTIPLE_NODES + +static inline TupleTableSlot *ExecProcNode(PlanState *node) +{ + TupleTableSlot* result; + Assert(node->ExecProcNode); + if (unlikely(node->nodeContext)) { + MemoryContext old_context = MemoryContextSwitchTo(node->nodeContext); /* Switch to Node Level Memory Context */ + if (node->chgParam != NULL) /* something changed? */ + ExecReScan(node); /* let ReScan handle this */ + result = node->ExecProcNode(node); + MemoryContextSwitchTo(old_context); + } else { + if (node->chgParam != NULL) /* something changed? */ + ExecReScan(node); + result = node->ExecProcNode(node); + } + node->ps_rownum++; + return result; +} +#else /*ENABLE_MULTIPLE_NODES*/ + +static inline TupleTableSlot *ExecProcNode(PlanState *node) +{ + //TODO: FIX ENABLE_MULTIPLE_NODES + return NULL; +} + +#endif /*ENABLE_MULTIPLE_NODES*/ + +#endif /*FRONTEND*/ + + /* * prototypes from functions in execQual.c */ diff --git a/src/include/executor/node/nodeAgg.h b/src/include/executor/node/nodeAgg.h index 24ab9f5be..b00884f74 100644 --- a/src/include/executor/node/nodeAgg.h +++ b/src/include/executor/node/nodeAgg.h @@ -268,7 +268,6 @@ typedef struct AggHashEntryData { } AggHashEntryData; /* VARIABLE LENGTH STRUCT */ extern AggState* ExecInitAgg(Agg* node, EState* estate, int eflags); -extern TupleTableSlot* ExecAgg(AggState* node); extern void ExecEndAgg(AggState* node); extern void ExecReScanAgg(AggState* node); extern Datum GetAggInitVal(Datum textInitVal, Oid transtype); diff --git a/src/include/executor/node/nodeAppend.h b/src/include/executor/node/nodeAppend.h index 2aa5a6a83..3ae958738 100644 --- a/src/include/executor/node/nodeAppend.h +++ b/src/include/executor/node/nodeAppend.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern AppendState* ExecInitAppend(Append* node, EState* estate, int eflags); -extern TupleTableSlot* ExecAppend(AppendState* node); extern void ExecEndAppend(AppendState* node); extern void ExecReScanAppend(AppendState* node); extern bool exec_append_initialize_next(AppendState* appendstate); diff --git a/src/include/executor/node/nodeBitmapHeapscan.h b/src/include/executor/node/nodeBitmapHeapscan.h index 63c37574f..09c156020 100644 --- a/src/include/executor/node/nodeBitmapHeapscan.h +++ b/src/include/executor/node/nodeBitmapHeapscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern BitmapHeapScanState* ExecInitBitmapHeapScan(BitmapHeapScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecBitmapHeapScan(BitmapHeapScanState* node); extern void ExecEndBitmapHeapScan(BitmapHeapScanState* node); extern void ExecReScanBitmapHeapScan(BitmapHeapScanState* node); diff --git a/src/include/executor/node/nodeCtescan.h b/src/include/executor/node/nodeCtescan.h index 8287c9022..a3d281646 100644 --- a/src/include/executor/node/nodeCtescan.h +++ b/src/include/executor/node/nodeCtescan.h @@ -18,7 +18,6 @@ #include "nodes/execnodes.h" extern CteScanState* ExecInitCteScan(CteScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecCteScan(CteScanState* node); extern void ExecEndCteScan(CteScanState* node); extern void ExecReScanCteScan(CteScanState* node); @@ -38,7 +37,6 @@ typedef struct StartWithFuncEvalState } StartWithFuncEvalState; extern StartWithOpState* ExecInitStartWithOp(StartWithOp* node, EState* estate, int eflags); -extern TupleTableSlot* ExecStartWithOp(StartWithOpState *node); extern void ExecEndStartWithOp(StartWithOpState *node); extern void ExecReScanStartWithOp(StartWithOpState *node); diff --git a/src/include/executor/node/nodeExtensible.h b/src/include/executor/node/nodeExtensible.h index f4b08b6df..e16f9b4e7 100644 --- a/src/include/executor/node/nodeExtensible.h +++ b/src/include/executor/node/nodeExtensible.h @@ -31,7 +31,6 @@ * General executor code */ extern ExtensiblePlanState* ExecInitExtensiblePlan(ExtensiblePlan* eplan, EState* estate, int eflags); -extern TupleTableSlot* ExecExtensiblePlan(ExtensiblePlanState* node); extern void ExecEndExtensiblePlan(ExtensiblePlanState* node); extern void ExecReScanExtensiblePlan(ExtensiblePlanState* node); diff --git a/src/include/executor/node/nodeForeignscan.h b/src/include/executor/node/nodeForeignscan.h index 4c4658a33..b91574d4b 100644 --- a/src/include/executor/node/nodeForeignscan.h +++ b/src/include/executor/node/nodeForeignscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern ForeignScanState* ExecInitForeignScan(ForeignScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecForeignScan(ForeignScanState* node); extern void ExecEndForeignScan(ForeignScanState* node); extern void ExecReScanForeignScan(ForeignScanState* node); diff --git a/src/include/executor/node/nodeFunctionscan.h b/src/include/executor/node/nodeFunctionscan.h index ea6ae29f3..b82eb59ba 100644 --- a/src/include/executor/node/nodeFunctionscan.h +++ b/src/include/executor/node/nodeFunctionscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern FunctionScanState* ExecInitFunctionScan(FunctionScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecFunctionScan(FunctionScanState* node); extern void ExecEndFunctionScan(FunctionScanState* node); extern void ExecReScanFunctionScan(FunctionScanState* node); diff --git a/src/include/executor/node/nodeGroup.h b/src/include/executor/node/nodeGroup.h index af3079d71..62df19c50 100644 --- a/src/include/executor/node/nodeGroup.h +++ b/src/include/executor/node/nodeGroup.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern GroupState* ExecInitGroup(Group* node, EState* estate, int eflags); -extern TupleTableSlot* ExecGroup(GroupState* node); extern void ExecEndGroup(GroupState* node); extern void ExecReScanGroup(GroupState* node); diff --git a/src/include/executor/node/nodeHash.h b/src/include/executor/node/nodeHash.h index a4fa402d5..00165fadd 100644 --- a/src/include/executor/node/nodeHash.h +++ b/src/include/executor/node/nodeHash.h @@ -21,7 +21,6 @@ #define BUCKET_OVERHEAD 8 extern HashState* ExecInitHash(Hash* node, EState* estate, int eflags); -extern TupleTableSlot* ExecHash(void); extern Node* MultiExecHash(HashState* node); extern void ExecEndHash(HashState* node); extern void ExecReScanHash(HashState* node); diff --git a/src/include/executor/node/nodeHashjoin.h b/src/include/executor/node/nodeHashjoin.h index 118d2863b..30da67925 100644 --- a/src/include/executor/node/nodeHashjoin.h +++ b/src/include/executor/node/nodeHashjoin.h @@ -19,7 +19,6 @@ #include "optimizer/planmem_walker.h" extern HashJoinState* ExecInitHashJoin(HashJoin* node, EState* estate, int eflags); -extern TupleTableSlot* ExecHashJoin(HashJoinState* node); extern void ExecEndHashJoin(HashJoinState* node); extern void ExecReScanHashJoin(HashJoinState* node); extern void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue, BufFile** fileptr); diff --git a/src/include/executor/node/nodeIndexonlyscan.h b/src/include/executor/node/nodeIndexonlyscan.h index abe570378..b097bee66 100644 --- a/src/include/executor/node/nodeIndexonlyscan.h +++ b/src/include/executor/node/nodeIndexonlyscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern IndexOnlyScanState* ExecInitIndexOnlyScan(IndexOnlyScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecIndexOnlyScan(IndexOnlyScanState* node); extern void ExecEndIndexOnlyScan(IndexOnlyScanState* node); extern void ExecIndexOnlyMarkPos(IndexOnlyScanState* node); extern void ExecIndexOnlyRestrPos(IndexOnlyScanState* node); diff --git a/src/include/executor/node/nodeIndexscan.h b/src/include/executor/node/nodeIndexscan.h index 4371c36c2..0fdbf67a5 100644 --- a/src/include/executor/node/nodeIndexscan.h +++ b/src/include/executor/node/nodeIndexscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern IndexScanState* ExecInitIndexScan(IndexScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecIndexScan(IndexScanState* node); extern void ExecEndIndexScan(IndexScanState* node); extern void ExecIndexMarkPos(IndexScanState* node); extern void ExecIndexRestrPos(IndexScanState* node); diff --git a/src/include/executor/node/nodeLimit.h b/src/include/executor/node/nodeLimit.h index 2224b9e6d..dc17023ec 100644 --- a/src/include/executor/node/nodeLimit.h +++ b/src/include/executor/node/nodeLimit.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern LimitState* ExecInitLimit(Limit* node, EState* estate, int eflags); -extern TupleTableSlot* ExecLimit(LimitState* node); extern void ExecEndLimit(LimitState* node); extern void ExecReScanLimit(LimitState* node); extern void recompute_limits(LimitState* node); diff --git a/src/include/executor/node/nodeLockRows.h b/src/include/executor/node/nodeLockRows.h index fcd13b6dd..de6e31863 100644 --- a/src/include/executor/node/nodeLockRows.h +++ b/src/include/executor/node/nodeLockRows.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern LockRowsState* ExecInitLockRows(LockRows* node, EState* estate, int eflags); -extern TupleTableSlot* ExecLockRows(LockRowsState* node); extern void ExecEndLockRows(LockRowsState* node); extern void ExecReScanLockRows(LockRowsState* node); diff --git a/src/include/executor/node/nodeMaterial.h b/src/include/executor/node/nodeMaterial.h index 0111a20e9..1960f8b4f 100644 --- a/src/include/executor/node/nodeMaterial.h +++ b/src/include/executor/node/nodeMaterial.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern MaterialState* ExecInitMaterial(Material* node, EState* estate, int eflags); -extern TupleTableSlot* ExecMaterial(MaterialState* node); extern void ExecEndMaterial(MaterialState* node); extern void ExecMaterialMarkPos(MaterialState* node); extern void ExecMaterialRestrPos(MaterialState* node); diff --git a/src/include/executor/node/nodeMergeAppend.h b/src/include/executor/node/nodeMergeAppend.h index 292033efb..f576d43e3 100644 --- a/src/include/executor/node/nodeMergeAppend.h +++ b/src/include/executor/node/nodeMergeAppend.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern MergeAppendState* ExecInitMergeAppend(MergeAppend* node, EState* estate, int eflags); -extern TupleTableSlot* ExecMergeAppend(MergeAppendState* node); extern void ExecEndMergeAppend(MergeAppendState* node); extern void ExecReScanMergeAppend(MergeAppendState* node); diff --git a/src/include/executor/node/nodeMergejoin.h b/src/include/executor/node/nodeMergejoin.h index f6e61ce3c..f78987db2 100644 --- a/src/include/executor/node/nodeMergejoin.h +++ b/src/include/executor/node/nodeMergejoin.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern MergeJoinState* ExecInitMergeJoin(MergeJoin* node, EState* estate, int eflags); -extern TupleTableSlot* ExecMergeJoin(MergeJoinState* node); extern void ExecEndMergeJoin(MergeJoinState* node); extern void ExecReScanMergeJoin(MergeJoinState* node); diff --git a/src/include/executor/node/nodeModifyTable.h b/src/include/executor/node/nodeModifyTable.h index 99098b5e6..46cc71c00 100644 --- a/src/include/executor/node/nodeModifyTable.h +++ b/src/include/executor/node/nodeModifyTable.h @@ -22,7 +22,6 @@ typedef TupleTableSlot* (*ExecUpdateMtd)(ItemPointer, Oid, Oid, HeapTupleHeader, TupleTableSlot*, EPQState*, ModifyTableState*, bool, bool); extern ModifyTableState* ExecInitModifyTable(ModifyTable* node, EState* estate, int eflags); -extern TupleTableSlot* ExecModifyTable(ModifyTableState* node); extern void ExecEndModifyTable(ModifyTableState* node); extern void ExecReScanModifyTable(ModifyTableState* node); extern void RecordDeletedTuple(Oid relid, int2 bucketid, const ItemPointer tupleid, const Relation deldelta_rel); diff --git a/src/include/executor/node/nodeNestloop.h b/src/include/executor/node/nodeNestloop.h index f2519a9f2..4e750dde7 100644 --- a/src/include/executor/node/nodeNestloop.h +++ b/src/include/executor/node/nodeNestloop.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern NestLoopState* ExecInitNestLoop(NestLoop* node, EState* estate, int eflags); -extern TupleTableSlot* ExecNestLoop(NestLoopState* node); extern void ExecEndNestLoop(NestLoopState* node); extern void ExecReScanNestLoop(NestLoopState* node); diff --git a/src/include/executor/node/nodePartIterator.h b/src/include/executor/node/nodePartIterator.h index bef9bca26..b171522b6 100644 --- a/src/include/executor/node/nodePartIterator.h +++ b/src/include/executor/node/nodePartIterator.h @@ -32,7 +32,6 @@ extern void SetPartitionIteratorParamter(PartIteratorState* node, List* subPartLengthList); extern PartIteratorState* ExecInitPartIterator(PartIterator* node, EState* estate, int eflags); -extern TupleTableSlot* ExecPartIterator(PartIteratorState* node); extern void ExecEndPartIterator(PartIteratorState* node); extern void ExecReScanPartIterator(PartIteratorState* node); diff --git a/src/include/executor/node/nodeRecursiveunion.h b/src/include/executor/node/nodeRecursiveunion.h index afe57e4fa..3b9d80f14 100644 --- a/src/include/executor/node/nodeRecursiveunion.h +++ b/src/include/executor/node/nodeRecursiveunion.h @@ -215,7 +215,6 @@ typedef struct RecursiveRefContext { extern void set_recursive_cteplan_ref(Plan* node, RecursiveRefContext* context); extern RecursiveUnionState* ExecInitRecursiveUnion(RecursiveUnion* node, EState* estate, int eflags); -extern TupleTableSlot* ExecRecursiveUnion(RecursiveUnionState* node); extern void ExecEndRecursiveUnion(RecursiveUnionState* node); extern void ExecReScanRecursiveUnion(RecursiveUnionState* node); extern bool IsFirstLevelStreamStateNode(StreamState* node); diff --git a/src/include/executor/node/nodeResult.h b/src/include/executor/node/nodeResult.h index bcf59a3d6..b2b0a9a9b 100644 --- a/src/include/executor/node/nodeResult.h +++ b/src/include/executor/node/nodeResult.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern ResultState* ExecInitResult(BaseResult* node, EState* estate, int eflags); -extern TupleTableSlot* ExecResult(ResultState* node); extern void ExecEndResult(ResultState* node); extern void ExecResultMarkPos(ResultState* node); extern void ExecResultRestrPos(ResultState* node); diff --git a/src/include/executor/node/nodeSeqscan.h b/src/include/executor/node/nodeSeqscan.h index 1a5ab3f4e..bf3e2a41a 100644 --- a/src/include/executor/node/nodeSeqscan.h +++ b/src/include/executor/node/nodeSeqscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern SeqScanState* ExecInitSeqScan(SeqScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecSeqScan(SeqScanState* node); extern void ExecEndSeqScan(SeqScanState* node); extern void ExecSeqMarkPos(SeqScanState* node); extern void ExecSeqRestrPos(SeqScanState* node); diff --git a/src/include/executor/node/nodeSetOp.h b/src/include/executor/node/nodeSetOp.h index bfb8e57a4..8ed3253db 100644 --- a/src/include/executor/node/nodeSetOp.h +++ b/src/include/executor/node/nodeSetOp.h @@ -29,7 +29,6 @@ typedef struct SetopWriteFileControl : public AggWriteFileControl { } SetopWriteFileControl; extern SetOpState* ExecInitSetOp(SetOp* node, EState* estate, int eflags); -extern TupleTableSlot* ExecSetOp(SetOpState* node); extern void ExecEndSetOp(SetOpState* node); extern void ExecReScanSetOp(SetOpState* node); extern void ExecEarlyFreeHashedSetop(SetOpState* node); diff --git a/src/include/executor/node/nodeSort.h b/src/include/executor/node/nodeSort.h index 4f557994b..6b2978e2c 100644 --- a/src/include/executor/node/nodeSort.h +++ b/src/include/executor/node/nodeSort.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern SortState* ExecInitSort(Sort* node, EState* estate, int eflags); -extern TupleTableSlot* ExecSort(SortState* node); extern void ExecEndSort(SortState* node); extern void ExecSortMarkPos(SortState* node); extern void ExecSortRestrPos(SortState* node); diff --git a/src/include/executor/node/nodeSubqueryscan.h b/src/include/executor/node/nodeSubqueryscan.h index 2fd6de555..c52d0caab 100644 --- a/src/include/executor/node/nodeSubqueryscan.h +++ b/src/include/executor/node/nodeSubqueryscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern SubqueryScanState* ExecInitSubqueryScan(SubqueryScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecSubqueryScan(SubqueryScanState* node); extern void ExecEndSubqueryScan(SubqueryScanState* node); extern void ExecReScanSubqueryScan(SubqueryScanState* node); diff --git a/src/include/executor/node/nodeTidscan.h b/src/include/executor/node/nodeTidscan.h index b544c248e..188374bac 100644 --- a/src/include/executor/node/nodeTidscan.h +++ b/src/include/executor/node/nodeTidscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern TidScanState* ExecInitTidScan(TidScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecTidScan(TidScanState* node); extern void ExecEndTidScan(TidScanState* node); extern void ExecTidMarkPos(TidScanState* node); extern void ExecTidRestrPos(TidScanState* node); diff --git a/src/include/executor/node/nodeTrainModel.h b/src/include/executor/node/nodeTrainModel.h index 978aa121b..5be6de353 100644 --- a/src/include/executor/node/nodeTrainModel.h +++ b/src/include/executor/node/nodeTrainModel.h @@ -27,7 +27,6 @@ #include "nodes/execnodes.h" extern TrainModelState* ExecInitTrainModel(TrainModel* node, EState* estate, int eflags); -extern TupleTableSlot* ExecTrainModel(TrainModelState* state); extern void ExecEndTrainModel(TrainModelState* state); #endif /* NODE_TRAIN_MODEL_H */ diff --git a/src/include/executor/node/nodeUnique.h b/src/include/executor/node/nodeUnique.h index 27ff650f2..a763723a0 100644 --- a/src/include/executor/node/nodeUnique.h +++ b/src/include/executor/node/nodeUnique.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern UniqueState* ExecInitUnique(Unique* node, EState* estate, int eflags); -extern TupleTableSlot* ExecUnique(UniqueState* node); extern void ExecEndUnique(UniqueState* node); extern void ExecReScanUnique(UniqueState* node); diff --git a/src/include/executor/node/nodeValuesscan.h b/src/include/executor/node/nodeValuesscan.h index 22f953a5b..8a9fb14e2 100644 --- a/src/include/executor/node/nodeValuesscan.h +++ b/src/include/executor/node/nodeValuesscan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern ValuesScanState* ExecInitValuesScan(ValuesScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecValuesScan(ValuesScanState* node); extern void ExecEndValuesScan(ValuesScanState* node); extern void ExecValuesMarkPos(ValuesScanState* node); extern void ExecValuesRestrPos(ValuesScanState* node); diff --git a/src/include/executor/node/nodeWindowAgg.h b/src/include/executor/node/nodeWindowAgg.h index c5f7ef46b..8faf30bac 100644 --- a/src/include/executor/node/nodeWindowAgg.h +++ b/src/include/executor/node/nodeWindowAgg.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern WindowAggState* ExecInitWindowAgg(WindowAgg* node, EState* estate, int eflags); -extern TupleTableSlot* ExecWindowAgg(WindowAggState* node); extern void ExecEndWindowAgg(WindowAggState* node); extern void ExecReScanWindowAgg(WindowAggState* node); diff --git a/src/include/executor/node/nodeWorktablescan.h b/src/include/executor/node/nodeWorktablescan.h index f91ebd75b..3576410bd 100644 --- a/src/include/executor/node/nodeWorktablescan.h +++ b/src/include/executor/node/nodeWorktablescan.h @@ -17,7 +17,6 @@ #include "nodes/execnodes.h" extern WorkTableScanState* ExecInitWorkTableScan(WorkTableScan* node, EState* estate, int eflags); -extern TupleTableSlot* ExecWorkTableScan(WorkTableScanState* node); extern void ExecEndWorkTableScan(WorkTableScanState* node); extern void ExecReScanWorkTableScan(WorkTableScanState* node); diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 9f62f6fe0..70b5cd11b 100755 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -1229,6 +1229,18 @@ typedef enum { PST_Scan = 2 } PlanStubType; +struct PlanState; + +/* ---------------- + * ExecProcNodeMtd + * + * This is the method called by ExecProcNode to return the next tuple + * from an executor node. It returns NULL, or an empty TupleTableSlot, + * if no more tuples are available. + * ---------------- + */ +typedef TupleTableSlot *(*ExecProcNodeMtd)(struct PlanState *pstate); + /* ---------------- * PlanState node * @@ -1245,14 +1257,19 @@ typedef struct PlanState { * nodes point to one EState for the whole * top-level plan */ + ExecProcNodeMtd ExecProcNode; /* function to return next tuple */ + ExecProcNodeMtd ExecProcNodeReal; /* actual function, if above is a + * wrapper */ + Instrumentation* instrument; /* Optional runtime stats for this node */ + MemoryContext nodeContext; /* Memory Context for this Node (only if enable_memory_limit) */ + /* * Common structural data for all Plan types. These links to subsidiary * state trees parallel links in the associated plan tree (except for the * subPlan list, which does not exist in the plan tree). */ - List* targetlist; /* target list to be computed at this node */ List* qual; /* implicitly-ANDed qual conditions */ struct PlanState* lefttree; /* input plan tree(s) */ struct PlanState* righttree; @@ -1263,8 +1280,7 @@ typedef struct PlanState { * State for management of parameter-change-driven rescanning */ Bitmapset* chgParam; /* set of IDs of changed Params */ - HbktScanSlot hbktScanSlot; - + /* * Other run-time state needed by most if not all node types. */ @@ -1273,12 +1289,18 @@ typedef struct PlanState { ProjectionInfo* ps_ProjInfo; /* info for doing tuple projection */ bool ps_TupFromTlist; /* state flag for processing set-valued functions in targetlist */ + int64 ps_rownum; /* store current rownum */ + List* targetlist; /* target list to be computed at this node */ + HbktScanSlot hbktScanSlot; bool vectorized; // is vectorized? - MemoryContext nodeContext; /* Memory Context for this Node */ - bool earlyFreed; /* node memory already freed? */ uint8 stubType; /* node stub execution type, see @PlanStubType */ + bool recursive_reset; /* node already reset? */ + bool qual_is_inited; + + bool do_not_reset_rownum; + bool ps_vec_TupFromTlist; /* state flag for processing set-valued functions in targetlist */ vectarget_func jitted_vectarget; /* LLVM IR function pointer to point to the codegened targetlist expr. */ /* @@ -1286,11 +1308,6 @@ typedef struct PlanState { * of data skew and inaccurate e-rows */ List* plan_issues; - bool recursive_reset; /* node already reset? */ - bool qual_is_inited; - - bool do_not_reset_rownum; - int64 ps_rownum; /* store current rownum */ } PlanState; static inline bool planstate_need_stub(PlanState* ps) @@ -1690,28 +1707,33 @@ typedef struct ScanState { Relation ss_currentRelation; TableScanDesc ss_currentScanDesc; TupleTableSlot* ss_ScanTupleSlot; - bool ss_ReScan; - Relation ss_currentPartition; - bool isPartTbl; - int currentSlot; /* current iteration position */ - ScanDirection partScanDirection; - List* partitions; /* list of Partition */ - List* subpartitions; /* list of SubPartition */ - LOCKMODE lockMode; - List* runTimeParamPredicates; - bool runTimePredicatesReady; - bool is_scan_end; /* @hdfs Mark whether iterator is over or not, if the scan uses informational constraint. */ - SeqScanAccessor* ss_scanaccessor; /* prefetch related */ - int part_id; - List* subPartLengthList; - int startPartitionId; /* start partition id for parallel threads. */ - int endPartitionId; /* end partition id for parallel threads. */ - RangeScanInRedis rangeScanInRedis; /* if it is a range scan in redistribution time */ - bool isSampleScan; /* identify is it table sample scan or not. */ - SampleScanParams sampleScanInfo; /* TABLESAMPLE params include type/seed/repeatable. */ + SeqScanGetNextMtd fillNextSlotFunc; ExecScanAccessMtd ScanNextMtd; + bool scanBatchMode; + bool ss_ReScan; + bool isPartTbl; + bool isSampleScan; /* identify is it table sample scan or not. */ + bool runTimePredicatesReady; + bool is_scan_end; /* @hdfs Mark whether iterator is over or not, if the scan uses informational constraint. */ + + int currentSlot; /* current iteration position */ + int part_id; + int startPartitionId; /* start partition id for parallel threads. */ + int endPartitionId; /* end partition id for parallel threads. */ + + LOCKMODE lockMode; + ScanDirection partScanDirection; + + Relation ss_currentPartition; + List* partitions; /* list of Partition */ + List* subpartitions; /* list of SubPartition */ + List* runTimeParamPredicates; + SeqScanAccessor* ss_scanaccessor; /* prefetch related */ + List* subPartLengthList; + RangeScanInRedis rangeScanInRedis; /* if it is a range scan in redistribution time */ + SampleScanParams sampleScanInfo; /* TABLESAMPLE params include type/seed/repeatable. */ ScanBatchState* scanBatchState; Snapshot timecapsuleSnapshot; /* timecapusule snap info */ } ScanState; diff --git a/src/include/pgxc/execRemote.h b/src/include/pgxc/execRemote.h index f0762216d..ac11e27a4 100644 --- a/src/include/pgxc/execRemote.h +++ b/src/include/pgxc/execRemote.h @@ -157,7 +157,6 @@ extern int DataNodeCopyInBinaryForAll(const char* msg_buf, int len, PGXCNodeHand extern int ExecCountSlotsRemoteQuery(RemoteQuery* node); extern RemoteQueryState* ExecInitRemoteQuery(RemoteQuery* node, EState* estate, int eflags, bool row_plan = true); -extern TupleTableSlot* ExecRemoteQuery(RemoteQueryState* step); extern void ExecEndRemoteQuery(RemoteQueryState* step, bool pre_end = false); extern void FreeParallelFunctionState(ParallelFunctionState* state); extern void StrategyFuncSum(ParallelFunctionState* state); diff --git a/src/include/vecexecutor/vecnodevectorow.h b/src/include/vecexecutor/vecnodevectorow.h index 87e6356e3..b52db6f69 100644 --- a/src/include/vecexecutor/vecnodevectorow.h +++ b/src/include/vecexecutor/vecnodevectorow.h @@ -29,7 +29,6 @@ #include "vecexecutor/vectorbatch.h" extern VecToRowState* ExecInitVecToRow(VecToRow* node, EState* estate, int eflags); -extern TupleTableSlot* ExecVecToRow(VecToRowState* node); extern void ExecEndVecToRow(VecToRowState* node); extern void ExecReScanVecToRow(VecToRowState* node); diff --git a/src/test/regress/pg_regress.cpp b/src/test/regress/pg_regress.cpp index ed1d65b75..a2559b1aa 100644 --- a/src/test/regress/pg_regress.cpp +++ b/src/test/regress/pg_regress.cpp @@ -5461,7 +5461,7 @@ static void check_global_variables() } } -#define BASE_PGXC_LIKE_MACRO_NUM 1392 +#define BASE_PGXC_LIKE_MACRO_NUM 1391 static void check_pgxc_like_macros() { #ifdef BUILD_BY_CMAKE