@ -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);
|
||||
|
||||
@ -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
|
||||
*/
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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 */
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user