diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 061f00619..dd2c26733 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -24,273 +24,419 @@ #include "distributed/multi_utility.h" #include "distributed/worker_protocol.h" #include "executor/execdebug.h" +#include "commands/copy.h" +#include "nodes/makefuncs.h" #include "storage/lmgr.h" #include "tcop/utility.h" #include "utils/snapmgr.h" - - -static void CopyQueryResults(List *masterCopyStmtList); +#include "utils/memutils.h" /* - * multi_ExecutorStart is a hook called at at the beginning of any execution - * of any query plan. - * - * If a distributed relation is the target of the query, perform some validity - * checks. If a legal statement, start the distributed execution. After that - * the to-be-executed query is replaced with the portion executing solely on - * the master. + * Define executor methods for the different executor types. */ -void -multi_ExecutorStart(QueryDesc *queryDesc, int eflags) +static CustomExecMethods RealTimeCustomExecMethods = { + .CustomName = "RealTimeScan", + .BeginCustomScan = CitusSelectBeginScan, + .ExecCustomScan = RealTimeExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = CitusExplainScan +}; + +static CustomExecMethods TaskTrackerCustomExecMethods = { + .CustomName = "TaskTrackerScan", + .BeginCustomScan = CitusSelectBeginScan, + .ExecCustomScan = TaskTrackerExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = CitusExplainScan +}; + +static CustomExecMethods RouterSingleModifyCustomExecMethods = { + .CustomName = "RouterSingleModifyScan", + .BeginCustomScan = CitusModifyBeginScan, + .ExecCustomScan = RouterSingleModifyExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = CitusExplainScan +}; + +static CustomExecMethods RouterMultiModifyCustomExecMethods = { + .CustomName = "RouterMultiModifyScan", + .BeginCustomScan = CitusModifyBeginScan, + .ExecCustomScan = RouterMultiModifyExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = CitusExplainScan +}; + +static CustomExecMethods RouterSelectCustomExecMethods = { + .CustomName = "RouterSelectScan", + .BeginCustomScan = CitusSelectBeginScan, + .ExecCustomScan = RouterSelectExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = CitusExplainScan +}; + + +/* local function forward declarations */ +static void PrepareMasterJobDirectory(Job *workerJob); +static void LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob); +static Relation StubRelation(TupleDesc tupleDescriptor); + + +/* + * RealTimeCreateScan creates the scan state for real-time executor queries. + */ +Node * +RealTimeCreateScan(CustomScan *scan) { - PlannedStmt *planStatement = queryDesc->plannedstmt; + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); - if (HasCitusToplevelNode(planStatement)) + scanState->executorType = MULTI_EXECUTOR_REAL_TIME; + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->multiPlan = GetMultiPlan(scan); + + scanState->customScanState.methods = &RealTimeCustomExecMethods; + + return (Node *) scanState; +} + + +/* + * TaskTrackerCreateScan creates the scan state for task-tracker executor queries. + */ +Node * +TaskTrackerCreateScan(CustomScan *scan) +{ + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); + + scanState->executorType = MULTI_EXECUTOR_TASK_TRACKER; + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->multiPlan = GetMultiPlan(scan); + + scanState->customScanState.methods = &TaskTrackerCustomExecMethods; + + return (Node *) scanState; +} + + +/* + * RouterCreateScan creates the scan state for router executor queries. + */ +Node * +RouterCreateScan(CustomScan *scan) +{ + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); + MultiPlan *multiPlan = NULL; + Job *workerJob = NULL; + List *taskList = NIL; + bool isModificationQuery = false; + + scanState->executorType = MULTI_EXECUTOR_ROUTER; + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->multiPlan = GetMultiPlan(scan); + + multiPlan = scanState->multiPlan; + workerJob = multiPlan->workerJob; + taskList = workerJob->taskList; + + isModificationQuery = IsModifyMultiPlan(multiPlan); + + /* check if this is a single shard query */ + if (list_length(taskList) == 1) { - MultiPlan *multiPlan = GetMultiPlan(planStatement); - MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST; - Job *workerJob = multiPlan->workerJob; - - /* ensure plan is executable */ - VerifyMultiPlanValidity(multiPlan); - - ExecCheckRTPerms(planStatement->rtable, true); - - executorType = JobExecutorType(multiPlan); - if (executorType == MULTI_EXECUTOR_ROUTER) + if (isModificationQuery) { - List *taskList = workerJob->taskList; - TupleDesc tupleDescriptor = ExecCleanTypeFromTL( - planStatement->planTree->targetlist, false); - List *dependendJobList PG_USED_FOR_ASSERTS_ONLY = workerJob->dependedJobList; - - /* router executor cannot execute task with depencencies */ - Assert(dependendJobList == NIL); - - /* we need to set tupleDesc in executorStart */ - queryDesc->tupDesc = tupleDescriptor; - - /* drop into the router executor */ - RouterExecutorStart(queryDesc, eflags, taskList); + scanState->customScanState.methods = &RouterSingleModifyCustomExecMethods; } else { - PlannedStmt *masterSelectPlan = MasterNodeSelectPlan(multiPlan); - CreateStmt *masterCreateStmt = MasterNodeCreateStatement(multiPlan); - List *masterCopyStmtList = MasterNodeCopyStatementList(multiPlan); - RangeTblEntry *masterRangeTableEntry = NULL; - StringInfo jobDirectoryName = NULL; - - /* - * We create a directory on the master node to keep task execution results. - * We also register this directory for automatic cleanup on portal delete. - */ - jobDirectoryName = MasterJobDirectoryName(workerJob->jobId); - CreateDirectory(jobDirectoryName); - - ResourceOwnerEnlargeJobDirectories(CurrentResourceOwner); - ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId); - - /* pick distributed executor to use */ - if (eflags & EXEC_FLAG_EXPLAIN_ONLY) - { - /* skip distributed query execution for EXPLAIN commands */ - } - else if (executorType == MULTI_EXECUTOR_REAL_TIME) - { - MultiRealTimeExecute(workerJob); - } - else if (executorType == MULTI_EXECUTOR_TASK_TRACKER) - { - MultiTaskTrackerExecute(workerJob); - } - - /* then create the result relation */ - ProcessUtility((Node *) masterCreateStmt, - "(temp table creation)", - PROCESS_UTILITY_QUERY, - NULL, - None_Receiver, - NULL); - - /* make the temporary table visible */ - CommandCounterIncrement(); - - if (!(eflags & EXEC_FLAG_EXPLAIN_ONLY)) - { - CopyQueryResults(masterCopyStmtList); - } - - /* - * Update the QueryDesc's snapshot so it sees the table. That's not - * particularly pretty, but we don't have much of a choice. One might - * think we could unregister the snapshot, push a new active one, - * update it, register it, and be happy. That only works if it's only - * registered once though... - */ - queryDesc->snapshot->curcid = GetCurrentCommandId(false); - - /* - * Set the OID of the RTE used in the master select statement to point - * to the now created (and filled) temporary table. The target - * relation's oid is only known now. - */ - masterRangeTableEntry = - (RangeTblEntry *) linitial(masterSelectPlan->rtable); - masterRangeTableEntry->relid = - RelnameGetRelid(masterRangeTableEntry->eref->aliasname); - - /* - * Replace to-be-run query with the master select query. As the - * planned statement is now replaced we can't call GetMultiPlan() in - * the later hooks, so we set a flag marking this as a distributed - * statement running on the master. That e.g. allows us to drop the - * temp table later. - * - * We copy the original statement's queryId, to allow - * pg_stat_statements and similar extension to associate the - * statement with the toplevel statement. - */ - masterSelectPlan->queryId = queryDesc->plannedstmt->queryId; - queryDesc->plannedstmt = masterSelectPlan; - - eflags |= EXEC_FLAG_CITUS_MASTER_SELECT; + scanState->customScanState.methods = &RouterSelectCustomExecMethods; } } - - /* if the execution is not done for router executor, drop into standard executor */ - if (queryDesc->estate == NULL || - !(queryDesc->estate->es_top_eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR)) + else { - standard_ExecutorStart(queryDesc, eflags); + Assert(isModificationQuery); + scanState->customScanState.methods = &RouterMultiModifyCustomExecMethods; } + + return (Node *) scanState; } /* - * CopyQueryResults executes the commands that copy query results into a - * temporary table. + * DelayedErrorCreateScan is only called if we could not plan for the given + * query. This is the case when a plan is not ready for execution because + * CreateDistributedPlan() couldn't find a plan due to unresolved prepared + * statement parameters, but didn't error out, because we expect custom plans + * to come to our rescue. But sql (not plpgsql) functions unfortunately don't + * go through a codepath supporting custom plans. Here, we error out with this + * delayed error message. + */ +Node * +DelayedErrorCreateScan(CustomScan *scan) +{ + MultiPlan *multiPlan = GetMultiPlan(scan); + + /* raise the deferred error */ + RaiseDeferredError(multiPlan->planningError, ERROR); + + return NULL; +} + + +/* + * CitusSelectBeginScan is an empty function for BeginCustomScan callback. + */ +void +CitusSelectBeginScan(CustomScanState *node, EState *estate, int eflags) +{ + /* just an empty function */ +} + + +/* + * RealTimeExecScan is a callback function which returns next tuple from a real-time + * execution. In the first call, it executes distributed real-time plan and loads + * results from temporary files into custom scan's tuple store. Then, it returns + * tuples one by one from this tuple store. + */ +TupleTableSlot * +RealTimeExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; + + if (!scanState->finishedRemoteScan) + { + MultiPlan *multiPlan = scanState->multiPlan; + Job *workerJob = multiPlan->workerJob; + + PrepareMasterJobDirectory(workerJob); + MultiRealTimeExecute(workerJob); + + LoadTuplesIntoTupleStore(scanState, workerJob); + + scanState->finishedRemoteScan = true; + } + + resultSlot = ReturnTupleFromTuplestore(scanState); + + return resultSlot; +} + + +/* + * PrepareMasterJobDirectory creates a directory on the master node to keep job + * execution results. We also register this directory for automatic cleanup on + * portal delete. */ static void -CopyQueryResults(List *masterCopyStmtList) +PrepareMasterJobDirectory(Job *workerJob) { - ListCell *masterCopyStmtCell = NULL; + StringInfo jobDirectoryName = MasterJobDirectoryName(workerJob->jobId); + CreateDirectory(jobDirectoryName); - /* now copy data from all the remote nodes into temp table */ - foreach(masterCopyStmtCell, masterCopyStmtList) - { - Node *masterCopyStmt = (Node *) lfirst(masterCopyStmtCell); - - Assert(IsA(masterCopyStmt, CopyStmt)); - - ProcessUtility(masterCopyStmt, - "(copy job)", - PROCESS_UTILITY_QUERY, - NULL, - None_Receiver, - NULL); - } - - /* make the copied contents visible */ - CommandCounterIncrement(); + ResourceOwnerEnlargeJobDirectories(CurrentResourceOwner); + ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId); } -/* Execute query plan. */ -void -multi_ExecutorRun(QueryDesc *queryDesc, ScanDirection direction, tuplecount_t count) +/* + * Load data collected by real-time or task-tracker executors into the tuplestore + * of CitusScanState. For that, we first create a tuple store, and then copy the + * files one-by-one into the tuple store. + * + * Note that in the long term it'd be a lot better if Multi*Execute() directly + * filled the tuplestores, but that's a fair bit of work. + */ +static void +LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob) { - int eflags = queryDesc->estate->es_top_eflags; + CustomScanState customScanState = citusScanState->customScanState; + List *workerTaskList = workerJob->taskList; + List *copyOptions = NIL; + EState *executorState = NULL; + MemoryContext executorTupleContext = NULL; + ExprContext *executorExpressionContext = NULL; + TupleDesc tupleDescriptor = NULL; + Relation stubRelation = NULL; + ListCell *workerTaskCell = NULL; + uint32 columnCount = 0; + Datum *columnValues = NULL; + bool *columnNulls = NULL; + bool randomAccess = true; + bool interTransactions = false; - if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR) + executorState = citusScanState->customScanState.ss.ps.state; + executorTupleContext = GetPerTupleMemoryContext(executorState); + executorExpressionContext = GetPerTupleExprContext(executorState); + + tupleDescriptor = customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; + stubRelation = StubRelation(tupleDescriptor); + + columnCount = tupleDescriptor->natts; + columnValues = palloc0(columnCount * sizeof(Datum)); + columnNulls = palloc0(columnCount * sizeof(bool)); + + Assert(citusScanState->tuplestorestate == NULL); + citusScanState->tuplestorestate = + tuplestore_begin_heap(randomAccess, interTransactions, work_mem); + + if (BinaryMasterCopyFormat) { - /* drop into the router executor */ - RouterExecutorRun(queryDesc, direction, count); + DefElem *copyOption = makeDefElem("format", (Node *) makeString("binary")); + copyOptions = lappend(copyOptions, copyOption); } - else - { - /* drop into the standard executor */ - standard_ExecutorRun(queryDesc, direction, count); - } -} - -/* Perform actions, like e.g. firing triggers, after the query has run. */ -void -multi_ExecutorFinish(QueryDesc *queryDesc) -{ - int eflags = queryDesc->estate->es_top_eflags; - - if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR) + foreach(workerTaskCell, workerTaskList) { - /* drop into the router executor */ - RouterExecutorFinish(queryDesc); - } - else - { - /* drop into the standard executor */ - standard_ExecutorFinish(queryDesc); + Task *workerTask = (Task *) lfirst(workerTaskCell); + StringInfo jobDirectoryName = NULL; + StringInfo taskFilename = NULL; + CopyState copyState = NULL; + + jobDirectoryName = MasterJobDirectoryName(workerTask->jobId); + taskFilename = TaskFilename(jobDirectoryName, workerTask->taskId); + + copyState = BeginCopyFrom(stubRelation, taskFilename->data, false, NULL, + copyOptions); + + while (true) + { + MemoryContext oldContext = NULL; + bool nextRowFound = false; + + ResetPerTupleExprContext(executorState); + oldContext = MemoryContextSwitchTo(executorTupleContext); + + nextRowFound = NextCopyFrom(copyState, executorExpressionContext, + columnValues, columnNulls, NULL); + if (!nextRowFound) + { + MemoryContextSwitchTo(oldContext); + break; + } + + tuplestore_putvalues(citusScanState->tuplestorestate, tupleDescriptor, + columnValues, columnNulls); + MemoryContextSwitchTo(oldContext); + } + + EndCopyFrom(copyState); } } /* - * multi_ExecutorEnd is a hook called to deallocate resources used during - * query execution. - * - * If the query executed was the portion of a distributed query running on the - * master, remove the resources that were needed for distributed execution. + * StubRelation creates a stub Relation from the given tuple descriptor. + * To be able to use copy.c, we need a Relation descriptor. As there is no + * relation corresponding to the data loaded from workers, we need to fake one. + * We just need the bare minimal set of fields accessed by BeginCopyFrom(). + */ +static Relation +StubRelation(TupleDesc tupleDescriptor) +{ + Relation stubRelation = palloc0(sizeof(RelationData)); + stubRelation->rd_att = tupleDescriptor; + stubRelation->rd_rel = palloc0(sizeof(FormData_pg_class)); + stubRelation->rd_rel->relkind = RELKIND_RELATION; + + return stubRelation; +} + + +/* + * ReturnTupleFromTuplestore reads the next tuple from the tuple store of the + * given Citus scan node and returns it. It returns null if all tuples are read + * from the tuple store. + */ +TupleTableSlot * +ReturnTupleFromTuplestore(CitusScanState *scanState) +{ + Tuplestorestate *tupleStore = scanState->tuplestorestate; + TupleTableSlot *resultSlot = NULL; + ScanDirection scanDirection = NoMovementScanDirection; + bool forwardScanDirection = true; + + if (tupleStore == NULL) + { + return NULL; + } + + scanDirection = scanState->customScanState.ss.ps.state->es_direction; + Assert(ScanDirectionIsValid(scanDirection)); + + if (ScanDirectionIsBackward(scanDirection)) + { + forwardScanDirection = false; + } + + resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot; + tuplestore_gettupleslot(tupleStore, forwardScanDirection, false, resultSlot); + + return resultSlot; +} + + +/* + * TaskTrackerExecScan is a callback function which returns next tuple from a + * task-tracker execution. In the first call, it executes distributed task-tracker + * plan and loads results from temporary files into custom scan's tuple store. + * Then, it returns tuples one by one from this tuple store. + */ +TupleTableSlot * +TaskTrackerExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; + + if (!scanState->finishedRemoteScan) + { + MultiPlan *multiPlan = scanState->multiPlan; + Job *workerJob = multiPlan->workerJob; + + PrepareMasterJobDirectory(workerJob); + MultiTaskTrackerExecute(workerJob); + + LoadTuplesIntoTupleStore(scanState, workerJob); + + scanState->finishedRemoteScan = true; + } + + resultSlot = ReturnTupleFromTuplestore(scanState); + + return resultSlot; +} + + +/* + * CitusEndScan is used to clean up tuple store of the given custom scan state. */ void -multi_ExecutorEnd(QueryDesc *queryDesc) +CitusEndScan(CustomScanState *node) { - int eflags = queryDesc->estate->es_top_eflags; + CitusScanState *scanState = (CitusScanState *) node; - if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR) + if (scanState->tuplestorestate) { - /* drop into the router executor */ - RouterExecutorEnd(queryDesc); - } - else - { - /* drop into the standard executor */ - standard_ExecutorEnd(queryDesc); - } - - /* - * Final step of a distributed query is executing the master node select - * query. We clean up the temp tables after executing it, if we already created it. - */ - if (eflags & EXEC_FLAG_CITUS_MASTER_SELECT) - { - PlannedStmt *planStatement = queryDesc->plannedstmt; - int savedLogMinMessages = 0; - int savedClientMinMessages = 0; - - RangeTblEntry *rangeTableEntry = linitial(planStatement->rtable); - Oid masterTableRelid = rangeTableEntry->relid; - - ObjectAddress masterTableObject = { InvalidOid, InvalidOid, 0 }; - - masterTableObject.classId = RelationRelationId; - masterTableObject.objectId = masterTableRelid; - masterTableObject.objectSubId = 0; - - /* - * Temporarily change logging level to avoid DEBUG2 logging output by - * performDeletion. This avoids breaking the regression tests which - * use DEBUG2 logging. - */ - savedLogMinMessages = log_min_messages; - savedClientMinMessages = client_min_messages; - - log_min_messages = INFO; - client_min_messages = INFO; - - performDeletion(&masterTableObject, DROP_RESTRICT, PERFORM_DELETION_INTERNAL); - - log_min_messages = savedLogMinMessages; - client_min_messages = savedClientMinMessages; + tuplestore_end(scanState->tuplestorestate); + scanState->tuplestorestate = NULL; } } + + +/* + * CitusReScan is just a place holder for rescan callback. Currently, we don't + * support rescan given that there is not any way to reach this code path. + */ +void +CitusReScan(CustomScanState *node) +{ + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("rescan is unsupported"), + errdetail("We don't expect this code path to be executed."))); +} diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index ceed971b2..9a26090f8 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -74,83 +74,31 @@ bool EnableDeadlockPrevention = true; /* functions needed during run phase */ static void ReacquireMetadataLocks(List *taskList); -static void ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, +static void ExecuteSingleModifyTask(CitusScanState *scanState, Task *task, bool expectResults); -static void ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task); -static List * GetModifyConnections(List *taskPlacementList, - bool markCritical, +static void ExecuteSingleSelectTask(CitusScanState *scanState, Task *task); +static List * GetModifyConnections(List *taskPlacementList, bool markCritical, bool startedInTransaction); -static void ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList, +static void ExecuteMultipleTasks(CitusScanState *scanState, List *taskList, bool isModificationQuery, bool expectResults); static int64 ExecuteModifyTasks(List *taskList, bool expectResults, - ParamListInfo paramListInfo, - MaterialState *routerState, - TupleDesc tupleDescriptor); + ParamListInfo paramListInfo, CitusScanState *scanState); static List * TaskShardIntervalList(List *taskList); static void AcquireExecutorShardLock(Task *task, CmdType commandType); static void AcquireExecutorMultiShardLocks(List *taskList); static bool RequiresConsistentSnapshot(Task *task); -static uint64 ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor, - DestReceiver *destination, - Tuplestorestate *tupleStore); +static void ProcessMasterEvaluableFunctions(Job *workerJob); static void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterTypes, const char ***parameterValues); static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query, ParamListInfo paramListInfo); -static bool StoreQueryResult(MaterialState *routerState, MultiConnection *connection, - TupleDesc tupleDescriptor, bool failOnError, int64 *rows); +static bool StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, + bool failOnError, int64 *rows); static bool ConsumeQueryResult(MultiConnection *connection, bool failOnError, int64 *rows); -/* - * RouterExecutorStart sets up the executor state and queryDesc for router - * execution. - */ -void -RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList) -{ - EState *executorState = NULL; - CmdType commandType = queryDesc->operation; - - /* - * If we are executing a prepared statement, then we may not yet have obtained - * the metadata locks in this transaction. To prevent a concurrent shard copy, - * we re-obtain them here or error out if a shard copy has already started. - * - * If a shard copy finishes in between fetching a plan from cache and - * re-acquiring the locks, then we might still run a stale plan, which could - * cause shard placements to diverge. To minimize this window, we take the - * locks as early as possible. - */ - ReacquireMetadataLocks(taskList); - - /* disallow triggers during distributed modify commands */ - if (commandType != CMD_SELECT) - { - eflags |= EXEC_FLAG_SKIP_TRIGGERS; - } - - /* signal that it is a router execution */ - eflags |= EXEC_FLAG_CITUS_ROUTER_EXECUTOR; - - /* build empty executor state to obtain per-query memory context */ - executorState = CreateExecutorState(); - executorState->es_top_eflags = eflags; - executorState->es_instrument = queryDesc->instrument_options; - - queryDesc->estate = executorState; - - /* - * As it's similar to what we're doing, use a MaterialState node to store - * our state. This is used to store our tuplestore, so cursors etc. can - * work. - */ - queryDesc->planstate = (PlanState *) makeNode(MaterialState); -} - - /* * ReacquireMetadataLocks re-acquires the metadata locks that are normally * acquired during planning. @@ -458,140 +406,140 @@ RequiresConsistentSnapshot(Task *task) /* - * RouterExecutorRun actually executes a single task on a worker. + * CitusModifyBeginScan checks the validity of the given custom scan node and + * gets locks on the shards involved in the task list of the distributed plan. */ void -RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count) +CitusModifyBeginScan(CustomScanState *node, EState *estate, int eflags) { - PlannedStmt *planStatement = queryDesc->plannedstmt; - MultiPlan *multiPlan = GetMultiPlan(planStatement); + CitusScanState *scanState = (CitusScanState *) node; + MultiPlan *multiPlan = scanState->multiPlan; Job *workerJob = multiPlan->workerJob; List *taskList = workerJob->taskList; - EState *estate = queryDesc->estate; - CmdType operation = queryDesc->operation; - MemoryContext oldcontext = NULL; - DestReceiver *destination = queryDesc->dest; - MaterialState *routerState = (MaterialState *) queryDesc->planstate; - bool sendTuples = operation == CMD_SELECT || queryDesc->plannedstmt->hasReturning; - - Assert(estate != NULL); - Assert(!(estate->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY)); - - oldcontext = MemoryContextSwitchTo(estate->es_query_cxt); - - if (queryDesc->totaltime != NULL) - { - InstrStartNode(queryDesc->totaltime); - } - - estate->es_processed = 0; - - /* startup the tuple receiver */ - if (sendTuples) - { - (*destination->rStartup)(destination, operation, queryDesc->tupDesc); - } - - /* we only support returning nothing or scanning forward */ - if (ScanDirectionIsNoMovement(direction)) - { - /* comments in PortalRunSelect() explain the reason for this case */ - goto out; - } - else if (!ScanDirectionIsForward(direction)) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("scan directions other than forward scans " - "are unsupported"))); - } /* - * If query has not yet been executed, do so now. The main reason why the - * query might already have been executed is cursors. + * If we are executing a prepared statement, then we may not yet have obtained + * the metadata locks in this transaction. To prevent a concurrent shard copy, + * we re-obtain them here or error out if a shard copy has already started. + * + * If a shard copy finishes in between fetching a plan from cache and + * re-acquiring the locks, then we might still run a stale plan, which could + * cause shard placements to diverge. To minimize this window, we take the + * locks as early as possible. */ - if (!routerState->eof_underlying) + ReacquireMetadataLocks(taskList); +} + + +/* + * RouterSingleModifyExecScan executes a single modification query on a + * distributed plan and returns results if there is any. + */ +TupleTableSlot * +RouterSingleModifyExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; + + if (!scanState->finishedRemoteScan) { - bool isModificationQuery = false; - bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation; + MultiPlan *multiPlan = scanState->multiPlan; + bool hasReturning = multiPlan->hasReturning; + Job *workerJob = multiPlan->workerJob; + List *taskList = workerJob->taskList; + Task *task = (Task *) linitial(taskList); - if (operation == CMD_INSERT || operation == CMD_UPDATE || - operation == CMD_DELETE) - { - isModificationQuery = true; - } - else if (operation != CMD_SELECT) - { - ereport(ERROR, (errmsg("unrecognized operation code: %d", - (int) operation))); - } + ProcessMasterEvaluableFunctions(workerJob); - if (requiresMasterEvaluation) - { - Query *jobQuery = workerJob->jobQuery; + ExecuteSingleModifyTask(scanState, task, hasReturning); - ExecuteMasterEvaluableFunctions(jobQuery); - RebuildQueryStrings(jobQuery, taskList); - } - - if (list_length(taskList) == 1) - { - Task *task = (Task *) linitial(taskList); - - if (isModificationQuery) - { - ExecuteSingleModifyTask(queryDesc, task, sendTuples); - } - else - { - ExecuteSingleSelectTask(queryDesc, task); - } - } - else - { - ExecuteMultipleTasks(queryDesc, taskList, isModificationQuery, - sendTuples); - } - - /* mark underlying query as having executed */ - routerState->eof_underlying = true; + scanState->finishedRemoteScan = true; } - /* if the underlying query produced output, return it */ - if (routerState->tuplestorestate != NULL) + resultSlot = ReturnTupleFromTuplestore(scanState); + + return resultSlot; +} + + +/* + * ProcessMasterEvaluableFunctions executes evaluable functions and rebuilds + * the query strings in task lists. + */ +static void +ProcessMasterEvaluableFunctions(Job *workerJob) +{ + if (workerJob->requiresMasterEvaluation) { - TupleDesc resultTupleDescriptor = queryDesc->tupDesc; - int64 returnedRows = 0; + Query *jobQuery = workerJob->jobQuery; + List *taskList = workerJob->taskList; - /* return rows from the tuplestore */ - returnedRows = ReturnRowsFromTuplestore(count, resultTupleDescriptor, - destination, - routerState->tuplestorestate); + ExecuteMasterEvaluableFunctions(jobQuery); + RebuildQueryStrings(jobQuery, taskList); + } +} - /* - * Count tuples processed, if this is a SELECT. (For modifications - * it'll already have been increased, as we want the number of - * modified tuples, not the number of RETURNed tuples.) - */ - if (operation == CMD_SELECT) - { - estate->es_processed += returnedRows; - } + +/* + * RouterMultiModifyExecScan executes a list of tasks on remote nodes, retrieves + * the results and, if RETURNING is used, stores them in custom scan's tuple store. + * Then, it returns tuples one by one from this tuple store. + */ +TupleTableSlot * +RouterMultiModifyExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; + + if (!scanState->finishedRemoteScan) + { + MultiPlan *multiPlan = scanState->multiPlan; + Job *workerJob = multiPlan->workerJob; + List *taskList = workerJob->taskList; + bool hasReturning = multiPlan->hasReturning; + bool isModificationQuery = true; + + ProcessMasterEvaluableFunctions(workerJob); + + ExecuteMultipleTasks(scanState, taskList, isModificationQuery, hasReturning); + + scanState->finishedRemoteScan = true; } -out: + resultSlot = ReturnTupleFromTuplestore(scanState); - /* shutdown tuple receiver, if we started it */ - if (sendTuples) + return resultSlot; +} + + +/* + * RouterSelectExecScan executes a single select task on the remote node, + * retrieves the results and stores them in custom scan's tuple store. Then, it + * returns tuples one by one from this tuple store. + */ +TupleTableSlot * +RouterSelectExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; + + if (!scanState->finishedRemoteScan) { - (*destination->rShutdown)(destination); + MultiPlan *multiPlan = scanState->multiPlan; + Job *workerJob = multiPlan->workerJob; + List *taskList = workerJob->taskList; + Task *task = (Task *) linitial(taskList); + + ProcessMasterEvaluableFunctions(workerJob); + + ExecuteSingleSelectTask(scanState, task); + + scanState->finishedRemoteScan = true; } - if (queryDesc->totaltime != NULL) - { - InstrStopNode(queryDesc->totaltime, estate->es_processed); - } + resultSlot = ReturnTupleFromTuplestore(scanState); - MemoryContextSwitchTo(oldcontext); + return resultSlot; } @@ -603,11 +551,10 @@ out: * other placements or errors out if the query fails on all placements. */ static void -ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task) +ExecuteSingleSelectTask(CitusScanState *scanState, Task *task) { - TupleDesc tupleDescriptor = queryDesc->tupDesc; - MaterialState *routerState = (MaterialState *) queryDesc->planstate; - ParamListInfo paramListInfo = queryDesc->params; + ParamListInfo paramListInfo = + scanState->customScanState.ss.ps.state->es_param_list_info; List *taskPlacementList = task->taskPlacementList; ListCell *taskPlacementCell = NULL; char *queryString = task->queryString; @@ -639,8 +586,8 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task) continue; } - queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, - dontFailOnError, ¤tAffectedTupleCount); + queryOK = StoreQueryResult(scanState, connection, dontFailOnError, + ¤tAffectedTupleCount); if (queryOK) { return; @@ -661,21 +608,19 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task) * framework), or errors out (failed on all placements). */ static void -ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, - bool expectResults) +ExecuteSingleModifyTask(CitusScanState *scanState, Task *task, bool expectResults) { - CmdType operation = queryDesc->operation; - TupleDesc tupleDescriptor = queryDesc->tupDesc; - EState *executorState = queryDesc->estate; - MaterialState *routerState = (MaterialState *) queryDesc->planstate; - ParamListInfo paramListInfo = queryDesc->params; - bool resultsOK = false; + CmdType operation = scanState->multiPlan->operation; + EState *executorState = scanState->customScanState.ss.ps.state; + ParamListInfo paramListInfo = executorState->es_param_list_info; List *taskPlacementList = task->taskPlacementList; List *connectionList = NIL; ListCell *taskPlacementCell = NULL; ListCell *connectionCell = NULL; int64 affectedTupleCount = -1; + bool resultsOK = false; bool gotResults = false; + char *queryString = task->queryString; bool taskRequiresTwoPhaseCommit = (task->replicationModel == REPLICATION_MODEL_2PC); bool startedInTransaction = @@ -761,8 +706,8 @@ ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, */ if (!gotResults && expectResults) { - queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, - failOnError, ¤tAffectedTupleCount); + queryOK = StoreQueryResult(scanState, connection, failOnError, + ¤tAffectedTupleCount); } else { @@ -893,20 +838,18 @@ GetModifyConnections(List *taskPlacementList, bool markCritical, bool noNewTrans * commits. */ static void -ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList, +ExecuteMultipleTasks(CitusScanState *scanState, List *taskList, bool isModificationQuery, bool expectResults) { - TupleDesc tupleDescriptor = queryDesc->tupDesc; - EState *executorState = queryDesc->estate; - MaterialState *routerState = (MaterialState *) queryDesc->planstate; - ParamListInfo paramListInfo = queryDesc->params; + EState *executorState = scanState->customScanState.ss.ps.state; + ParamListInfo paramListInfo = executorState->es_param_list_info; int64 affectedTupleCount = -1; /* can only support modifications right now */ Assert(isModificationQuery); affectedTupleCount = ExecuteModifyTasks(taskList, expectResults, paramListInfo, - routerState, tupleDescriptor); + scanState); executorState->es_processed = affectedTupleCount; } @@ -922,7 +865,7 @@ ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList, int64 ExecuteModifyTasksWithoutResults(List *taskList) { - return ExecuteModifyTasks(taskList, false, NULL, NULL, NULL); + return ExecuteModifyTasks(taskList, false, NULL, NULL); } @@ -936,7 +879,7 @@ ExecuteModifyTasksWithoutResults(List *taskList) */ static int64 ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListInfo, - MaterialState *routerState, TupleDesc tupleDescriptor) + CitusScanState *scanState) { int64 totalAffectedTupleCount = 0; ListCell *taskCell = NULL; @@ -1020,8 +963,7 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn continue; } - connection = - (MultiConnection *) list_nth(connectionList, placementIndex); + connection = (MultiConnection *) list_nth(connectionList, placementIndex); queryOK = SendQueryInSingleRowMode(connection, queryString, paramListInfo); if (!queryOK) @@ -1066,10 +1008,10 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn */ if (placementIndex == 0 && expectResults) { - Assert(routerState != NULL && tupleDescriptor != NULL); + Assert(scanState != NULL); - queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, - failOnError, ¤tAffectedTupleCount); + queryOK = StoreQueryResult(scanState, connection, failOnError, + ¤tAffectedTupleCount); } else { @@ -1149,50 +1091,6 @@ TaskShardIntervalList(List *taskList) } -/* - * ReturnRowsFromTuplestore moves rows from a given tuplestore into a - * receiver. It performs the necessary limiting to support cursors. - */ -static uint64 -ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor, - DestReceiver *destination, Tuplestorestate *tupleStore) -{ - TupleTableSlot *tupleTableSlot = NULL; - uint64 currentTupleCount = 0; - - tupleTableSlot = MakeSingleTupleTableSlot(tupleDescriptor); - - /* iterate over tuples in tuple store, and send them to destination */ - for (;;) - { - bool nextTuple = tuplestore_gettupleslot(tupleStore, true, false, tupleTableSlot); - if (!nextTuple) - { - break; - } - - (*destination->receiveSlot)(tupleTableSlot, destination); - - ExecClearTuple(tupleTableSlot); - - currentTupleCount++; - - /* - * If numberTuples is zero fetch all tuples, otherwise stop after - * count tuples. - */ - if (tupleCount > 0 && tupleCount == currentTupleCount) - { - break; - } - } - - ExecDropSingleTupleTableSlot(tupleTableSlot); - - return currentTupleCount; -} - - /* * SendQueryInSingleRowMode sends the given query on the connection in an * asynchronous way. The function also sets the single-row mode on the @@ -1318,13 +1216,18 @@ ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterT * the connection. */ static bool -StoreQueryResult(MaterialState *routerState, MultiConnection *connection, - TupleDesc tupleDescriptor, bool failOnError, int64 *rows) +StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, + bool failOnError, int64 *rows) { + TupleDesc tupleDescriptor = + scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; AttInMetadata *attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor); - Tuplestorestate *tupleStore = NULL; - uint32 expectedColumnCount = tupleDescriptor->natts; + List *targetList = scanState->customScanState.ss.ps.plan->targetlist; + uint32 expectedColumnCount = ExecCleanTargetListLength(targetList); char **columnArray = (char **) palloc0(expectedColumnCount * sizeof(char *)); + Tuplestorestate *tupleStore = NULL; + bool randomAccess = true; + bool interTransactions = false; bool commandFailed = false; MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext, "StoreQueryResult", @@ -1333,17 +1236,18 @@ StoreQueryResult(MaterialState *routerState, MultiConnection *connection, ALLOCSET_DEFAULT_MAXSIZE); *rows = 0; - if (routerState->tuplestorestate == NULL) + if (scanState->tuplestorestate == NULL) { - routerState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem); + scanState->tuplestorestate = + tuplestore_begin_heap(randomAccess, interTransactions, work_mem); } else if (!failOnError) { /* might have failed query execution on another placement before */ - tuplestore_clear(routerState->tuplestorestate); + tuplestore_clear(scanState->tuplestorestate); } - tupleStore = routerState->tuplestorestate; + tupleStore = scanState->tuplestorestate; for (;;) { @@ -1537,39 +1441,3 @@ ConsumeQueryResult(MultiConnection *connection, bool failOnError, int64 *rows) return gotResponse && !commandFailed; } - - -/* - * RouterExecutorFinish cleans up after a distributed execution. - */ -void -RouterExecutorFinish(QueryDesc *queryDesc) -{ - EState *estate = queryDesc->estate; - Assert(estate != NULL); - - estate->es_finished = true; -} - - -/* - * RouterExecutorEnd cleans up the executor state after a distributed - * execution. - */ -void -RouterExecutorEnd(QueryDesc *queryDesc) -{ - EState *estate = queryDesc->estate; - MaterialState *routerState = (MaterialState *) queryDesc->planstate; - - if (routerState->tuplestorestate) - { - tuplestore_end(routerState->tuplestorestate); - } - - Assert(estate != NULL); - - FreeExecutorState(estate); - queryDesc->estate = NULL; - queryDesc->totaltime = NULL; -} diff --git a/src/backend/distributed/executor/multi_utility.c b/src/backend/distributed/executor/multi_utility.c index 849b7093c..b0c980dbe 100644 --- a/src/backend/distributed/executor/multi_utility.c +++ b/src/backend/distributed/executor/multi_utility.c @@ -369,51 +369,6 @@ multi_ProcessUtility(Node *parsetree, " necessary users and roles."))); } - /* due to an explain-hook limitation we have to special-case EXPLAIN EXECUTE */ - if (IsA(parsetree, ExplainStmt) && IsA(((ExplainStmt *) parsetree)->query, Query)) - { - ExplainStmt *explainStmt = (ExplainStmt *) parsetree; - Query *query = (Query *) explainStmt->query; - - if (query->commandType == CMD_UTILITY && - IsA(query->utilityStmt, ExecuteStmt)) - { - ExecuteStmt *execstmt = (ExecuteStmt *) query->utilityStmt; - PreparedStatement *entry = FetchPreparedStatement(execstmt->name, true); - CachedPlanSource *plansource = entry->plansource; - Node *parseTreeCopy; - Query *originalQuery; - - /* copied from ExplainExecuteQuery, will never trigger if you used PREPARE */ - if (!plansource->fixed_result) - { - ereport(ERROR, (errmsg("EXPLAIN EXECUTE does not support variable-result" - " cached plans"))); - } - - parseTreeCopy = copyObject(plansource->raw_parse_tree); - - originalQuery = parse_analyze(parseTreeCopy, - plansource->query_string, - plansource->param_types, - plansource->num_params); - - if (ExtractFirstDistributedTableId(originalQuery) != InvalidOid) - { - /* - * since pg no longer sees EXECUTE it will use the explain hook we've - * installed - */ - explainStmt->query = (Node *) originalQuery; - standard_ProcessUtility(parsetree, plansource->query_string, context, - params, dest, completionTag); - return; - } - - /* if this is a normal query fall through to the usual executor */ - } - } - if (commandMustRunAsOwner) { GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index 46d410c90..25fa2ab9c 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -65,9 +65,6 @@ typedef struct RemoteExplainPlan /* Explain functions for distributed queries */ -static void ExplainMasterPlan(PlannedStmt *masterPlan, IntoClause *into, - ExplainState *es, const char *queryString, - ParamListInfo params, const instr_time *planDuration); static void ExplainJob(Job *job, ExplainState *es); static void ExplainMapMergeJob(MapMergeJob *mapMergeJob, ExplainState *es); static void ExplainTaskList(List *taskList, ExplainState *es); @@ -77,9 +74,6 @@ static void ExplainTask(Task *task, int placementIndex, List *explainOutputList, static void ExplainTaskPlacement(ShardPlacement *taskPlacement, List *explainOutputList, ExplainState *es); static StringInfo BuildRemoteExplainQuery(char *queryString, ExplainState *es); -static void MultiExplainOnePlan(PlannedStmt *plan, IntoClause *into, - ExplainState *es, const char *queryString, - ParamListInfo params, const instr_time *planDuration); /* Static Explain functions copied from explain.c */ static void ExplainOpenGroup(const char *objtype, const char *labelname, @@ -92,288 +86,29 @@ static void ExplainYAMLLineStarting(ExplainState *es); /* - * MultiExplainOneQuery takes the given query, and checks if the query is local - * or distributed. If the query is local, the function runs the standard explain - * logic. If the query is distributed, the function looks up configuration and - * prints out the distributed logical and physical plans as appropriate. + * CitusExplainScan is a custom scan explain callback function which is used to + * print explain information of a Citus plan which includes both master and + * distributed plan. */ void -MultiExplainOneQuery(Query *query, IntoClause *into, ExplainState *es, - const char *queryString, ParamListInfo params) +CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es) { - instr_time planStart; - instr_time planDuration; - int cursorOptions = 0; - PlannedStmt *plan = NULL; - -#if PG_VERSION_NUM >= 90600 - - /* - * Allow parallel plans in 9.6+ unless selecting into a table. - * Without this, we're breaking explain for non-Citus plans. - */ - if (!into) - { - cursorOptions |= CURSOR_OPT_PARALLEL_OK; - } -#endif - - /* plan query, just like ExplainOneQuery does */ - INSTR_TIME_SET_CURRENT(planStart); - - /* plan the query */ - plan = pg_plan_query(query, cursorOptions, params); - - INSTR_TIME_SET_CURRENT(planDuration); - INSTR_TIME_SUBTRACT(planDuration, planStart); - - /* if not a distributed query, use plain explain infrastructure */ - if (!HasCitusToplevelNode(plan)) - { - /* run it (if needed) and produce output */ - ExplainOnePlan(plan, into, es, queryString, params, &planDuration); - } - else - { - MultiExplainOnePlan(plan, into, es, queryString, params, &planDuration); - } -} - - -/* - * MultiExplainOnePlan explains the plan for an individual distributed query. - */ -static void -MultiExplainOnePlan(PlannedStmt *plan, IntoClause *into, - ExplainState *es, const char *queryString, - ParamListInfo params, const instr_time *planDuration) -{ - MultiPlan *multiPlan = NULL; - CmdType commandType = CMD_UNKNOWN; - Job *workerJob = NULL; - bool routerExecutablePlan = false; - - commandType = plan->commandType; - if (commandType == CMD_INSERT || commandType == CMD_UPDATE || - commandType == CMD_DELETE) - { - if (es->analyze) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("Using ANALYZE for INSERT/UPDATE/DELETE on " - "distributed tables is not supported."))); - } - } - - multiPlan = GetMultiPlan(plan); - - /* ensure plan is executable */ - VerifyMultiPlanValidity(multiPlan); + CitusScanState *scanState = (CitusScanState *) node; + MultiPlan *multiPlan = scanState->multiPlan; if (!ExplainDistributedQueries) { + appendStringInfoSpaces(es->str, es->indent * 2); appendStringInfo(es->str, "explain statements for distributed queries "); appendStringInfo(es->str, "are not enabled\n"); return; } - ExplainOpenGroup("Distributed Query", NULL, true, es); + ExplainOpenGroup("Distributed Query", "Distributed Query", true, es); - if (es->format == EXPLAIN_FORMAT_TEXT) - { - appendStringInfoSpaces(es->str, es->indent * 2); - appendStringInfo(es->str, "Distributed Query"); + ExplainJob(multiPlan->workerJob, es); - if (multiPlan->masterTableName != NULL) - { - appendStringInfo(es->str, " into %s", multiPlan->masterTableName); - } - - appendStringInfo(es->str, "\n"); - - es->indent += 1; - } - - routerExecutablePlan = multiPlan->routerExecutable; - - if (routerExecutablePlan) - { - ExplainPropertyText("Executor", "Router", es); - } - else - { - switch (TaskExecutorType) - { - case MULTI_EXECUTOR_REAL_TIME: - { - ExplainPropertyText("Executor", "Real-Time", es); - } - break; - - case MULTI_EXECUTOR_TASK_TRACKER: - { - ExplainPropertyText("Executor", "Task-Tracker", es); - } - break; - - default: - { - ExplainPropertyText("Executor", "Other", es); - } - break; - } - } - - workerJob = multiPlan->workerJob; - ExplainJob(workerJob, es); - - if (es->format == EXPLAIN_FORMAT_TEXT) - { - es->indent -= 1; - } - - if (!routerExecutablePlan) - { - if (es->format == EXPLAIN_FORMAT_TEXT) - { - appendStringInfoSpaces(es->str, es->indent * 2); - appendStringInfo(es->str, "Master Query\n"); - es->indent += 1; - } - - ExplainOpenGroup("Master Query", "Master Query", false, es); - - ExplainMasterPlan(plan, into, es, queryString, params, planDuration); - - ExplainCloseGroup("Master Query", "Master Query", false, es); - - if (es->format == EXPLAIN_FORMAT_TEXT) - { - es->indent -= 1; - } - } - - ExplainCloseGroup("Distributed Query", NULL, true, es); -} - - -/* - * ExplainMasterPlan generates EXPLAIN output for the master query that merges results. - * When using EXPLAIN ANALYZE, this function shows the execution time of the master query - * in isolation. Calling ExplainOnePlan directly would show the overall execution time of - * the distributed query, which makes it hard to determine how much time the master query - * took. - * - * Parts of this function are copied directly from ExplainOnePlan. - */ -static void -ExplainMasterPlan(PlannedStmt *masterPlan, IntoClause *into, - ExplainState *es, const char *queryString, - ParamListInfo params, const instr_time *planDuration) -{ - DestReceiver *dest = NULL; - int eflags = 0; - QueryDesc *queryDesc = NULL; - int instrument_option = 0; - - if (es->analyze && es->timing) - { - instrument_option |= INSTRUMENT_TIMER; - } - else if (es->analyze) - { - instrument_option |= INSTRUMENT_ROWS; - } - - if (es->buffers) - { - instrument_option |= INSTRUMENT_BUFFERS; - } - - /* - * Use a snapshot with an updated command ID to ensure this query sees - * results of any previously executed queries. - */ - PushCopiedSnapshot(GetActiveSnapshot()); - UpdateActiveSnapshotCommandId(); - - /* - * Normally we discard the query's output, but if explaining CREATE TABLE - * AS, we'd better use the appropriate tuple receiver. - */ - if (into) - { - dest = CreateIntoRelDestReceiver(into); - } - else - { - dest = None_Receiver; - } - - /* Create a QueryDesc for the query */ - queryDesc = CreateQueryDesc(masterPlan, queryString, - GetActiveSnapshot(), InvalidSnapshot, - dest, params, instrument_option); - - /* Select execution options */ - if (es->analyze) - { - eflags = 0; /* default run-to-completion flags */ - } - else - { - eflags = EXEC_FLAG_EXPLAIN_ONLY; - } - if (into) - { - eflags |= GetIntoRelEFlags(into); - } - - /* - * ExecutorStart creates the merge table. If using ANALYZE, it also executes the - * worker job and populates the merge table. - */ - ExecutorStart(queryDesc, eflags); - - if (es->analyze) - { - ScanDirection dir; - - /* if using analyze, then finish query execution */ - - /* EXPLAIN ANALYZE CREATE TABLE AS WITH NO DATA is weird */ - if (into && into->skipData) - { - dir = NoMovementScanDirection; - } - else - { - dir = ForwardScanDirection; - } - - /* run the plan */ - ExecutorRun(queryDesc, dir, 0L); - - /* run cleanup too */ - ExecutorFinish(queryDesc); - } - - /* - * ExplainOnePlan executes the master query again, which ensures that the execution - * time only shows the execution time of the master query itself, instead of the - * overall execution time. - */ - ExplainOnePlan(queryDesc->plannedstmt, into, es, queryString, params, planDuration); - - /* - * ExecutorEnd for the distributed query is deferred until after the master query - * is executed again, otherwise the merge table would be dropped. - */ - ExecutorEnd(queryDesc); - - FreeQueryDesc(queryDesc); - - PopActiveSnapshot(); + ExplainCloseGroup("Distributed Query", "Distributed Query", true, es); } diff --git a/src/backend/distributed/planner/multi_master_planner.c b/src/backend/distributed/planner/multi_master_planner.c index 3210fe42d..6b4be984f 100644 --- a/src/backend/distributed/planner/multi_master_planner.c +++ b/src/backend/distributed/planner/multi_master_planner.c @@ -15,6 +15,7 @@ #include "distributed/multi_master_planner.h" #include "distributed/multi_physical_planner.h" +#include "distributed/multi_planner.h" #include "distributed/multi_server_executor.h" #include "distributed/worker_protocol.h" #include "nodes/makefuncs.h" @@ -67,49 +68,6 @@ MasterTargetList(List *workerTargetList) } -/* - * BuildCreateStatement builds the executable create statement for creating a - * temporary table on the master; and then returns this create statement. This - * function obtains the needed column type information from the target list. - */ -static CreateStmt * -BuildCreateStatement(char *masterTableName, List *masterTargetList, - List *masterColumnNameList) -{ - CreateStmt *createStatement = NULL; - RangeVar *relation = NULL; - char *relationName = NULL; - List *columnTypeList = NIL; - List *columnDefinitionList = NIL; - ListCell *masterTargetCell = NULL; - - /* build rangevar object for temporary table */ - relationName = masterTableName; - relation = makeRangeVar(NULL, relationName, -1); - relation->relpersistence = RELPERSISTENCE_TEMP; - - /* build the list of column types as cstrings */ - foreach(masterTargetCell, masterTargetList) - { - TargetEntry *targetEntry = (TargetEntry *) lfirst(masterTargetCell); - Var *column = (Var *) targetEntry->expr; - Oid columnTypeId = exprType((Node *) column); - int32 columnTypeMod = exprTypmod((Node *) column); - - char *columnTypeName = format_type_with_typemod(columnTypeId, columnTypeMod); - columnTypeList = lappend(columnTypeList, columnTypeName); - } - - /* build the column definition list */ - columnDefinitionList = ColumnDefinitionList(masterColumnNameList, columnTypeList); - - /* build the create statement */ - createStatement = CreateStatement(relation, columnDefinitionList); - - return createStatement; -} - - /* * BuildAggregatePlan creates and returns an aggregate plan. This aggregate plan * builds aggreation and grouping operators (if any) that are to be executed on @@ -207,61 +165,57 @@ BuildAggregatePlan(Query *masterQuery, Plan *subPlan) /* * BuildSelectStatement builds the final select statement to run on the master - * node, before returning results to the user. The function first builds a scan - * statement for all results fetched to the master, and layers aggregation, sort + * node, before returning results to the user. The function first gets the custom + * scan node for all results fetched to the master, and layers aggregation, sort * and limit plans on top of the scan statement if necessary. */ static PlannedStmt * -BuildSelectStatement(Query *masterQuery, char *masterTableName, - List *masterTargetList) +BuildSelectStatement(Query *masterQuery, List *masterTargetList, CustomScan *remoteScan) { PlannedStmt *selectStatement = NULL; - RangeTblEntry *rangeTableEntry = NULL; - RangeTblEntry *queryRangeTableEntry = NULL; - SeqScan *sequentialScan = NULL; + RangeTblEntry *customScanRangeTableEntry = NULL; Agg *aggregationPlan = NULL; Plan *topLevelPlan = NULL; + ListCell *targetEntryCell = NULL; + List *columnNameList = NULL; /* (1) make PlannedStmt and set basic information */ selectStatement = makeNode(PlannedStmt); selectStatement->canSetTag = true; - selectStatement->relationOids = NIL; /* to be filled in exec_Start */ + selectStatement->relationOids = NIL; selectStatement->commandType = CMD_SELECT; - /* prepare the range table entry for our temporary table */ + /* top level select query should have only one range table entry */ Assert(list_length(masterQuery->rtable) == 1); - queryRangeTableEntry = (RangeTblEntry *) linitial(masterQuery->rtable); - rangeTableEntry = copyObject(queryRangeTableEntry); - rangeTableEntry->rtekind = RTE_RELATION; - rangeTableEntry->eref = makeAlias(masterTableName, NIL); - rangeTableEntry->relid = 0; /* to be filled in exec_Start */ - rangeTableEntry->inh = false; - rangeTableEntry->inFromCl = true; + /* compute column names for the custom range table entry */ + foreach(targetEntryCell, masterTargetList) + { + TargetEntry *targetEntry = lfirst(targetEntryCell); + columnNameList = lappend(columnNameList, makeString(targetEntry->resname)); + } + + customScanRangeTableEntry = RemoteScanRangeTableEntry(columnNameList); /* set the single element range table list */ - selectStatement->rtable = list_make1(rangeTableEntry); + selectStatement->rtable = list_make1(customScanRangeTableEntry); - /* (2) build and initialize sequential scan node */ - sequentialScan = makeNode(SeqScan); - sequentialScan->scanrelid = 1; /* always one */ - - /* (3) add an aggregation plan if needed */ + /* (2) add an aggregation plan if needed */ if (masterQuery->hasAggs || masterQuery->groupClause) { - sequentialScan->plan.targetlist = masterTargetList; + remoteScan->scan.plan.targetlist = masterTargetList; - aggregationPlan = BuildAggregatePlan(masterQuery, (Plan *) sequentialScan); + aggregationPlan = BuildAggregatePlan(masterQuery, &remoteScan->scan.plan); topLevelPlan = (Plan *) aggregationPlan; } else { /* otherwise set the final projections on the scan plan directly */ - sequentialScan->plan.targetlist = masterQuery->targetList; - topLevelPlan = (Plan *) sequentialScan; + remoteScan->scan.plan.targetlist = masterQuery->targetList; + topLevelPlan = &remoteScan->scan.plan; } - /* (4) add a sorting plan if needed */ + /* (3) add a sorting plan if needed */ if (masterQuery->sortClause) { List *sortClauseList = masterQuery->sortClause; @@ -279,7 +233,7 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName, topLevelPlan = (Plan *) sortPlan; } - /* (5) add a limit plan if needed */ + /* (4) add a limit plan if needed */ if (masterQuery->limitCount || masterQuery->limitOffset) { Node *limitCount = masterQuery->limitCount; @@ -296,7 +250,7 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName, topLevelPlan = (Plan *) limitPlan; } - /* (6) finally set our top level plan in the plan tree */ + /* (5) finally set our top level plan in the plan tree */ selectStatement->planTree = topLevelPlan; return selectStatement; @@ -304,113 +258,24 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName, /* - * ValueToStringList walks over the given list of string value types, converts - * value types to cstrings, and adds these cstrings into a new list. - */ -static List * -ValueToStringList(List *valueList) -{ - List *stringList = NIL; - ListCell *valueCell = NULL; - - foreach(valueCell, valueList) - { - Value *value = (Value *) lfirst(valueCell); - char *stringValue = strVal(value); - - stringList = lappend(stringList, stringValue); - } - - return stringList; -} - - -/* - * MasterNodeCreateStatement takes in a multi plan, and constructs a statement - * to create a temporary table on the master node for final result - * aggregation. - */ -CreateStmt * -MasterNodeCreateStatement(MultiPlan *multiPlan) -{ - Query *masterQuery = multiPlan->masterQuery; - Job *workerJob = multiPlan->workerJob; - List *workerTargetList = workerJob->jobQuery->targetList; - List *rangeTableList = masterQuery->rtable; - char *tableName = multiPlan->masterTableName; - CreateStmt *createStatement = NULL; - - RangeTblEntry *rangeTableEntry = (RangeTblEntry *) linitial(rangeTableList); - List *columnNameValueList = rangeTableEntry->eref->colnames; - List *columnNameList = ValueToStringList(columnNameValueList); - List *targetList = MasterTargetList(workerTargetList); - - createStatement = BuildCreateStatement(tableName, targetList, columnNameList); - - return createStatement; -} - - -/* - * MasterNodeSelectPlan takes in a distributed plan, finds the master node query - * structure in that plan, and builds the final select plan to execute on the - * master node. Note that this select plan is executed after result files are - * retrieved from worker nodes and are merged into a temporary table. + * MasterNodeSelectPlan takes in a distributed plan and a custom scan node which + * wraps remote part of the plan. This function finds the master node query + * structure in the multi plan, and builds the final select plan to execute on + * the tuples returned by remote scan on the master node. Note that this select + * plan is executed after result files are retrieved from worker nodes and + * filled into the tuple store inside provided custom scan. */ PlannedStmt * -MasterNodeSelectPlan(MultiPlan *multiPlan) +MasterNodeSelectPlan(MultiPlan *multiPlan, CustomScan *remoteScan) { Query *masterQuery = multiPlan->masterQuery; - char *tableName = multiPlan->masterTableName; PlannedStmt *masterSelectPlan = NULL; Job *workerJob = multiPlan->workerJob; List *workerTargetList = workerJob->jobQuery->targetList; List *masterTargetList = MasterTargetList(workerTargetList); - masterSelectPlan = BuildSelectStatement(masterQuery, tableName, masterTargetList); + masterSelectPlan = BuildSelectStatement(masterQuery, masterTargetList, remoteScan); return masterSelectPlan; } - - -/* - * MasterNodeCopyStatementList takes in a multi plan, and constructs - * statements that copy over worker task results to a temporary table on the - * master node. - */ -List * -MasterNodeCopyStatementList(MultiPlan *multiPlan) -{ - Job *workerJob = multiPlan->workerJob; - List *workerTaskList = workerJob->taskList; - char *tableName = multiPlan->masterTableName; - List *copyStatementList = NIL; - - ListCell *workerTaskCell = NULL; - foreach(workerTaskCell, workerTaskList) - { - Task *workerTask = (Task *) lfirst(workerTaskCell); - StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId); - StringInfo taskFilename = TaskFilename(jobDirectoryName, workerTask->taskId); - - RangeVar *relation = makeRangeVar(NULL, tableName, -1); - CopyStmt *copyStatement = makeNode(CopyStmt); - copyStatement->relation = relation; - copyStatement->is_from = true; - copyStatement->filename = taskFilename->data; - if (BinaryMasterCopyFormat) - { - DefElem *copyOption = makeDefElem("format", (Node *) makeString("binary")); - copyStatement->options = list_make1(copyOption); - } - else - { - copyStatement->options = NIL; - } - - copyStatementList = lappend(copyStatementList, copyStatement); - } - - return copyStatementList; -} diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 22ee7aa60..a33808d5b 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -196,9 +196,7 @@ MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) { MultiPlan *multiPlan = NULL; - StringInfo jobSchemaName = NULL; Job *workerJob = NULL; - uint64 workerJobId = 0; Query *masterQuery = NULL; List *masterDependedJobList = NIL; @@ -207,10 +205,6 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) /* create the tree of executable tasks for the worker job */ workerJob = BuildJobTreeTaskList(workerJob); - workerJobId = workerJob->jobId; - - /* get job schema name */ - jobSchemaName = JobSchemaName(workerJobId); /* build the final merge query to execute on the master */ masterDependedJobList = list_make1(workerJob); @@ -219,8 +213,8 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) multiPlan = CitusMakeNode(MultiPlan); multiPlan->workerJob = workerJob; multiPlan->masterQuery = masterQuery; - multiPlan->masterTableName = jobSchemaName->data; multiPlan->routerExecutable = MultiPlanRouterExecutable(multiPlan); + multiPlan->operation = CMD_SELECT; return multiPlan; } diff --git a/src/backend/distributed/planner/multi_planner.c b/src/backend/distributed/planner/multi_planner.c index ca8ea5769..7839c7cfc 100644 --- a/src/backend/distributed/planner/multi_planner.c +++ b/src/backend/distributed/planner/multi_planner.c @@ -16,36 +16,55 @@ #include "distributed/citus_nodefuncs.h" #include "distributed/citus_nodes.h" #include "distributed/metadata_cache.h" +#include "distributed/multi_executor.h" #include "distributed/multi_planner.h" #include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_planner.h" #include "distributed/multi_physical_planner.h" +#include "distributed/multi_master_planner.h" #include "distributed/multi_router_planner.h" - #include "executor/executor.h" - #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" - #include "optimizer/planner.h" - #include "utils/memutils.h" static List *relationRestrictionContextList = NIL; +/* create custom scan methods for separate executors */ +static CustomScanMethods RealTimeCustomScanMethods = { + "Citus Real-Time", + RealTimeCreateScan +}; + +static CustomScanMethods TaskTrackerCustomScanMethods = { + "Citus Task-Tracker", + TaskTrackerCreateScan +}; + +static CustomScanMethods RouterCustomScanMethods = { + "Citus Router", + RouterCreateScan +}; + +static CustomScanMethods DelayedErrorCustomScanMethods = { + "Citus Delayed Error", + DelayedErrorCreateScan +}; + /* local function forward declarations */ +static PlannedStmt * CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, + Query *query, ParamListInfo boundParams, + RelationRestrictionContext *restrictionContext); +static Node * SerializeMultiPlan(struct MultiPlan *multiPlan); +static MultiPlan * DeserializeMultiPlan(Node *node); +static PlannedStmt * FinalizePlan(PlannedStmt *localPlan, MultiPlan *multiPlan); +static PlannedStmt * FinalizeNonRouterPlan(PlannedStmt *localPlan, MultiPlan *multiPlan, + CustomScan *customScan); +static PlannedStmt * FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan); static void CheckNodeIsDumpable(Node *node); -static char * GetMultiPlanString(PlannedStmt *result); -static PlannedStmt * MultiQueryContainerNode(PlannedStmt *result, - struct MultiPlan *multiPlan); -static struct PlannedStmt * CreateDistributedPlan(PlannedStmt *localPlan, - Query *originalQuery, - Query *query, - ParamListInfo boundParams, - RelationRestrictionContext * - restrictionContext); static RelationRestrictionContext * CreateAndPushRestrictionContext(void); static RelationRestrictionContext * CurrentRestrictionContext(void); static void PopRestrictionContext(void); @@ -143,22 +162,21 @@ IsModifyCommand(Query *query) /* - * VerifyMultiPlanValidity verifies that multiPlan is ready for execution, or - * errors out if not. - * - * A plan may e.g. not be ready for execution because CreateDistributedPlan() - * couldn't find a plan due to unresolved prepared statement parameters, but - * didn't error out, because we expect custom plans to come to our rescue. - * But sql (not plpgsql) functions unfortunately don't go through a codepath - * supporting custom plans. + * IsModifyMultiPlan returns true if the multi plan performs modifications, + * false otherwise. */ -void -VerifyMultiPlanValidity(MultiPlan *multiPlan) +bool +IsModifyMultiPlan(MultiPlan *multiPlan) { - if (multiPlan->planningError) + bool isModifyMultiPlan = false; + CmdType operation = multiPlan->operation; + + if (operation == CMD_INSERT || operation == CMD_UPDATE || operation == CMD_DELETE) { - RaiseDeferredError(multiPlan->planningError, ERROR); + isModifyMultiPlan = true; } + + return isModifyMultiPlan; } @@ -273,8 +291,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query RaiseDeferredError(distributedPlan->planningError, ERROR); } - /* store required data into the planned statement */ - resultPlan = MultiQueryContainerNode(localPlan, distributedPlan); + /* create final plan by combining local plan with distributed plan */ + resultPlan = FinalizePlan(localPlan, distributedPlan); /* * As explained above, force planning costs to be unrealistically high if @@ -294,16 +312,63 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query /* - * GetMultiPlan returns the associated MultiPlan for a PlannedStmt if the - * statement requires distributed execution, NULL otherwise. + * GetMultiPlan returns the associated MultiPlan for a CustomScan. */ MultiPlan * -GetMultiPlan(PlannedStmt *result) +GetMultiPlan(CustomScan *customScan) { + MultiPlan *multiPlan = NULL; + + Assert(list_length(customScan->custom_private) == 1); + + multiPlan = DeserializeMultiPlan(linitial(customScan->custom_private)); + + return multiPlan; +} + + +/* + * SerializeMultiPlan returns the string representing the distributed plan in a + * Const node. + * + * Note that this should be improved for 9.6+, we we can copy trees efficiently. + * I.e. we should introduce copy support for relevant node types, and just + * return the MultiPlan as-is for 9.6. + */ +static Node * +SerializeMultiPlan(MultiPlan *multiPlan) +{ + char *serializedMultiPlan = NULL; + Const *multiPlanData = NULL; + + serializedMultiPlan = CitusNodeToString(multiPlan); + + multiPlanData = makeNode(Const); + multiPlanData->consttype = CSTRINGOID; + multiPlanData->constlen = strlen(serializedMultiPlan); + multiPlanData->constvalue = CStringGetDatum(serializedMultiPlan); + multiPlanData->constbyval = false; + multiPlanData->location = -1; + + return (Node *) multiPlanData; +} + + +/* + * DeserializeMultiPlan returns the deserialized distributed plan from the string + * representation in a Const node. + */ +static MultiPlan * +DeserializeMultiPlan(Node *node) +{ + Const *multiPlanData = NULL; char *serializedMultiPlan = NULL; MultiPlan *multiPlan = NULL; - serializedMultiPlan = GetMultiPlanString(result); + Assert(IsA(node, Const)); + multiPlanData = (Const *) node; + serializedMultiPlan = DatumGetCString(multiPlanData->constvalue); + multiPlan = (MultiPlan *) CitusStringToNode(serializedMultiPlan); Assert(CitusIsA(multiPlan, MultiPlan)); @@ -311,159 +376,172 @@ GetMultiPlan(PlannedStmt *result) } -/* Does the passed in statement require distributed execution? */ -bool -HasCitusToplevelNode(PlannedStmt *result) +/* + * FinalizePlan combines local plan with distributed plan and creates a plan + * which can be run by the PostgreSQL executor. + */ +static PlannedStmt * +FinalizePlan(PlannedStmt *localPlan, MultiPlan *multiPlan) { - /* - * Can't be a distributed query if the extension hasn't been loaded - * yet. Directly return false, part of the required infrastructure for - * further checks might not be present. - */ - if (!CitusHasBeenLoaded()) + PlannedStmt *finalPlan = NULL; + CustomScan *customScan = makeNode(CustomScan); + Node *multiPlanData = NULL; + MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST; + + if (!multiPlan->planningError) { - return false; + executorType = JobExecutorType(multiPlan); } - if (GetMultiPlanString(result) == NULL) + switch (executorType) { - return false; + case MULTI_EXECUTOR_REAL_TIME: + { + customScan->methods = &RealTimeCustomScanMethods; + break; + } + + case MULTI_EXECUTOR_TASK_TRACKER: + { + customScan->methods = &TaskTrackerCustomScanMethods; + break; + } + + case MULTI_EXECUTOR_ROUTER: + { + customScan->methods = &RouterCustomScanMethods; + break; + } + + default: + { + customScan->methods = &DelayedErrorCustomScanMethods; + break; + } + } + + multiPlanData = SerializeMultiPlan(multiPlan); + + customScan->custom_private = list_make1(multiPlanData); + customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN; + + /* check if we have a master query */ + if (multiPlan->masterQuery) + { + finalPlan = FinalizeNonRouterPlan(localPlan, multiPlan, customScan); } else { - return true; + finalPlan = FinalizeRouterPlan(localPlan, customScan); } + + return finalPlan; } /* - * CreateCitusToplevelNode creates the top-level planTree node for a - * distributed statement. That top-level node is a) recognizable by the - * executor hooks, allowing them to redirect execution, b) contains the - * parameters required for distributed execution. - * - * The exact representation of the top-level node is an implementation detail - * which should not be referred to outside this file, as it's likely to become - * version dependant. Use GetMultiPlan() and HasCitusToplevelNode() to access. - * - * Internally the data is stored as arguments to a 'citus_extradata_container' - * function, which has to be removed from the really executed plan tree before - * query execution. + * FinalizeNonRouterPlan gets the distributed custom scan plan, and creates the + * final master select plan on the top of this distributed plan for real-time + * and task-tracker executors. */ -PlannedStmt * -MultiQueryContainerNode(PlannedStmt *result, MultiPlan *multiPlan) +static PlannedStmt * +FinalizeNonRouterPlan(PlannedStmt *localPlan, MultiPlan *multiPlan, + CustomScan *customScan) { - FunctionScan *fauxFunctionScan = NULL; - RangeTblFunction *fauxFunction = NULL; - FuncExpr *fauxFuncExpr = NULL; - Const *multiPlanData = NULL; - char *serializedPlan = NULL; + PlannedStmt *finalPlan = NULL; - /* pass multiPlan serialized as a constant function argument */ - serializedPlan = CitusNodeToString(multiPlan); - multiPlanData = makeNode(Const); - multiPlanData->consttype = CSTRINGOID; - multiPlanData->constlen = strlen(serializedPlan); - multiPlanData->constvalue = CStringGetDatum(serializedPlan); - multiPlanData->constbyval = false; - multiPlanData->location = -1; + finalPlan = MasterNodeSelectPlan(multiPlan, customScan); + finalPlan->queryId = localPlan->queryId; + finalPlan->utilityStmt = localPlan->utilityStmt; - fauxFuncExpr = makeNode(FuncExpr); - fauxFuncExpr->funcid = CitusExtraDataContainerFuncId(); - fauxFuncExpr->funcretset = true; - fauxFuncExpr->location = -1; - - fauxFuncExpr->args = list_make1(multiPlanData); - fauxFunction = makeNode(RangeTblFunction); - fauxFunction->funcexpr = (Node *) fauxFuncExpr; - - fauxFunctionScan = makeNode(FunctionScan); - fauxFunctionScan->functions = lappend(fauxFunctionScan->functions, fauxFunction); - - /* copy original targetlist, accessed for RETURNING queries */ - fauxFunctionScan->scan.plan.targetlist = copyObject(result->planTree->targetlist); - - /* - * Add set returning function to target list if the original (postgres - * created) plan doesn't support backward scans; doing so prevents - * backward scans being supported by the new plantree as well. This is - * ugly as hell, but until we can rely on custom scans (which can signal - * this via CUSTOMPATH_SUPPORT_BACKWARD_SCAN), there's not really a pretty - * method to achieve this. - * - * FIXME: This should really be done on the master select plan. - */ - if (!ExecSupportsBackwardScan(result->planTree)) - { - FuncExpr *funcExpr = makeNode(FuncExpr); - TargetEntry *targetEntry = NULL; - bool resjunkAttribute = true; - - funcExpr->funcretset = true; - - targetEntry = makeTargetEntry((Expr *) funcExpr, InvalidAttrNumber, NULL, - resjunkAttribute); - - fauxFunctionScan->scan.plan.targetlist = - lappend(fauxFunctionScan->scan.plan.targetlist, - targetEntry); - } - - result->planTree = (Plan *) fauxFunctionScan; - - return result; + return finalPlan; } /* - * GetMultiPlanString returns either NULL, if the plan is not a distributed - * one, or the string representing the distributed plan. + * FinalizeRouterPlan gets a CustomScan node which already wrapped distributed + * part of a router plan and sets it as the direct child of the router plan + * because we don't run any query on master node for router executable queries. + * Here, we also rebuild the column list to read from the remote scan. */ -static char * -GetMultiPlanString(PlannedStmt *result) +static PlannedStmt * +FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan) { - FunctionScan *fauxFunctionScan = NULL; - RangeTblFunction *fauxFunction = NULL; - FuncExpr *fauxFuncExpr = NULL; - Const *multiPlanData = NULL; + PlannedStmt *routerPlan = NULL; + RangeTblEntry *remoteScanRangeTableEntry = NULL; + ListCell *targetEntryCell = NULL; + List *targetList = NIL; + List *columnNameList = NIL; - if (!IsA(result->planTree, FunctionScan)) + /* we will have only one range table entry */ + int customScanRangeTableIndex = 1; + + /* build a targetlist to read from the custom scan output */ + foreach(targetEntryCell, localPlan->planTree->targetlist) { - return NULL; + TargetEntry *targetEntry = lfirst(targetEntryCell); + TargetEntry *newTargetEntry = NULL; + Var *newVar = NULL; + Value *columnName = NULL; + + Assert(IsA(targetEntry, TargetEntry)); + + /* + * This is unlikely to be hit because we would not need resjunk stuff + * at the toplevel of a router query - all things needing it have been + * pushed down. + */ + if (targetEntry->resjunk) + { + continue; + } + + /* build target entry pointing to remote scan range table entry */ + newVar = makeVarFromTargetEntry(customScanRangeTableIndex, targetEntry); + newTargetEntry = flatCopyTargetEntry(targetEntry); + newTargetEntry->expr = (Expr *) newVar; + targetList = lappend(targetList, newTargetEntry); + + columnName = makeString(targetEntry->resname); + columnNameList = lappend(columnNameList, columnName); } - fauxFunctionScan = (FunctionScan *) result->planTree; + customScan->scan.plan.targetlist = targetList; - if (list_length(fauxFunctionScan->functions) != 1) - { - return NULL; - } + routerPlan = makeNode(PlannedStmt); + routerPlan->planTree = (Plan *) customScan; - fauxFunction = linitial(fauxFunctionScan->functions); + remoteScanRangeTableEntry = RemoteScanRangeTableEntry(columnNameList); + routerPlan->rtable = list_make1(remoteScanRangeTableEntry); - if (!IsA(fauxFunction->funcexpr, FuncExpr)) - { - return NULL; - } + routerPlan->canSetTag = true; + routerPlan->relationOids = NIL; - fauxFuncExpr = (FuncExpr *) fauxFunction->funcexpr; + routerPlan->queryId = localPlan->queryId; + routerPlan->utilityStmt = localPlan->utilityStmt; + routerPlan->commandType = localPlan->commandType; + routerPlan->hasReturning = localPlan->hasReturning; - if (fauxFuncExpr->funcid != CitusExtraDataContainerFuncId()) - { - return NULL; - } + return routerPlan; +} - if (list_length(fauxFuncExpr->args) != 1) - { - ereport(ERROR, (errmsg("unexpected number of function arguments to " - "citus_extradata_container"))); - } - multiPlanData = (Const *) linitial(fauxFuncExpr->args); - Assert(IsA(multiPlanData, Const)); - Assert(multiPlanData->consttype == CSTRINGOID); +/* + * RemoteScanRangeTableEntry creates a range table entry from given column name + * list to represent a remote scan. + */ +RangeTblEntry * +RemoteScanRangeTableEntry(List *columnNameList) +{ + RangeTblEntry *remoteScanRangeTableEntry = makeNode(RangeTblEntry); - return DatumGetCString(multiPlanData->constvalue); + /* we use RTE_VALUES for custom scan because we can't look up relation */ + remoteScanRangeTableEntry->rtekind = RTE_VALUES; + remoteScanRangeTableEntry->eref = makeAlias("remote_scan", columnNameList); + remoteScanRangeTableEntry->inh = false; + remoteScanRangeTableEntry->inFromCl = true; + + return remoteScanRangeTableEntry; } diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 368a96094..39bf01782 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -195,6 +195,8 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query, List *placementList = NIL; MultiPlan *multiPlan = CitusMakeNode(MultiPlan); + multiPlan->operation = query->commandType; + if (commandType == CMD_INSERT || commandType == CMD_UPDATE || commandType == CMD_DELETE) { @@ -234,8 +236,13 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query, multiPlan->workerJob = job; multiPlan->masterQuery = NULL; - multiPlan->masterTableName = NULL; multiPlan->routerExecutable = true; + multiPlan->hasReturning = false; + + if (list_length(originalQuery->returningList) > 0) + { + multiPlan->hasReturning = true; + } return multiPlan; } @@ -264,6 +271,8 @@ CreateInsertSelectRouterPlan(Query *originalQuery, int shardCount = targetCacheEntry->shardIntervalArrayLength; bool allReferenceTables = restrictionContext->allReferenceTables; + multiPlan->operation = originalQuery->commandType; + /* * Error semantics for INSERT ... SELECT queries are different than regular * modify queries. Thus, handle separately. @@ -316,9 +325,14 @@ CreateInsertSelectRouterPlan(Query *originalQuery, /* and finally the multi plan */ multiPlan->workerJob = workerJob; - multiPlan->masterTableName = NULL; multiPlan->masterQuery = NULL; multiPlan->routerExecutable = true; + multiPlan->hasReturning = false; + + if (list_length(originalQuery->returningList) > 0) + { + multiPlan->hasReturning = true; + } return multiPlan; } diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index c400e1e42..689296b08 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -24,7 +24,6 @@ #include "distributed/master_metadata_utility.h" #include "distributed/master_protocol.h" #include "distributed/multi_copy.h" -#include "distributed/multi_executor.h" #include "distributed/multi_explain.h" #include "distributed/multi_join_order.h" #include "distributed/multi_logical_optimizer.h" @@ -115,13 +114,7 @@ _PG_init(void) * (thus as the innermost/last running hook) to be able to do our * duties. For simplicity insist that all hooks are previously unused. */ - if (planner_hook != NULL || - ExplainOneQuery_hook != NULL || - ExecutorStart_hook != NULL || - ExecutorRun_hook != NULL || - ExecutorFinish_hook != NULL || - ExecutorEnd_hook != NULL || - ProcessUtility_hook != NULL) + if (planner_hook != NULL || ProcessUtility_hook != NULL) { ereport(ERROR, (errmsg("Citus has to be loaded first"), errhint("Place citus at the beginning of " @@ -147,15 +140,6 @@ _PG_init(void) /* intercept planner */ planner_hook = multi_planner; - /* intercept explain */ - ExplainOneQuery_hook = MultiExplainOneQuery; - - /* intercept executor */ - ExecutorStart_hook = multi_ExecutorStart; - ExecutorRun_hook = multi_ExecutorRun; - ExecutorFinish_hook = multi_ExecutorFinish; - ExecutorEnd_hook = multi_ExecutorEnd; - /* register utility hook */ ProcessUtility_hook = multi_ProcessUtility; diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 0e9ed1aa6..8aa485857 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -276,9 +276,11 @@ OutMultiPlan(OUTFUNC_ARGS) WRITE_NODE_TYPE("MULTIPLAN"); + WRITE_INT_FIELD(operation); + WRITE_BOOL_FIELD(hasReturning); + WRITE_NODE_FIELD(workerJob); WRITE_NODE_FIELD(masterQuery); - WRITE_STRING_FIELD(masterTableName); WRITE_BOOL_FIELD(routerExecutable); WRITE_NODE_FIELD(planningError); } diff --git a/src/backend/distributed/utils/citus_readfuncs.c b/src/backend/distributed/utils/citus_readfuncs.c index 4d51ae86d..37c9245fa 100644 --- a/src/backend/distributed/utils/citus_readfuncs.c +++ b/src/backend/distributed/utils/citus_readfuncs.c @@ -183,9 +183,11 @@ ReadMultiPlan(READFUNC_ARGS) { READ_LOCALS(MultiPlan); + READ_INT_FIELD(operation); + READ_BOOL_FIELD(hasReturning); + READ_NODE_FIELD(workerJob); READ_NODE_FIELD(masterQuery); - READ_STRING_FIELD(masterTableName); READ_BOOL_FIELD(routerExecutable); READ_NODE_FIELD(planningError); diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 995d9d525..e3b53a327 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -12,10 +12,11 @@ #include "executor/execdesc.h" #include "nodes/parsenodes.h" +#include "nodes/execnodes.h" + +#include "distributed/multi_physical_planner.h" +#include "distributed/multi_server_executor.h" -/* signal currently executed statement is a master select statement or router execution */ -#define EXEC_FLAG_CITUS_MASTER_SELECT 0x100 -#define EXEC_FLAG_CITUS_ROUTER_EXECUTOR 0x200 #if (PG_VERSION_NUM >= 90600) #define tuplecount_t uint64 @@ -23,10 +24,29 @@ #define tuplecount_t long #endif -extern void multi_ExecutorStart(QueryDesc *queryDesc, int eflags); -extern void multi_ExecutorRun(QueryDesc *queryDesc, - ScanDirection direction, tuplecount_t count); -extern void multi_ExecutorFinish(QueryDesc *queryDesc); -extern void multi_ExecutorEnd(QueryDesc *queryDesc); + +typedef struct CitusScanState +{ + CustomScanState customScanState; /* underlying custom scan node */ + MultiPlan *multiPlan; /* distributed execution plan */ + MultiExecutorType executorType; /* distributed executor type */ + bool finishedRemoteScan; /* flag to check if remote scan is finished */ + Tuplestorestate *tuplestorestate; /* tuple store to store distributed results */ +} CitusScanState; + + +extern Node * RealTimeCreateScan(CustomScan *scan); +extern Node * TaskTrackerCreateScan(CustomScan *scan); +extern Node * RouterCreateScan(CustomScan *scan); +extern Node * DelayedErrorCreateScan(CustomScan *scan); +extern void CitusSelectBeginScan(CustomScanState *node, EState *estate, int eflags); +extern TupleTableSlot * RealTimeExecScan(CustomScanState *node); +extern TupleTableSlot * TaskTrackerExecScan(CustomScanState *node); +extern void CitusEndScan(CustomScanState *node); +extern void CitusReScan(CustomScanState *node); +extern void CitusExplainScan(CustomScanState *node, List *ancestors, struct + ExplainState *es); +extern TupleTableSlot * ReturnTupleFromTuplestore(CitusScanState *scanState); + #endif /* MULTI_EXECUTOR_H */ diff --git a/src/include/distributed/multi_explain.h b/src/include/distributed/multi_explain.h index 55f4bf75d..cab0f0960 100644 --- a/src/include/distributed/multi_explain.h +++ b/src/include/distributed/multi_explain.h @@ -16,7 +16,4 @@ extern bool ExplainDistributedQueries; extern bool ExplainAllTasks; -extern void MultiExplainOneQuery(Query *query, IntoClause *into, ExplainState *es, - const char *queryString, ParamListInfo params); - #endif /* MULTI_EXPLAIN_H */ diff --git a/src/include/distributed/multi_master_planner.h b/src/include/distributed/multi_master_planner.h index 970889d08..9873ce4ff 100644 --- a/src/include/distributed/multi_master_planner.h +++ b/src/include/distributed/multi_master_planner.h @@ -21,8 +21,9 @@ /* Function declarations for building local plans on the master node */ struct MultiPlan; -extern CreateStmt * MasterNodeCreateStatement(struct MultiPlan *multiPlan); -extern List * MasterNodeCopyStatementList(struct MultiPlan *multiPlan); -extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan); +struct CustomScan; +extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan, + struct CustomScan *dataScan); + #endif /* MULTI_MASTER_PLANNER_H */ diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 1949afadc..0cf340899 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -213,9 +213,11 @@ typedef struct JoinSequenceNode typedef struct MultiPlan { CitusNode type; + CmdType operation; + + bool hasReturning; Job *workerJob; Query *masterQuery; - char *masterTableName; bool routerExecutable; /* diff --git a/src/include/distributed/multi_planner.h b/src/include/distributed/multi_planner.h index 1c1d7a337..8a24b2efa 100644 --- a/src/include/distributed/multi_planner.h +++ b/src/include/distributed/multi_planner.h @@ -51,12 +51,13 @@ typedef struct RelationShard extern PlannedStmt * multi_planner(Query *parse, int cursorOptions, ParamListInfo boundParams); -extern bool HasCitusToplevelNode(PlannedStmt *planStatement); struct MultiPlan; -extern struct MultiPlan * GetMultiPlan(PlannedStmt *planStatement); +extern struct MultiPlan * GetMultiPlan(CustomScan *node); extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index index, RangeTblEntry *rte); extern bool IsModifyCommand(Query *query); -extern void VerifyMultiPlanValidity(struct MultiPlan *multiPlan); +extern bool IsModifyMultiPlan(struct MultiPlan *multiPlan); +extern RangeTblEntry * RemoteScanRangeTableEntry(List *columnNameList); + #endif /* MULTI_PLANNER_H */ diff --git a/src/include/distributed/multi_router_executor.h b/src/include/distributed/multi_router_executor.h index e369d0f1f..8c9eafb7d 100644 --- a/src/include/distributed/multi_router_executor.h +++ b/src/include/distributed/multi_router_executor.h @@ -12,8 +12,10 @@ #include "c.h" #include "access/sdir.h" +#include "distributed/multi_executor.h" #include "distributed/multi_physical_planner.h" #include "executor/execdesc.h" +#include "executor/tuptable.h" #include "nodes/pg_list.h" @@ -33,12 +35,12 @@ typedef struct XactShardConnSet extern bool AllModificationsCommutative; extern bool EnableDeadlockPrevention; - -extern void RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList); -extern void RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count); -extern void RouterExecutorFinish(QueryDesc *queryDesc); -extern void RouterExecutorEnd(QueryDesc *queryDesc); +extern void CitusModifyBeginScan(CustomScanState *node, EState *estate, int eflags); +extern TupleTableSlot * RouterSingleModifyExecScan(CustomScanState *node); +extern TupleTableSlot * RouterSelectExecScan(CustomScanState *node); +extern TupleTableSlot * RouterMultiModifyExecScan(CustomScanState *node); extern int64 ExecuteModifyTasksWithoutResults(List *taskList); + #endif /* MULTI_ROUTER_EXECUTOR_H_ */ diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index bafe813bb..e2ef27342 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -39,88 +39,81 @@ $BODY$ LANGUAGE plpgsql; EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_570000 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Sort - Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570000_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570000_0 - -> HashAggregate - Group Key: intermediate_column_570000_0 - -> Seq Scan on pg_merge_job_570000 +Sort + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_290001 lineitem -- Test JSON format EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; [ { - "Executor": "Real-Time", - "Job": { - "Task Count": 8, - "Tasks Shown": "One of 8", - "Tasks": [ + "Plan": { + "Node Type": "Sort", + "Parallel Aware": false, + "Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"], + "Plans": [ { - "Node": "host=localhost port=57637 dbname=regression", - "Remote Plan": [ - [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Partial Mode": "Simple", - "Parallel Aware": false, - "Group Key": ["l_quantity"], - "Plans": [ + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Partial Mode": "Simple", + "Parent Relationship": "Outer", + "Parallel Aware": false, + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Real-Time", + "Parallel Aware": false, + "Distributed Query": { + "Job": { + "Task Count": 8, + "Tasks Shown": "One of 8", + "Tasks": [ { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Relation Name": "lineitem_290001", - "Alias": "lineitem" + "Node": "host=localhost port=57637 dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Partial Mode": "Simple", + "Parallel Aware": false, + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Seq Scan", + "Parent Relationship": "Outer", + "Parallel Aware": false, + "Relation Name": "lineitem_290001", + "Alias": "lineitem" + } + ] + } + } + ] + + ] } ] } } - ] - - ] - } - ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Sort", - "Parallel Aware": false, - "Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570001_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_570001_0"], - "Plans": [ - { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Partial Mode": "Simple", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Group Key": ["intermediate_column_570001_0"], - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Relation Name": "pg_merge_job_570001", - "Alias": "pg_merge_job_570001" - } - ] } ] } - } - ] + ] + } } ] -- Validate JSON format @@ -133,75 +126,71 @@ EXPLAIN (COSTS FALSE, FORMAT XML) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; - - Real-Time - - 8 - One of 8 - - - host=localhost port=57637 dbname=regression - - - - - Aggregate - Hashed - Simple - false - - l_quantity - - - - Seq Scan - Outer - false - lineitem_290001 - lineitem - - - - - - - - - - - + + + Sort + false + + COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint) + l_quantity + + - Sort + Aggregate + Hashed + Simple + Outer false - - COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570003_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) - intermediate_column_570003_0 - + + l_quantity + - Aggregate - Hashed - Simple + Custom Scan Outer + Citus Real-Time false - - intermediate_column_570003_0 - - - - Seq Scan - Outer - false - pg_merge_job_570003 - pg_merge_job_570003 - - + + + 8 + One of 8 + + + host=localhost port=57637 dbname=regression + + + + + Aggregate + Hashed + Simple + false + + l_quantity + + + + Seq Scan + Outer + false + lineitem_290001 + lineitem + + + + + + + + + + - - - + + + -- Validate XML format SELECT true AS valid FROM explain_xml($$ @@ -212,115 +201,103 @@ t EXPLAIN (COSTS FALSE, FORMAT YAML) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -- Executor: "Real-Time" - Job: - Task Count: 8 - Tasks Shown: "One of 8" - Tasks: - - Node: "host=localhost port=57637 dbname=regression" - Remote Plan: - - Plan: - Node Type: "Aggregate" - Strategy: "Hashed" - Partial Mode: "Simple" - Parallel Aware: false - Group Key: - - "l_quantity" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Parallel Aware: false - Relation Name: "lineitem_290001" - Alias: "lineitem" - - Master Query: - - Plan: - Node Type: "Sort" +- Plan: + Node Type: "Sort" + Parallel Aware: false + Sort Key: + - "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)" + - "l_quantity" + Plans: + - Node Type: "Aggregate" + Strategy: "Hashed" + Partial Mode: "Simple" + Parent Relationship: "Outer" Parallel Aware: false - Sort Key: - - "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570005_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)" - - "intermediate_column_570005_0" + Group Key: + - "l_quantity" Plans: - - Node Type: "Aggregate" - Strategy: "Hashed" - Partial Mode: "Simple" + - Node Type: "Custom Scan" Parent Relationship: "Outer" + Custom Plan Provider: "Citus Real-Time" Parallel Aware: false - Group Key: - - "intermediate_column_570005_0" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Parallel Aware: false - Relation Name: "pg_merge_job_570005" - Alias: "pg_merge_job_570005" + Distributed Query: + Job: + Task Count: 8 + Tasks Shown: "One of 8" + Tasks: + - Node: "host=localhost port=57637 dbname=regression" + Remote Plan: + - Plan: + Node Type: "Aggregate" + Strategy: "Hashed" + Partial Mode: "Simple" + Parallel Aware: false + Group Key: + - "l_quantity" + Plans: + - Node Type: "Seq Scan" + Parent Relationship: "Outer" + Parallel Aware: false + Relation Name: "lineitem_290001" + Alias: "lineitem" + -- Test Text format EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_570006 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Sort - Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570006_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570006_0 - -> HashAggregate - Group Key: intermediate_column_570006_0 - -> Seq Scan on pg_merge_job_570006 +Sort + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_290001 lineitem -- Test verbose EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem; -Distributed Query into pg_merge_job_570007 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - Output: sum(l_quantity), sum(l_quantity), count(l_quantity) - -> Seq Scan on public.lineitem_290001 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> Aggregate - Output: (sum(intermediate_column_570007_0) / (sum(intermediate_column_570007_1) / pg_catalog.sum(intermediate_column_570007_2))) - -> Seq Scan on pg_temp_2.pg_merge_job_570007 - Output: intermediate_column_570007_0, intermediate_column_570007_1, intermediate_column_570007_2 +Aggregate + Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2"))) + -> Custom Scan (Citus Real-Time) + Output: "?column?", "?column?_1", "?column?_2" + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + Output: sum(l_quantity), sum(l_quantity), count(l_quantity) + -> Seq Scan on public.lineitem_290001 lineitem + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test join EXPLAIN (COSTS FALSE) SELECT * FROM lineitem JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0 ORDER BY l_quantity LIMIT 10; -Distributed Query into pg_merge_job_570008 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Limit - -> Sort - Sort Key: lineitem.l_quantity - -> Hash Join - Hash Cond: (lineitem.l_orderkey = orders.o_orderkey) - -> Seq Scan on lineitem_290001 lineitem - Filter: (l_quantity < 5.0) - -> Hash - -> Seq Scan on orders_290008 orders -Master Query - -> Limit - -> Sort - Sort Key: intermediate_column_570008_4 - -> Seq Scan on pg_merge_job_570008 +Limit + -> Sort + Sort Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: lineitem.l_quantity + -> Hash Join + Hash Cond: (lineitem.l_orderkey = orders.o_orderkey) + -> Seq Scan on lineitem_290001 lineitem + Filter: (l_quantity < 5.0) + -> Hash + -> Seq Scan on orders_290008 orders -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem VALUES(1,0); -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -332,8 +309,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -348,8 +324,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -363,8 +338,7 @@ Distributed Query -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; -Distributed Query into pg_merge_job_570009 - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -383,89 +357,77 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem; -Distributed Query into pg_merge_job_570012 - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task Node: host=localhost port=57637 dbname=regression -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Seq Scan on pg_merge_job_570012 -- Test having EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem HAVING sum(l_quantity) > 100; -Distributed Query into pg_merge_job_570013 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity) - -> Seq Scan on public.lineitem_290001 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> Aggregate - Output: (sum(intermediate_column_570013_0) / (sum(intermediate_column_570013_1) / pg_catalog.sum(intermediate_column_570013_2))) - Filter: (sum(pg_merge_job_570013.intermediate_column_570013_3) > '100'::numeric) - -> Seq Scan on pg_temp_2.pg_merge_job_570013 - Output: intermediate_column_570013_0, intermediate_column_570013_1, intermediate_column_570013_2, intermediate_column_570013_3 +Aggregate + Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2"))) + Filter: (sum(remote_scan.worker_column_4) > '100'::numeric) + -> Custom Scan (Citus Real-Time) + Output: "?column?", "?column?_1", "?column?_2", worker_column_4 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity) + -> Seq Scan on public.lineitem_290001 lineitem + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test having without aggregate EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT l_quantity FROM lineitem GROUP BY l_quantity HAVING l_quantity > (100 * random()); -Distributed Query into pg_merge_job_570014 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Output: l_quantity, l_quantity - Group Key: lineitem.l_quantity - -> Seq Scan on public.lineitem_290001 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> HashAggregate - Output: intermediate_column_570014_0 - Group Key: pg_merge_job_570014.intermediate_column_570014_0 - Filter: ((pg_merge_job_570014.intermediate_column_570014_1)::double precision > ('100'::double precision * random())) - -> Seq Scan on pg_temp_2.pg_merge_job_570014 - Output: intermediate_column_570014_0, intermediate_column_570014_1 +HashAggregate + Output: l_quantity + Group Key: remote_scan.l_quantity + Filter: ((remote_scan.worker_column_2)::double precision > ('100'::double precision * random())) + -> Custom Scan (Citus Real-Time) + Output: l_quantity, worker_column_2 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Output: l_quantity, l_quantity + Group Key: lineitem.l_quantity + -> Seq Scan on public.lineitem_290001 lineitem + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test all tasks output SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_570015 - Executor: Real-Time - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290004 lineitem - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290007 lineitem - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290006 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570015 +Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290004 lineitem + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290007 lineitem + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290006 lineitem + Filter: (l_orderkey > 9030) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$); t @@ -477,18 +439,15 @@ SET citus.task_executor_type TO 'task-tracker'; SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_570018 - Executor: Task-Tracker - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570018 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -497,19 +456,16 @@ EXPLAIN (COSTS FALSE) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -Distributed Query into pg_merge_job_570021 - Executor: Task-Tracker - Task Count: 1 - Tasks Shown: None, not supported for re-partition queries - -> MapMergeJob - Map Task Count: 1 - Merge Task Count: 1 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 1 + Tasks Shown: None, not supported for re-partition queries -> MapMergeJob - Map Task Count: 8 + Map Task Count: 1 Merge Task Count: 1 -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570021 + -> MapMergeJob + Map Task Count: 8 + Merge Task Count: 1 EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT count(*) FROM lineitem, orders, customer, supplier_single_shard @@ -518,42 +474,38 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) AND l_suppkey = s_suppkey; [ { - "Executor": "Task-Tracker", - "Job": { - "Task Count": 1, - "Tasks Shown": "None, not supported for re-partition queries", - "Depended Jobs": [ + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Plain", + "Partial Mode": "Simple", + "Parallel Aware": false, + "Plans": [ { - "Map Task Count": 1, - "Merge Task Count": 1, - "Depended Jobs": [ - { - "Map Task Count": 8, - "Merge Task Count": 1 + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Task-Tracker", + "Parallel Aware": false, + "Distributed Query": { + "Job": { + "Task Count": 1, + "Tasks Shown": "None, not supported for re-partition queries", + "Depended Jobs": [ + { + "Map Task Count": 1, + "Merge Task Count": 1, + "Depended Jobs": [ + { + "Map Task Count": 8, + "Merge Task Count": 1 + } + ] + } + ] } - ] + } } ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Plain", - "Partial Mode": "Simple", - "Parallel Aware": false, - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Relation Name": "pg_merge_job_570024", - "Alias": "pg_merge_job_570024" - } - ] - } - } - ] + } } ] SELECT true AS valid FROM explain_json($$ @@ -570,44 +522,40 @@ EXPLAIN (COSTS FALSE, FORMAT XML) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; - - Task-Tracker - - 1 - None, not supported for re-partition queries - - - 1 - 1 - - - 8 - 1 - - - - - - - + + + Aggregate + Plain + Simple + false + - Aggregate - Plain - Simple + Custom Scan + Outer + Citus Task-Tracker false - - - Seq Scan - Outer - false - pg_merge_job_570030 - pg_merge_job_570030 - - + + + 1 + None, not supported for re-partition queries + + + 1 + 1 + + + 8 + 1 + + + + + + - - - + + + SELECT true AS valid FROM explain_xml($$ SELECT count(*) @@ -635,28 +583,26 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -- Executor: "Task-Tracker" - Job: - Task Count: 1 - Tasks Shown: "None, not supported for re-partition queries" - Depended Jobs: - - Map Task Count: 1 - Merge Task Count: 1 - Depended Jobs: - - Map Task Count: 8 - Merge Task Count: 1 - Master Query: - - Plan: - Node Type: "Aggregate" - Strategy: "Plain" - Partial Mode: "Simple" +- Plan: + Node Type: "Aggregate" + Strategy: "Plain" + Partial Mode: "Simple" + Parallel Aware: false + Plans: + - Node Type: "Custom Scan" + Parent Relationship: "Outer" + Custom Plan Provider: "Citus Task-Tracker" Parallel Aware: false - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Parallel Aware: false - Relation Name: "pg_merge_job_570035" - Alias: "pg_merge_job_570035" + Distributed Query: + Job: + Task Count: 1 + Tasks Shown: "None, not supported for re-partition queries" + Depended Jobs: + - Map Task Count: 1 + Merge Task Count: 1 + Depended Jobs: + - Map Task Count: 8 + Merge Task Count: 1 -- test parallel aggregates SET parallel_setup_cost=0; SET parallel_tuple_cost=0; @@ -672,38 +618,31 @@ Finalize Aggregate -> Parallel Seq Scan on lineitem_clone -- ensure distributed plans don't break EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; -Distributed Query into pg_merge_job_570036 - Executor: Task-Tracker - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570036 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290001 lineitem -- ensure EXPLAIN EXECUTE doesn't crash PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; -Distributed Query into pg_merge_job_570037 - Executor: Task-Tracker - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570037 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) SET citus.task_executor_type TO 'real-time'; PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; EXPLAIN EXECUTE router_executor_query; -Distributed Query into pg_merge_job_570038 - Executor: Router +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) Task Count: 1 Tasks Shown: All -> Task @@ -715,22 +654,25 @@ Distributed Query into pg_merge_job_570038 PREPARE real_time_executor_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query; -Distributed Query into pg_merge_job_570039 - Executor: Real-Time - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570039 +Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) -- EXPLAIN EXECUTE of parametrized prepared statements is broken, but -- at least make sure to fail without crashing PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1; EXPLAIN EXECUTE router_executor_query_param(5); -ERROR: could not create distributed plan -DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus. -HINT: Consider using PLPGSQL functions instead. +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Bitmap Heap Scan on lineitem_290000 lineitem (cost=4.30..13.44 rows=3 width=18) + Recheck Cond: (l_orderkey = 5) + -> Bitmap Index Scan on lineitem_pkey_290000 (cost=0.00..4.30 rows=3 width=0) + Index Cond: (l_orderkey = 5) diff --git a/src/test/regress/expected/multi_explain_0.out b/src/test/regress/expected/multi_explain_0.out index 7425b80e3..af839b514 100644 --- a/src/test/regress/expected/multi_explain_0.out +++ b/src/test/regress/expected/multi_explain_0.out @@ -39,81 +39,74 @@ $BODY$ LANGUAGE plpgsql; EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_570000 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Sort - Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_570000_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570000_0 - -> HashAggregate - Group Key: intermediate_column_570000_0 - -> Seq Scan on pg_merge_job_570000 +Sort + Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_290001 lineitem -- Test JSON format EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; [ { - "Executor": "Real-Time", - "Job": { - "Task Count": 8, - "Tasks Shown": "One of 8", - "Tasks": [ + "Plan": { + "Node Type": "Sort", + "Sort Key": ["COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"], + "Plans": [ { - "Node": "host=localhost port=57637 dbname=regression", - "Remote Plan": [ - [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Group Key": ["l_quantity"], - "Plans": [ + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Parent Relationship": "Outer", + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Real-Time", + "Distributed Query": { + "Job": { + "Task Count": 8, + "Tasks Shown": "One of 8", + "Tasks": [ { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Relation Name": "lineitem_290001", - "Alias": "lineitem" + "Node": "host=localhost port=57637 dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Seq Scan", + "Parent Relationship": "Outer", + "Relation Name": "lineitem_290001", + "Alias": "lineitem" + } + ] + } + } + ] + + ] } ] } } - ] - - ] - } - ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Sort", - "Sort Key": ["COALESCE((sum((COALESCE((sum(intermediate_column_570001_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_570001_0"], - "Plans": [ - { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Parent Relationship": "Outer", - "Group Key": ["intermediate_column_570001_0"], - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Relation Name": "pg_merge_job_570001", - "Alias": "pg_merge_job_570001" - } - ] } ] } - } - ] + ] + } } ] -- Validate JSON format @@ -126,68 +119,64 @@ EXPLAIN (COSTS FALSE, FORMAT XML) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; - - Real-Time - - 8 - One of 8 - - - host=localhost port=57637 dbname=regression - - - - - Aggregate - Hashed - - l_quantity - - - - Seq Scan - Outer - lineitem_290001 - lineitem - - - - - - - - - - - + + + Sort + + COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint) + l_quantity + + - Sort - - COALESCE((sum((COALESCE((sum(intermediate_column_570003_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) - intermediate_column_570003_0 - + Aggregate + Hashed + Outer + + l_quantity + - Aggregate - Hashed + Custom Scan Outer - - intermediate_column_570003_0 - - - - Seq Scan - Outer - pg_merge_job_570003 - pg_merge_job_570003 - - + Citus Real-Time + + + 8 + One of 8 + + + host=localhost port=57637 dbname=regression + + + + + Aggregate + Hashed + + l_quantity + + + + Seq Scan + Outer + lineitem_290001 + lineitem + + + + + + + + + + - - - + + + -- Validate XML format SELECT true AS valid FROM explain_xml($$ @@ -198,108 +187,96 @@ t EXPLAIN (COSTS FALSE, FORMAT YAML) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -- Executor: "Real-Time" - Job: - Task Count: 8 - Tasks Shown: "One of 8" - Tasks: - - Node: "host=localhost port=57637 dbname=regression" - Remote Plan: - - Plan: - Node Type: "Aggregate" - Strategy: "Hashed" - Group Key: - - "l_quantity" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Relation Name: "lineitem_290001" - Alias: "lineitem" - - Master Query: - - Plan: - Node Type: "Sort" - Sort Key: - - "COALESCE((sum((COALESCE((sum(intermediate_column_570005_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)" - - "intermediate_column_570005_0" +- Plan: + Node Type: "Sort" + Sort Key: + - "COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)" + - "l_quantity" + Plans: + - Node Type: "Aggregate" + Strategy: "Hashed" + Parent Relationship: "Outer" + Group Key: + - "l_quantity" Plans: - - Node Type: "Aggregate" - Strategy: "Hashed" + - Node Type: "Custom Scan" Parent Relationship: "Outer" - Group Key: - - "intermediate_column_570005_0" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Relation Name: "pg_merge_job_570005" - Alias: "pg_merge_job_570005" + Custom Plan Provider: "Citus Real-Time" + Distributed Query: + Job: + Task Count: 8 + Tasks Shown: "One of 8" + Tasks: + - Node: "host=localhost port=57637 dbname=regression" + Remote Plan: + - Plan: + Node Type: "Aggregate" + Strategy: "Hashed" + Group Key: + - "l_quantity" + Plans: + - Node Type: "Seq Scan" + Parent Relationship: "Outer" + Relation Name: "lineitem_290001" + Alias: "lineitem" + -- Test Text format EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_570006 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Sort - Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_570006_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570006_0 - -> HashAggregate - Group Key: intermediate_column_570006_0 - -> Seq Scan on pg_merge_job_570006 +Sort + Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_290001 lineitem -- Test verbose EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem; -Distributed Query into pg_merge_job_570007 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - Output: sum(l_quantity), sum(l_quantity), count(l_quantity) - -> Seq Scan on public.lineitem_290001 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> Aggregate - Output: (sum(intermediate_column_570007_0) / (sum(intermediate_column_570007_1) / sum(intermediate_column_570007_2))) - -> Seq Scan on pg_temp_2.pg_merge_job_570007 - Output: intermediate_column_570007_0, intermediate_column_570007_1, intermediate_column_570007_2 +Aggregate + Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2"))) + -> Custom Scan (Citus Real-Time) + Output: "?column?", "?column?_1", "?column?_2" + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + Output: sum(l_quantity), sum(l_quantity), count(l_quantity) + -> Seq Scan on public.lineitem_290001 lineitem + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test join EXPLAIN (COSTS FALSE) SELECT * FROM lineitem JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0 ORDER BY l_quantity LIMIT 10; -Distributed Query into pg_merge_job_570008 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Limit - -> Sort - Sort Key: lineitem.l_quantity - -> Hash Join - Hash Cond: (lineitem.l_orderkey = orders.o_orderkey) - -> Seq Scan on lineitem_290001 lineitem - Filter: (l_quantity < 5.0) - -> Hash - -> Seq Scan on orders_290008 orders -Master Query - -> Limit - -> Sort - Sort Key: intermediate_column_570008_4 - -> Seq Scan on pg_merge_job_570008 +Limit + -> Sort + Sort Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: lineitem.l_quantity + -> Hash Join + Hash Cond: (lineitem.l_orderkey = orders.o_orderkey) + -> Seq Scan on lineitem_290001 lineitem + Filter: (l_quantity < 5.0) + -> Hash + -> Seq Scan on orders_290008 orders -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem VALUES(1,0); -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -311,8 +288,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -327,8 +303,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -342,8 +317,7 @@ Distributed Query -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; -Distributed Query into pg_merge_job_570009 - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -362,89 +336,77 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem; -Distributed Query into pg_merge_job_570012 - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task Node: host=localhost port=57637 dbname=regression -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Seq Scan on pg_merge_job_570012 -- Test having EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem HAVING sum(l_quantity) > 100; -Distributed Query into pg_merge_job_570013 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity) - -> Seq Scan on public.lineitem_290001 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> Aggregate - Output: (sum(intermediate_column_570013_0) / (sum(intermediate_column_570013_1) / sum(intermediate_column_570013_2))) - Filter: (sum(pg_merge_job_570013.intermediate_column_570013_3) > '100'::numeric) - -> Seq Scan on pg_temp_2.pg_merge_job_570013 - Output: intermediate_column_570013_0, intermediate_column_570013_1, intermediate_column_570013_2, intermediate_column_570013_3 +Aggregate + Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2"))) + Filter: (sum(remote_scan.worker_column_4) > '100'::numeric) + -> Custom Scan (Citus Real-Time) + Output: "?column?", "?column?_1", "?column?_2", worker_column_4 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity) + -> Seq Scan on public.lineitem_290001 lineitem + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test having without aggregate EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT l_quantity FROM lineitem GROUP BY l_quantity HAVING l_quantity > (100 * random()); -Distributed Query into pg_merge_job_570014 - Executor: Real-Time - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Output: l_quantity, l_quantity - Group Key: lineitem.l_quantity - -> Seq Scan on public.lineitem_290001 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> HashAggregate - Output: intermediate_column_570014_0 - Group Key: pg_merge_job_570014.intermediate_column_570014_0 - Filter: ((pg_merge_job_570014.intermediate_column_570014_1)::double precision > ('100'::double precision * random())) - -> Seq Scan on pg_temp_2.pg_merge_job_570014 - Output: intermediate_column_570014_0, intermediate_column_570014_1 +HashAggregate + Output: l_quantity + Group Key: remote_scan.l_quantity + Filter: ((remote_scan.worker_column_2)::double precision > ('100'::double precision * random())) + -> Custom Scan (Citus Real-Time) + Output: l_quantity, worker_column_2 + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Output: l_quantity, l_quantity + Group Key: lineitem.l_quantity + -> Seq Scan on public.lineitem_290001 lineitem + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test all tasks output SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_570015 - Executor: Real-Time - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290004 lineitem - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290007 lineitem - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290006 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570015 +Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290004 lineitem + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290007 lineitem + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290006 lineitem + Filter: (l_orderkey > 9030) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$); t @@ -456,18 +418,15 @@ SET citus.task_executor_type TO 'task-tracker'; SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_570018 - Executor: Task-Tracker - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570018 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -476,19 +435,16 @@ EXPLAIN (COSTS FALSE) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -Distributed Query into pg_merge_job_570021 - Executor: Task-Tracker - Task Count: 1 - Tasks Shown: None, not supported for re-partition queries - -> MapMergeJob - Map Task Count: 1 - Merge Task Count: 1 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 1 + Tasks Shown: None, not supported for re-partition queries -> MapMergeJob - Map Task Count: 8 + Map Task Count: 1 Merge Task Count: 1 -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570021 + -> MapMergeJob + Map Task Count: 8 + Merge Task Count: 1 EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT count(*) FROM lineitem, orders, customer, supplier_single_shard @@ -497,39 +453,35 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) AND l_suppkey = s_suppkey; [ { - "Executor": "Task-Tracker", - "Job": { - "Task Count": 1, - "Tasks Shown": "None, not supported for re-partition queries", - "Depended Jobs": [ + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Plain", + "Plans": [ { - "Map Task Count": 1, - "Merge Task Count": 1, - "Depended Jobs": [ - { - "Map Task Count": 8, - "Merge Task Count": 1 + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Task-Tracker", + "Distributed Query": { + "Job": { + "Task Count": 1, + "Tasks Shown": "None, not supported for re-partition queries", + "Depended Jobs": [ + { + "Map Task Count": 1, + "Merge Task Count": 1, + "Depended Jobs": [ + { + "Map Task Count": 8, + "Merge Task Count": 1 + } + ] + } + ] } - ] + } } ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Plain", - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Relation Name": "pg_merge_job_570024", - "Alias": "pg_merge_job_570024" - } - ] - } - } - ] + } } ] SELECT true AS valid FROM explain_json($$ @@ -546,41 +498,37 @@ EXPLAIN (COSTS FALSE, FORMAT XML) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; - - Task-Tracker - - 1 - None, not supported for re-partition queries - - - 1 - 1 - - - 8 - 1 - - - - - - - + + + Aggregate + Plain + - Aggregate - Plain - - - Seq Scan - Outer - pg_merge_job_570030 - pg_merge_job_570030 - - + Custom Scan + Outer + Citus Task-Tracker + + + 1 + None, not supported for re-partition queries + + + 1 + 1 + + + 8 + 1 + + + + + + - - - + + + SELECT true AS valid FROM explain_xml($$ SELECT count(*) @@ -608,25 +556,23 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -- Executor: "Task-Tracker" - Job: - Task Count: 1 - Tasks Shown: "None, not supported for re-partition queries" - Depended Jobs: - - Map Task Count: 1 - Merge Task Count: 1 - Depended Jobs: - - Map Task Count: 8 - Merge Task Count: 1 - Master Query: - - Plan: - Node Type: "Aggregate" - Strategy: "Plain" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Relation Name: "pg_merge_job_570035" - Alias: "pg_merge_job_570035" +- Plan: + Node Type: "Aggregate" + Strategy: "Plain" + Plans: + - Node Type: "Custom Scan" + Parent Relationship: "Outer" + Custom Plan Provider: "Citus Task-Tracker" + Distributed Query: + Job: + Task Count: 1 + Tasks Shown: "None, not supported for re-partition queries" + Depended Jobs: + - Map Task Count: 1 + Merge Task Count: 1 + Depended Jobs: + - Map Task Count: 8 + Merge Task Count: 1 -- test parallel aggregates SET parallel_setup_cost=0; ERROR: unrecognized configuration parameter "parallel_setup_cost" @@ -643,38 +589,31 @@ Aggregate -> Seq Scan on lineitem_clone -- ensure distributed plans don't break EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; -Distributed Query into pg_merge_job_570036 - Executor: Task-Tracker - Task Count: 8 - Tasks Shown: One of 8 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290001 lineitem -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570036 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290001 lineitem -- ensure EXPLAIN EXECUTE doesn't crash PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; -Distributed Query into pg_merge_job_570037 - Executor: Task-Tracker - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570037 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) SET citus.task_executor_type TO 'real-time'; PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; EXPLAIN EXECUTE router_executor_query; -Distributed Query into pg_merge_job_570038 - Executor: Router +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) Task Count: 1 Tasks Shown: All -> Task @@ -686,22 +625,25 @@ Distributed Query into pg_merge_job_570038 PREPARE real_time_executor_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query; -Distributed Query into pg_merge_job_570039 - Executor: Real-Time - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_290005 lineitem - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_570039 +Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_290005 lineitem + Filter: (l_orderkey > 9030) -- EXPLAIN EXECUTE of parametrized prepared statements is broken, but -- at least make sure to fail without crashing PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1; EXPLAIN EXECUTE router_executor_query_param(5); -ERROR: could not create distributed plan -DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus. -HINT: Consider using PLPGSQL functions instead. +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Bitmap Heap Scan on lineitem_290000 lineitem (cost=4.30..13.44 rows=3 width=18) + Recheck Cond: (l_orderkey = 5) + -> Bitmap Index Scan on lineitem_pkey_290000 (cost=0.00..4.30 rows=3 width=0) + Index Cond: (l_orderkey = 5) diff --git a/src/test/regress/expected/multi_join_order_additional.out b/src/test/regress/expected/multi_join_order_additional.out index 7cebe1ec9..da8ddd2f5 100644 --- a/src/test/regress/expected/multi_join_order_additional.out +++ b/src/test/regress/expected/multi_join_order_additional.out @@ -6,6 +6,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 650000; -- Set configuration to print table join order and pruned shards SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; +SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise SET client_min_messages TO DEBUG2; -- Create new table definitions for use in testing in distributed planning and -- execution functionality. Also create indexes to boost performance. @@ -139,10 +140,11 @@ DEBUG: join prunable for intervals [13473,14947] and [2951,4455] DEBUG: join prunable for intervals [13473,14947] and [4480,5986] DEBUG: join prunable for intervals [13473,14947] and [8997,10560] DEBUG: join prunable for intervals [13473,14947] and [10560,12036] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------- + Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(2 rows) -- Update configuration to treat lineitem and orders tables as large SET citus.large_table_shard_count TO 2; @@ -154,10 +156,12 @@ EXPLAIN SELECT count(*) FROM lineitem, orders WHERE (l_orderkey = o_orderkey AND l_quantity > 5) OR (l_orderkey = o_orderkey AND l_quantity < 10); LOG: join order: [ "lineitem" ][ local partition join "orders" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT l_quantity FROM lineitem, orders WHERE (l_orderkey = o_orderkey OR l_quantity > 5); @@ -173,28 +177,34 @@ BEGIN; EXPLAIN SELECT count(*) FROM orders, lineitem_hash WHERE o_orderkey = l_orderkey; LOG: join order: [ "orders" ][ single partition join "lineitem_hash" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Verify we handle local joins between two hash-partitioned tables. EXPLAIN SELECT count(*) FROM orders_hash, lineitem_hash WHERE o_orderkey = l_orderkey; LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Validate that we can handle broadcast joins with hash-partitioned tables. EXPLAIN SELECT count(*) FROM customer_hash, nation WHERE c_nationkey = n_nationkey; LOG: join order: [ "customer_hash" ][ broadcast join "nation" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Update the large table shard count for all the following tests. SET citus.large_table_shard_count TO 1; @@ -203,30 +213,36 @@ SET citus.large_table_shard_count TO 1; EXPLAIN SELECT count(*) FROM orders, lineitem, customer WHERE o_custkey = l_partkey AND o_custkey = c_nationkey; LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Validate that we don't chose a single-partition join method with a -- hash-partitioned base table EXPLAIN SELECT count(*) FROM orders, customer_hash WHERE c_custkey = o_custkey; LOG: join order: [ "orders" ][ dual partition join "customer_hash" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Validate that we can re-partition a hash partitioned table to join with a -- range partitioned one. EXPLAIN SELECT count(*) FROM orders_hash, customer WHERE c_custkey = o_custkey; LOG: join order: [ "orders_hash" ][ single partition join "customer" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) COMMIT; -- Reset client logging level to its previous value diff --git a/src/test/regress/expected/multi_join_order_tpch_large.out b/src/test/regress/expected/multi_join_order_tpch_large.out index 327687eac..9489b567e 100644 --- a/src/test/regress/expected/multi_join_order_tpch_large.out +++ b/src/test/regress/expected/multi_join_order_tpch_large.out @@ -6,6 +6,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 660000; -- Enable configuration to print table join order SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; +SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise SET client_min_messages TO LOG; -- Change configuration to treat lineitem, orders, customer, and part tables as -- large. The following queries are basically the same as the ones in tpch_small @@ -23,10 +24,12 @@ WHERE and l_discount between 0.06 - 0.01 and 0.06 + 0.01 and l_quantity < 24; LOG: join order: [ "lineitem" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Query #3 from the TPC-H decision support benchmark EXPLAIN SELECT @@ -52,10 +55,15 @@ ORDER BY revenue DESC, o_orderdate; LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------------- + Sort (cost=0.00..0.00 rows=0 width=0) + Sort Key: sum((sum(revenue))) DESC, o_orderdate + -> HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: l_orderkey, o_orderdate, o_shippriority + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(6 rows) -- Query #10 from the TPC-H decision support benchmark EXPLAIN SELECT @@ -90,10 +98,15 @@ GROUP BY ORDER BY revenue DESC; LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ][ broadcast join "nation" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +---------------------------------------------------------------------------------------- + Sort (cost=0.00..0.00 rows=0 width=0) + Sort Key: sum((sum(revenue))) DESC + -> HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(6 rows) -- Query #19 from the TPC-H decision support benchmark (modified) EXPLAIN SELECT @@ -126,10 +139,12 @@ WHERE AND l_shipinstruct = 'DELIVER IN PERSON' ); LOG: join order: [ "lineitem" ][ single partition join "part" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Query to test multiple re-partition jobs in a single query EXPLAIN SELECT @@ -143,10 +158,13 @@ WHERE GROUP BY l_partkey; LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single partition join "part" ][ single partition join "customer" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +-------------------------------------------------------------------------- + HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: l_partkey + -> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(4 rows) -- Reset client logging level to its previous value SET client_min_messages TO NOTICE; diff --git a/src/test/regress/expected/multi_join_order_tpch_small.out b/src/test/regress/expected/multi_join_order_tpch_small.out index 1c506136c..c0466b1e3 100644 --- a/src/test/regress/expected/multi_join_order_tpch_small.out +++ b/src/test/regress/expected/multi_join_order_tpch_small.out @@ -18,10 +18,12 @@ WHERE and l_discount between 0.06 - 0.01 and 0.06 + 0.01 and l_quantity < 24; LOG: join order: [ "lineitem" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Query #3 from the TPC-H decision support benchmark EXPLAIN SELECT @@ -47,10 +49,15 @@ ORDER BY revenue DESC, o_orderdate; LOG: join order: [ "orders" ][ broadcast join "customer" ][ local partition join "lineitem" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------------- + Sort (cost=0.00..0.00 rows=0 width=0) + Sort Key: sum((sum(revenue))) DESC, o_orderdate + -> HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: l_orderkey, o_orderdate, o_shippriority + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(6 rows) -- Query #10 from the TPC-H decision support benchmark EXPLAIN SELECT @@ -85,10 +92,15 @@ GROUP BY ORDER BY revenue DESC; LOG: join order: [ "orders" ][ broadcast join "customer" ][ broadcast join "nation" ][ local partition join "lineitem" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +---------------------------------------------------------------------------------------- + Sort (cost=0.00..0.00 rows=0 width=0) + Sort Key: sum((sum(revenue))) DESC + -> HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(6 rows) -- Query #19 from the TPC-H decision support benchmark (modified) EXPLAIN SELECT @@ -121,10 +133,12 @@ WHERE AND l_shipinstruct = 'DELIVER IN PERSON' ); LOG: join order: [ "lineitem" ][ broadcast join "part" ] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Reset client logging level to its previous value SET client_min_messages TO NOTICE; diff --git a/src/test/regress/expected/multi_join_pruning.out b/src/test/regress/expected/multi_join_pruning.out index 9e2fd914e..e04799723 100644 --- a/src/test/regress/expected/multi_join_pruning.out +++ b/src/test/regress/expected/multi_join_pruning.out @@ -100,20 +100,24 @@ EXPLAIN SELECT count(*) WHERE table1.array_column = table2.array_column; DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM composite_partitioned_table table1, composite_partitioned_table table2 WHERE table1.composite_column = table2.composite_column; DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)] DEBUG: join prunable for intervals [(c,5,d),(d,6,e)] and [(a,3,b),(b,4,c)] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Test that large table joins on partition varchar columns work EXPLAIN SELECT count(*) @@ -121,8 +125,10 @@ EXPLAIN SELECT count(*) WHERE table1.varchar_column = table2.varchar_column; DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U2AMO4ZGX,BZZXSP27F21T6] DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6] - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) diff --git a/src/test/regress/expected/multi_mx_explain.out b/src/test/regress/expected/multi_mx_explain.out index 7e8be01a0..02dfbee99 100644 --- a/src/test/regress/expected/multi_mx_explain.out +++ b/src/test/regress/expected/multi_mx_explain.out @@ -61,88 +61,81 @@ $BODY$ LANGUAGE plpgsql; EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_68720796736 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_mx_1220052 lineitem_mx -Master Query - -> Sort - Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_68720796736_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_68720796736_0 - -> HashAggregate - Group Key: intermediate_column_68720796736_0 - -> Seq Scan on pg_merge_job_68720796736 +Sort + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_mx_1220052 lineitem_mx -- Test JSON format EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; [ { - "Executor": "Real-Time", - "Job": { - "Task Count": 16, - "Tasks Shown": "One of 16", - "Tasks": [ + "Plan": { + "Node Type": "Sort", + "Parallel Aware": false, + "Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"], + "Plans": [ { - "Node": "host=localhost port=57637 dbname=regression", - "Remote Plan": [ - [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Partial Mode": "Simple", - "Parallel Aware": false, - "Group Key": ["l_quantity"], - "Plans": [ + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Partial Mode": "Simple", + "Parent Relationship": "Outer", + "Parallel Aware": false, + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Real-Time", + "Parallel Aware": false, + "Distributed Query": { + "Job": { + "Task Count": 16, + "Tasks Shown": "One of 16", + "Tasks": [ { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Relation Name": "lineitem_mx_1220052", - "Alias": "lineitem_mx" + "Node": "host=localhost port=57637 dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Partial Mode": "Simple", + "Parallel Aware": false, + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Seq Scan", + "Parent Relationship": "Outer", + "Parallel Aware": false, + "Relation Name": "lineitem_mx_1220052", + "Alias": "lineitem_mx" + } + ] + } + } + ] + + ] } ] } } - ] - - ] - } - ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Sort", - "Parallel Aware": false, - "Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_68720796737_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_68720796737_0"], - "Plans": [ - { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Partial Mode": "Simple", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Group Key": ["intermediate_column_68720796737_0"], - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Relation Name": "pg_merge_job_68720796737", - "Alias": "pg_merge_job_68720796737" - } - ] } ] } - } - ] + ] + } } ] -- Validate JSON format @@ -156,75 +149,71 @@ EXPLAIN (COSTS FALSE, FORMAT XML) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; - - Real-Time - - 16 - One of 16 - - - host=localhost port=57637 dbname=regression - - - - - Aggregate - Hashed - Simple - false - - l_quantity - - - - Seq Scan - Outer - false - lineitem_mx_1220052 - lineitem_mx - - - - - - - - - - - + + + Sort + false + + COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint) + l_quantity + + - Sort + Aggregate + Hashed + Simple + Outer false - - COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_60130862144_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) - intermediate_column_60130862144_0 - + + l_quantity + - Aggregate - Hashed - Simple + Custom Scan Outer + Citus Real-Time false - - intermediate_column_60130862144_0 - - - - Seq Scan - Outer - false - pg_merge_job_60130862144 - pg_merge_job_60130862144 - - + + + 16 + One of 16 + + + host=localhost port=57637 dbname=regression + + + + + Aggregate + Hashed + Simple + false + + l_quantity + + + + Seq Scan + Outer + false + lineitem_mx_1220052 + lineitem_mx + + + + + + + + + + - - - + + + -- Validate XML format SELECT true AS valid FROM explain_xml($$ @@ -235,116 +224,104 @@ t EXPLAIN (COSTS FALSE, FORMAT YAML) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -- Executor: "Real-Time" - Job: - Task Count: 16 - Tasks Shown: "One of 16" - Tasks: - - Node: "host=localhost port=57637 dbname=regression" - Remote Plan: - - Plan: - Node Type: "Aggregate" - Strategy: "Hashed" - Partial Mode: "Simple" - Parallel Aware: false - Group Key: - - "l_quantity" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Parallel Aware: false - Relation Name: "lineitem_mx_1220052" - Alias: "lineitem_mx" - - Master Query: - - Plan: - Node Type: "Sort" +- Plan: + Node Type: "Sort" + Parallel Aware: false + Sort Key: + - "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)" + - "l_quantity" + Plans: + - Node Type: "Aggregate" + Strategy: "Hashed" + Partial Mode: "Simple" + Parent Relationship: "Outer" Parallel Aware: false - Sort Key: - - "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_60130862146_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)" - - "intermediate_column_60130862146_0" + Group Key: + - "l_quantity" Plans: - - Node Type: "Aggregate" - Strategy: "Hashed" - Partial Mode: "Simple" + - Node Type: "Custom Scan" Parent Relationship: "Outer" + Custom Plan Provider: "Citus Real-Time" Parallel Aware: false - Group Key: - - "intermediate_column_60130862146_0" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Parallel Aware: false - Relation Name: "pg_merge_job_60130862146" - Alias: "pg_merge_job_60130862146" + Distributed Query: + Job: + Task Count: 16 + Tasks Shown: "One of 16" + Tasks: + - Node: "host=localhost port=57637 dbname=regression" + Remote Plan: + - Plan: + Node Type: "Aggregate" + Strategy: "Hashed" + Partial Mode: "Simple" + Parallel Aware: false + Group Key: + - "l_quantity" + Plans: + - Node Type: "Seq Scan" + Parent Relationship: "Outer" + Parallel Aware: false + Relation Name: "lineitem_mx_1220052" + Alias: "lineitem_mx" + -- Test Text format EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_60130862147 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_mx_1220052 lineitem_mx -Master Query - -> Sort - Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_60130862147_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_60130862147_0 - -> HashAggregate - Group Key: intermediate_column_60130862147_0 - -> Seq Scan on pg_merge_job_60130862147 +Sort + Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_mx_1220052 lineitem_mx \c - - - :worker_2_port -- Test verbose EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx; -Distributed Query into pg_merge_job_68720796739 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - Output: sum(l_quantity), sum(l_quantity), count(l_quantity) - -> Seq Scan on public.lineitem_mx_1220052 lineitem_mx - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> Aggregate - Output: (sum(intermediate_column_68720796739_0) / (sum(intermediate_column_68720796739_1) / pg_catalog.sum(intermediate_column_68720796739_2))) - -> Seq Scan on pg_temp_2.pg_merge_job_68720796739 - Output: intermediate_column_68720796739_0, intermediate_column_68720796739_1, intermediate_column_68720796739_2 +Aggregate + Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2"))) + -> Custom Scan (Citus Real-Time) + Output: "?column?", "?column?_1", "?column?_2" + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + Output: sum(l_quantity), sum(l_quantity), count(l_quantity) + -> Seq Scan on public.lineitem_mx_1220052 lineitem_mx + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test join EXPLAIN (COSTS FALSE) SELECT * FROM lineitem_mx JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0 ORDER BY l_quantity LIMIT 10; -Distributed Query into pg_merge_job_68720796740 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Limit - -> Sort - Sort Key: lineitem_mx.l_quantity - -> Hash Join - Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey) - -> Seq Scan on lineitem_mx_1220052 lineitem_mx - Filter: (l_quantity < 5.0) - -> Hash - -> Seq Scan on orders_mx_1220068 orders_mx -Master Query - -> Limit - -> Sort - Sort Key: intermediate_column_68720796740_4 - -> Seq Scan on pg_merge_job_68720796740 +Limit + -> Sort + Sort Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: lineitem_mx.l_quantity + -> Hash Join + Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey) + -> Seq Scan on lineitem_mx_1220052 lineitem_mx + Filter: (l_quantity < 5.0) + -> Hash + -> Seq Scan on orders_mx_1220068 orders_mx -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem_mx VALUES(1,0); -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -356,8 +333,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem_mx SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -370,8 +346,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem_mx WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -383,8 +358,7 @@ Distributed Query -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -403,106 +377,100 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem_mx; -Distributed Query into pg_merge_job_68720796741 - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task Node: host=localhost port=57637 dbname=regression -> Seq Scan on lineitem_mx_1220052 lineitem_mx -Master Query - -> Seq Scan on pg_merge_job_68720796741 -- Test all tasks output SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_68720796742 - Executor: Real-Time - Task Count: 16 - Tasks Shown: All - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220052 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220053 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220054 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220055 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220056 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220057 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220058 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220059 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220060 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220061 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220062 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220063 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220064 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220065 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220066 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220067 lineitem_mx - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_68720796742 +Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220052 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220053 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220054 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220055 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220056 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220057 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220058 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220059 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220060 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220061 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220062 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220063 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220064 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220065 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220066 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220067 lineitem_mx + Filter: (l_orderkey > 9030) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); t @@ -514,18 +482,15 @@ SET citus.task_executor_type TO 'task-tracker'; SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_68720796745 - Executor: Task-Tracker - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220052 lineitem_mx - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_68720796745 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220052 lineitem_mx + Filter: (l_orderkey > 9030) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -534,25 +499,22 @@ EXPLAIN (COSTS FALSE) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -Distributed Query into pg_merge_job_68720796750 - Executor: Task-Tracker - Task Count: 4 - Tasks Shown: None, not supported for re-partition queries - -> MapMergeJob - Map Task Count: 4 - Merge Task Count: 4 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: None, not supported for re-partition queries -> MapMergeJob - Map Task Count: 16 + Map Task Count: 4 Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 16 + Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 1 + Merge Task Count: 4 -> MapMergeJob Map Task Count: 1 Merge Task Count: 4 - -> MapMergeJob - Map Task Count: 1 - Merge Task Count: 4 -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_68720796750 EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT count(*) FROM lineitem_mx, orders_mx, customer_mx, supplier_mx @@ -561,50 +523,46 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) AND l_suppkey = s_suppkey; [ { - "Executor": "Task-Tracker", - "Job": { - "Task Count": 4, - "Tasks Shown": "None, not supported for re-partition queries", - "Depended Jobs": [ + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Plain", + "Partial Mode": "Simple", + "Parallel Aware": false, + "Plans": [ { - "Map Task Count": 4, - "Merge Task Count": 4, - "Depended Jobs": [ - { - "Map Task Count": 16, - "Merge Task Count": 4 - }, - { - "Map Task Count": 1, - "Merge Task Count": 4 + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Task-Tracker", + "Parallel Aware": false, + "Distributed Query": { + "Job": { + "Task Count": 4, + "Tasks Shown": "None, not supported for re-partition queries", + "Depended Jobs": [ + { + "Map Task Count": 4, + "Merge Task Count": 4, + "Depended Jobs": [ + { + "Map Task Count": 16, + "Merge Task Count": 4 + }, + { + "Map Task Count": 1, + "Merge Task Count": 4 + } + ] + }, + { + "Map Task Count": 1, + "Merge Task Count": 4 + } + ] } - ] - }, - { - "Map Task Count": 1, - "Merge Task Count": 4 + } } ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Plain", - "Partial Mode": "Simple", - "Parallel Aware": false, - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Parallel Aware": false, - "Relation Name": "pg_merge_job_68720796755", - "Alias": "pg_merge_job_68720796755" - } - ] - } - } - ] + } } ] SELECT true AS valid FROM explain_json($$ @@ -621,52 +579,48 @@ EXPLAIN (COSTS FALSE, FORMAT XML) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; - - Task-Tracker - - 4 - None, not supported for re-partition queries - - - 4 - 4 - - - 16 - 4 - - - 1 - 4 - - - - - 1 - 4 - - - - - + + + Aggregate + Plain + Simple + false + - Aggregate - Plain - Simple + Custom Scan + Outer + Citus Task-Tracker false - - - Seq Scan - Outer - false - pg_merge_job_68720796765 - pg_merge_job_68720796765 - - + + + 4 + None, not supported for re-partition queries + + + 4 + 4 + + + 16 + 4 + + + 1 + 4 + + + + + 1 + 4 + + + + - - - + + + SELECT true AS valid FROM explain_xml($$ SELECT count(*) @@ -681,29 +635,27 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -- Executor: "Task-Tracker" - Job: - Task Count: 4 - Tasks Shown: "None, not supported for re-partition queries" - Depended Jobs: - - Map Task Count: 4 - Merge Task Count: 4 - Depended Jobs: - - Map Task Count: 16 - Merge Task Count: 4 - - Map Task Count: 1 - Merge Task Count: 4 - - Map Task Count: 1 - Merge Task Count: 4 - Master Query: - - Plan: - Node Type: "Aggregate" - Strategy: "Plain" - Partial Mode: "Simple" +- Plan: + Node Type: "Aggregate" + Strategy: "Plain" + Partial Mode: "Simple" + Parallel Aware: false + Plans: + - Node Type: "Custom Scan" + Parent Relationship: "Outer" + Custom Plan Provider: "Citus Task-Tracker" Parallel Aware: false - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Parallel Aware: false - Relation Name: "pg_merge_job_68720796775" - Alias: "pg_merge_job_68720796775" + Distributed Query: + Job: + Task Count: 4 + Tasks Shown: "None, not supported for re-partition queries" + Depended Jobs: + - Map Task Count: 4 + Merge Task Count: 4 + Depended Jobs: + - Map Task Count: 16 + Merge Task Count: 4 + - Map Task Count: 1 + Merge Task Count: 4 + - Map Task Count: 1 + Merge Task Count: 4 diff --git a/src/test/regress/expected/multi_mx_explain_0.out b/src/test/regress/expected/multi_mx_explain_0.out index 489e4c5ca..0a74d3001 100644 --- a/src/test/regress/expected/multi_mx_explain_0.out +++ b/src/test/regress/expected/multi_mx_explain_0.out @@ -61,81 +61,74 @@ $BODY$ LANGUAGE plpgsql; EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_68720796736 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_mx_1220052 lineitem_mx -Master Query - -> Sort - Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_68720796736_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_68720796736_0 - -> HashAggregate - Group Key: intermediate_column_68720796736_0 - -> Seq Scan on pg_merge_job_68720796736 +Sort + Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_mx_1220052 lineitem_mx -- Test JSON format EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; [ { - "Executor": "Real-Time", - "Job": { - "Task Count": 16, - "Tasks Shown": "One of 16", - "Tasks": [ + "Plan": { + "Node Type": "Sort", + "Sort Key": ["COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"], + "Plans": [ { - "Node": "host=localhost port=57637 dbname=regression", - "Remote Plan": [ - [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Group Key": ["l_quantity"], - "Plans": [ + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Parent Relationship": "Outer", + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Real-Time", + "Distributed Query": { + "Job": { + "Task Count": 16, + "Tasks Shown": "One of 16", + "Tasks": [ { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Relation Name": "lineitem_mx_1220052", - "Alias": "lineitem_mx" + "Node": "host=localhost port=57637 dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Hashed", + "Group Key": ["l_quantity"], + "Plans": [ + { + "Node Type": "Seq Scan", + "Parent Relationship": "Outer", + "Relation Name": "lineitem_mx_1220052", + "Alias": "lineitem_mx" + } + ] + } + } + ] + + ] } ] } } - ] - - ] - } - ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Sort", - "Sort Key": ["COALESCE((sum((COALESCE((sum(intermediate_column_68720796737_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_68720796737_0"], - "Plans": [ - { - "Node Type": "Aggregate", - "Strategy": "Hashed", - "Parent Relationship": "Outer", - "Group Key": ["intermediate_column_68720796737_0"], - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Relation Name": "pg_merge_job_68720796737", - "Alias": "pg_merge_job_68720796737" - } - ] } ] } - } - ] + ] + } } ] -- Validate JSON format @@ -149,68 +142,64 @@ EXPLAIN (COSTS FALSE, FORMAT XML) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; - - Real-Time - - 16 - One of 16 - - - host=localhost port=57637 dbname=regression - - - - - Aggregate - Hashed - - l_quantity - - - - Seq Scan - Outer - lineitem_mx_1220052 - lineitem_mx - - - - - - - - - - - + + + Sort + + COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint) + l_quantity + + - Sort - - COALESCE((sum((COALESCE((sum(intermediate_column_60130862144_1))::bigint, '0'::bigint))))::bigint, '0'::bigint) - intermediate_column_60130862144_0 - + Aggregate + Hashed + Outer + + l_quantity + - Aggregate - Hashed + Custom Scan Outer - - intermediate_column_60130862144_0 - - - - Seq Scan - Outer - pg_merge_job_60130862144 - pg_merge_job_60130862144 - - + Citus Real-Time + + + 16 + One of 16 + + + host=localhost port=57637 dbname=regression + + + + + Aggregate + Hashed + + l_quantity + + + + Seq Scan + Outer + lineitem_mx_1220052 + lineitem_mx + + + + + + + + + + - - - + + + -- Validate XML format SELECT true AS valid FROM explain_xml($$ @@ -221,109 +210,97 @@ t EXPLAIN (COSTS FALSE, FORMAT YAML) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -- Executor: "Real-Time" - Job: - Task Count: 16 - Tasks Shown: "One of 16" - Tasks: - - Node: "host=localhost port=57637 dbname=regression" - Remote Plan: - - Plan: - Node Type: "Aggregate" - Strategy: "Hashed" - Group Key: - - "l_quantity" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Relation Name: "lineitem_mx_1220052" - Alias: "lineitem_mx" - - Master Query: - - Plan: - Node Type: "Sort" - Sort Key: - - "COALESCE((sum((COALESCE((sum(intermediate_column_60130862146_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)" - - "intermediate_column_60130862146_0" +- Plan: + Node Type: "Sort" + Sort Key: + - "COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)" + - "l_quantity" + Plans: + - Node Type: "Aggregate" + Strategy: "Hashed" + Parent Relationship: "Outer" + Group Key: + - "l_quantity" Plans: - - Node Type: "Aggregate" - Strategy: "Hashed" + - Node Type: "Custom Scan" Parent Relationship: "Outer" - Group Key: - - "intermediate_column_60130862146_0" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Relation Name: "pg_merge_job_60130862146" - Alias: "pg_merge_job_60130862146" + Custom Plan Provider: "Citus Real-Time" + Distributed Query: + Job: + Task Count: 16 + Tasks Shown: "One of 16" + Tasks: + - Node: "host=localhost port=57637 dbname=regression" + Remote Plan: + - Plan: + Node Type: "Aggregate" + Strategy: "Hashed" + Group Key: + - "l_quantity" + Plans: + - Node Type: "Seq Scan" + Parent Relationship: "Outer" + Relation Name: "lineitem_mx_1220052" + Alias: "lineitem_mx" + -- Test Text format EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx GROUP BY l_quantity ORDER BY count_quantity, l_quantity; -Distributed Query into pg_merge_job_60130862147 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate - Group Key: l_quantity - -> Seq Scan on lineitem_mx_1220052 lineitem_mx -Master Query - -> Sort - Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_60130862147_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_60130862147_0 - -> HashAggregate - Group Key: intermediate_column_60130862147_0 - -> Seq Scan on pg_merge_job_60130862147 +Sort + Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity + -> HashAggregate + Group Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: l_quantity + -> Seq Scan on lineitem_mx_1220052 lineitem_mx \c - - - :worker_2_port -- Test verbose EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx; -Distributed Query into pg_merge_job_68720796739 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - Output: sum(l_quantity), sum(l_quantity), count(l_quantity) - -> Seq Scan on public.lineitem_mx_1220052 lineitem_mx - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -Master Query - -> Aggregate - Output: (sum(intermediate_column_68720796739_0) / (sum(intermediate_column_68720796739_1) / sum(intermediate_column_68720796739_2))) - -> Seq Scan on pg_temp_2.pg_merge_job_68720796739 - Output: intermediate_column_68720796739_0, intermediate_column_68720796739_1, intermediate_column_68720796739_2 +Aggregate + Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2"))) + -> Custom Scan (Citus Real-Time) + Output: "?column?", "?column?_1", "?column?_2" + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + Output: sum(l_quantity), sum(l_quantity), count(l_quantity) + -> Seq Scan on public.lineitem_mx_1220052 lineitem_mx + Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Test join EXPLAIN (COSTS FALSE) SELECT * FROM lineitem_mx JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0 ORDER BY l_quantity LIMIT 10; -Distributed Query into pg_merge_job_68720796740 - Executor: Real-Time - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Limit - -> Sort - Sort Key: lineitem_mx.l_quantity - -> Hash Join - Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey) - -> Seq Scan on lineitem_mx_1220052 lineitem_mx - Filter: (l_quantity < 5.0) - -> Hash - -> Seq Scan on orders_mx_1220068 orders_mx -Master Query - -> Limit - -> Sort - Sort Key: intermediate_column_68720796740_4 - -> Seq Scan on pg_merge_job_68720796740 +Limit + -> Sort + Sort Key: l_quantity + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: lineitem_mx.l_quantity + -> Hash Join + Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey) + -> Seq Scan on lineitem_mx_1220052 lineitem_mx + Filter: (l_quantity < 5.0) + -> Hash + -> Seq Scan on orders_mx_1220068 orders_mx -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem_mx VALUES(1,0); -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -335,8 +312,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem_mx SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -349,8 +325,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem_mx WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -362,8 +337,7 @@ Distributed Query -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; -Distributed Query - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -382,106 +356,100 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem_mx; -Distributed Query into pg_merge_job_68720796741 - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task Node: host=localhost port=57637 dbname=regression -> Seq Scan on lineitem_mx_1220052 lineitem_mx -Master Query - -> Seq Scan on pg_merge_job_68720796741 -- Test all tasks output SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_68720796742 - Executor: Real-Time - Task Count: 16 - Tasks Shown: All - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220052 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220053 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220054 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220055 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220056 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220057 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220058 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220059 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220060 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220061 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220062 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220063 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220064 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220065 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220066 lineitem_mx - Filter: (l_orderkey > 9030) - -> Task - Node: host=localhost port=57638 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220067 lineitem_mx - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_68720796742 +Aggregate + -> Custom Scan (Citus Real-Time) + Task Count: 16 + Tasks Shown: All + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220052 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220053 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220054 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220055 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220056 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220057 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220058 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220059 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220060 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220061 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220062 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220063 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220064 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220065 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220066 lineitem_mx + Filter: (l_orderkey > 9030) + -> Task + Node: host=localhost port=57638 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220067 lineitem_mx + Filter: (l_orderkey > 9030) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); t @@ -493,18 +461,15 @@ SET citus.task_executor_type TO 'task-tracker'; SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; -Distributed Query into pg_merge_job_68720796745 - Executor: Task-Tracker - Task Count: 16 - Tasks Shown: One of 16 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate - -> Seq Scan on lineitem_mx_1220052 lineitem_mx - Filter: (l_orderkey > 9030) -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_68720796745 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 16 + Tasks Shown: One of 16 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate + -> Seq Scan on lineitem_mx_1220052 lineitem_mx + Filter: (l_orderkey > 9030) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -513,25 +478,22 @@ EXPLAIN (COSTS FALSE) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -Distributed Query into pg_merge_job_68720796750 - Executor: Task-Tracker - Task Count: 4 - Tasks Shown: None, not supported for re-partition queries - -> MapMergeJob - Map Task Count: 4 - Merge Task Count: 4 +Aggregate + -> Custom Scan (Citus Task-Tracker) + Task Count: 4 + Tasks Shown: None, not supported for re-partition queries -> MapMergeJob - Map Task Count: 16 + Map Task Count: 4 Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 16 + Merge Task Count: 4 + -> MapMergeJob + Map Task Count: 1 + Merge Task Count: 4 -> MapMergeJob Map Task Count: 1 Merge Task Count: 4 - -> MapMergeJob - Map Task Count: 1 - Merge Task Count: 4 -Master Query - -> Aggregate - -> Seq Scan on pg_merge_job_68720796750 EXPLAIN (COSTS FALSE, FORMAT JSON) SELECT count(*) FROM lineitem_mx, orders_mx, customer_mx, supplier_mx @@ -540,47 +502,43 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) AND l_suppkey = s_suppkey; [ { - "Executor": "Task-Tracker", - "Job": { - "Task Count": 4, - "Tasks Shown": "None, not supported for re-partition queries", - "Depended Jobs": [ + "Plan": { + "Node Type": "Aggregate", + "Strategy": "Plain", + "Plans": [ { - "Map Task Count": 4, - "Merge Task Count": 4, - "Depended Jobs": [ - { - "Map Task Count": 16, - "Merge Task Count": 4 - }, - { - "Map Task Count": 1, - "Merge Task Count": 4 + "Node Type": "Custom Scan", + "Parent Relationship": "Outer", + "Custom Plan Provider": "Citus Task-Tracker", + "Distributed Query": { + "Job": { + "Task Count": 4, + "Tasks Shown": "None, not supported for re-partition queries", + "Depended Jobs": [ + { + "Map Task Count": 4, + "Merge Task Count": 4, + "Depended Jobs": [ + { + "Map Task Count": 16, + "Merge Task Count": 4 + }, + { + "Map Task Count": 1, + "Merge Task Count": 4 + } + ] + }, + { + "Map Task Count": 1, + "Merge Task Count": 4 + } + ] } - ] - }, - { - "Map Task Count": 1, - "Merge Task Count": 4 + } } ] - }, - "Master Query": [ - { - "Plan": { - "Node Type": "Aggregate", - "Strategy": "Plain", - "Plans": [ - { - "Node Type": "Seq Scan", - "Parent Relationship": "Outer", - "Relation Name": "pg_merge_job_68720796755", - "Alias": "pg_merge_job_68720796755" - } - ] - } - } - ] + } } ] SELECT true AS valid FROM explain_json($$ @@ -597,49 +555,45 @@ EXPLAIN (COSTS FALSE, FORMAT XML) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; - - Task-Tracker - - 4 - None, not supported for re-partition queries - - - 4 - 4 - - - 16 - 4 - - - 1 - 4 - - - - - 1 - 4 - - - - - + + + Aggregate + Plain + - Aggregate - Plain - - - Seq Scan - Outer - pg_merge_job_68720796765 - pg_merge_job_68720796765 - - + Custom Scan + Outer + Citus Task-Tracker + + + 4 + None, not supported for re-partition queries + + + 4 + 4 + + + 16 + 4 + + + 1 + 4 + + + + + 1 + 4 + + + + - - - + + + SELECT true AS valid FROM explain_xml($$ SELECT count(*) @@ -654,26 +608,24 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) WHERE l_orderkey = o_orderkey AND o_custkey = c_custkey AND l_suppkey = s_suppkey; -- Executor: "Task-Tracker" - Job: - Task Count: 4 - Tasks Shown: "None, not supported for re-partition queries" - Depended Jobs: - - Map Task Count: 4 - Merge Task Count: 4 - Depended Jobs: - - Map Task Count: 16 - Merge Task Count: 4 - - Map Task Count: 1 - Merge Task Count: 4 - - Map Task Count: 1 - Merge Task Count: 4 - Master Query: - - Plan: - Node Type: "Aggregate" - Strategy: "Plain" - Plans: - - Node Type: "Seq Scan" - Parent Relationship: "Outer" - Relation Name: "pg_merge_job_68720796775" - Alias: "pg_merge_job_68720796775" +- Plan: + Node Type: "Aggregate" + Strategy: "Plain" + Plans: + - Node Type: "Custom Scan" + Parent Relationship: "Outer" + Custom Plan Provider: "Citus Task-Tracker" + Distributed Query: + Job: + Task Count: 4 + Tasks Shown: "None, not supported for re-partition queries" + Depended Jobs: + - Map Task Count: 4 + Merge Task Count: 4 + Depended Jobs: + - Map Task Count: 16 + Merge Task Count: 4 + - Map Task Count: 1 + Merge Task Count: 4 + - Map Task Count: 1 + Merge Task Count: 4 diff --git a/src/test/regress/expected/multi_mx_reference_table.out b/src/test/regress/expected/multi_mx_reference_table.out index 34c39f1b0..0aecc94f6 100644 --- a/src/test/regress/expected/multi_mx_reference_table.out +++ b/src/test/regress/expected/multi_mx_reference_table.out @@ -525,6 +525,12 @@ FETCH test_cursor; -- fetch one row after the last ---------+---------+---------+--------- (0 rows) +FETCH BACKWARD test_cursor; + value_1 | value_2 | value_3 | value_4 +---------+---------+---------+-------------------------- + 2 | 2 | 2 | Fri Dec 02 00:00:00 2016 +(1 row) + END; -- table creation queries inside can be router plannable CREATE TEMP TABLE temp_reference_test as diff --git a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out index c0bf72d1b..3ccf84eaf 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -173,10 +173,9 @@ EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other ON repartition_udt.udtcol = repartition_udt_other.udtcol WHERE repartition_udt.pk > 1; LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ] - QUERY PLAN -------------------------------------------------------------------------- - Distributed Query into pg_merge_job_535003 - Executor: Task-Tracker + QUERY PLAN +-------------------------------------------------------------------- + Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -185,9 +184,7 @@ LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_ot -> MapMergeJob Map Task Count: 5 Merge Task Count: 4 - Master Query - -> Seq Scan on pg_merge_job_535003 (cost=0.00..0.00 rows=0 width=0) -(12 rows) +(9 rows) SELECT * FROM repartition_udt JOIN repartition_udt_other ON repartition_udt.udtcol = repartition_udt_other.udtcol diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index 95a851afe..9c6b09cf2 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -1312,6 +1312,12 @@ FETCH test_cursor; 11 | 1 | alamo | 1347 (1 row) +FETCH BACKWARD test_cursor; + id | author_id | title | word_count +----+-----------+----------+------------ + 1 | 1 | arsenous | 9572 +(1 row) + END; -- queries inside copy can be router plannable COPY ( @@ -1454,7 +1460,10 @@ CONTEXT: SQL statement "SELECT ah.id, ah.word_count WHERE author_id = 1" PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY DEBUG: Plan is router executable -CONTEXT: PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY +CONTEXT: SQL statement "SELECT ah.id, ah.word_count + FROM articles_hash_mx ah + WHERE author_id = 1" +PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY id | word_count ----+------------ 1 | 9572 @@ -1489,7 +1498,6 @@ NOTICE: cannot use shard pruning with ANY/ALL (array expression) HINT: Consider rewriting the expression with OR/AND clauses. NOTICE: cannot use shard pruning with ANY/ALL (array expression) HINT: Consider rewriting the expression with OR/AND clauses. -ERROR: cannot create temporary table within security-restricted operation -- router planner/executor is disabled for task-tracker executor -- following query is router plannable, but router planner is disabled diff --git a/src/test/regress/expected/multi_mx_schema_support.out b/src/test/regress/expected/multi_mx_schema_support.out index ed171c8c7..5035dc96f 100644 --- a/src/test/regress/expected/multi_mx_schema_support.out +++ b/src/test/regress/expected/multi_mx_schema_support.out @@ -37,6 +37,17 @@ FETCH test_cursor; 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon (1 row) +FETCH test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+--------+-------------+----------- +(0 rows) + +FETCH BACKWARD test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+---------------------------+-------------+------------------------------------------------------------------------------ + 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon +(1 row) + END; -- test with search_path is set SET search_path TO citus_mx_test_schema; @@ -51,6 +62,17 @@ FETCH test_cursor; 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon (1 row) +FETCH test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+--------+-------------+----------- +(0 rows) + +FETCH BACKWARD test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+---------------------------+-------------+------------------------------------------------------------------------------ + 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon +(1 row) + END; -- test inserting to table in different schema SET search_path TO public; diff --git a/src/test/regress/expected/multi_partition_pruning.out b/src/test/regress/expected/multi_partition_pruning.out index 446b67cfc..65034ae48 100644 --- a/src/test/regress/expected/multi_partition_pruning.out +++ b/src/test/regress/expected/multi_partition_pruning.out @@ -170,25 +170,31 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, -- one shard. EXPLAIN SELECT count(*) FROM varchar_partitioned_table WHERE varchar_column = 'BA2'; DEBUG: predicate pruning for shardId 100 - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM array_partitioned_table WHERE array_column > '{BA1000U2AMO4ZGX, BZZXSP27F21T6}'; DEBUG: predicate pruning for shardId 102 - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM composite_partitioned_table WHERE composite_column < '(b,5,c)'::composite_type; DEBUG: predicate pruning for shardId 105 - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) SET client_min_messages TO NOTICE; diff --git a/src/test/regress/expected/multi_query_directory_cleanup.out b/src/test/regress/expected/multi_query_directory_cleanup.out index 6395248b6..e7de86b7c 100644 --- a/src/test/regress/expected/multi_query_directory_cleanup.out +++ b/src/test/regress/expected/multi_query_directory_cleanup.out @@ -76,25 +76,145 @@ SELECT pg_ls_dir('base/pgsql_job_cache'); -- transaction. BEGIN; DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_00; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_01 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_01; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_02 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_02; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_03 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_03; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_04 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_04; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_05 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_05; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_06 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_06; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_07 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_07; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_08 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_08; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_09 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_09; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_10 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_10; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_11 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_11; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_12 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_12; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_13 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_13; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_14 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_14; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_15 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_15; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_16 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_16; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_17 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_17; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_18 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_18; + revenue +--------------- + 22770844.7654 +(1 row) + DECLARE c_19 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_19; + revenue +--------------- + 22770844.7654 +(1 row) + SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f; f ----------------- diff --git a/src/test/regress/expected/multi_reference_table.out b/src/test/regress/expected/multi_reference_table.out index 6544b9f40..2e363623d 100644 --- a/src/test/regress/expected/multi_reference_table.out +++ b/src/test/regress/expected/multi_reference_table.out @@ -571,6 +571,12 @@ FETCH test_cursor; -- fetch one row after the last ---------+---------+---------+--------- (0 rows) +FETCH BACKWARD test_cursor; + value_1 | value_2 | value_3 | value_4 +---------+---------+---------+-------------------------- + 2 | 2 | 2 | Fri Dec 02 00:00:00 2016 +(1 row) + END; -- table creation queries inside can be router plannable CREATE TEMP TABLE temp_reference_test as diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index 065b6d19d..6deeefddb 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -181,10 +181,9 @@ EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other ON repartition_udt.udtcol = repartition_udt_other.udtcol WHERE repartition_udt.pk > 1; LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ] - QUERY PLAN -------------------------------------------------------------------------- - Distributed Query into pg_merge_job_535003 - Executor: Task-Tracker + QUERY PLAN +-------------------------------------------------------------------- + Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -193,9 +192,7 @@ LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_ot -> MapMergeJob Map Task Count: 5 Merge Task Count: 4 - Master Query - -> Seq Scan on pg_merge_job_535003 (cost=0.00..0.00 rows=0 width=0) -(12 rows) +(9 rows) SELECT * FROM repartition_udt JOIN repartition_udt_other ON repartition_udt.udtcol = repartition_udt_other.udtcol diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index 76f27a989..63fe2e7ba 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -2057,6 +2057,12 @@ FETCH test_cursor; -- fetch one row after the last ----+-----------+-------+------------ (0 rows) +FETCH BACKWARD test_cursor; + id | author_id | title | word_count +----+-----------+----------+------------ + 41 | 1 | aznavour | 11814 +(1 row) + END; -- queries inside copy can be router plannable COPY ( @@ -2199,7 +2205,10 @@ CONTEXT: SQL statement "SELECT ah.id, ah.word_count WHERE author_id = 1" PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY DEBUG: Plan is router executable -CONTEXT: PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY +CONTEXT: SQL statement "SELECT ah.id, ah.word_count + FROM articles_hash ah + WHERE author_id = 1" +PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY id | word_count ----+------------ 1 | 9572 @@ -2210,12 +2219,12 @@ CONTEXT: PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUE (5 rows) -- materialized views can be created for router plannable queries -CREATE MATERIALIZED VIEW mv_articles_hash AS +CREATE MATERIALIZED VIEW mv_articles_hash_empty AS SELECT * FROM articles_hash WHERE author_id = 1; DEBUG: predicate pruning for shardId 840001 DEBUG: Creating router plan DEBUG: Plan is router executable -SELECT * FROM mv_articles_hash; +SELECT * FROM mv_articles_hash_empty; id | author_id | title | word_count ----+-----------+--------------+------------ 1 | 1 | arsenous | 9572 @@ -2225,14 +2234,27 @@ SELECT * FROM mv_articles_hash; 41 | 1 | aznavour | 11814 (5 rows) -CREATE MATERIALIZED VIEW mv_articles_hash_error AS +CREATE MATERIALIZED VIEW mv_articles_hash_data AS SELECT * FROM articles_hash WHERE author_id in (1,2); NOTICE: cannot use shard pruning with ANY/ALL (array expression) HINT: Consider rewriting the expression with OR/AND clauses. NOTICE: cannot use shard pruning with ANY/ALL (array expression) HINT: Consider rewriting the expression with OR/AND clauses. -ERROR: cannot create temporary table within security-restricted operation - +SELECT * FROM mv_articles_hash_data; + id | author_id | title | word_count +----+-----------+--------------+------------ + 1 | 1 | arsenous | 9572 + 11 | 1 | alamo | 1347 + 21 | 1 | arcading | 5890 + 31 | 1 | athwartships | 7271 + 41 | 1 | aznavour | 11814 + 2 | 2 | abducing | 13642 + 12 | 2 | archiblast | 18185 + 22 | 2 | antipope | 2728 + 32 | 2 | amazon | 11342 + 42 | 2 | ausable | 15885 +(10 rows) + -- router planner/executor is now enabled for task-tracker executor SET citus.task_executor_type to 'task-tracker'; SELECT id @@ -2346,7 +2368,8 @@ DROP USER router_user; DROP TABLE failure_test; DROP FUNCTION author_articles_max_id(); DROP FUNCTION author_articles_id_word_count(); -DROP MATERIALIZED VIEW mv_articles_hash; +DROP MATERIALIZED VIEW mv_articles_hash_empty; +DROP MATERIALIZED VIEW mv_articles_hash_data; DROP TABLE articles_hash; DROP TABLE articles_single_shard_hash; DROP TABLE authors_hash; diff --git a/src/test/regress/expected/multi_schema_support.out b/src/test/regress/expected/multi_schema_support.out index 97c582ceb..b476242b3 100644 --- a/src/test/regress/expected/multi_schema_support.out +++ b/src/test/regress/expected/multi_schema_support.out @@ -155,6 +155,18 @@ FETCH test_cursor; 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon (1 row) +FETCH test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+---------------------------+-------------+------------------------------------------------------------------------------ + 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon +(1 row) + +FETCH BACKWARD test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+---------------------------+-------------+------------------------------------------------------------------------------ + 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon +(1 row) + END; -- test with search_path is set SET search_path TO test_schema_support; @@ -169,6 +181,18 @@ FETCH test_cursor; 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon (1 row) +FETCH test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+---------------------------+-------------+------------------------------------------------------------------------------ + 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon +(1 row) + +FETCH BACKWARD test_cursor; + n_nationkey | n_name | n_regionkey | n_comment +-------------+---------------------------+-------------+------------------------------------------------------------------------------ + 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon +(1 row) + END; -- test inserting to table in different schema SET search_path TO public; diff --git a/src/test/regress/expected/multi_task_assignment_policy.out b/src/test/regress/expected/multi_task_assignment_policy.out index f571238ad..d5f4fdc95 100644 --- a/src/test/regress/expected/multi_task_assignment_policy.out +++ b/src/test/regress/expected/multi_task_assignment_policy.out @@ -60,10 +60,12 @@ DEBUG: assigned task 6 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57638 DEBUG: assigned task 4 to node localhost:57637 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM task_assignment_test_table; DEBUG: StartTransactionCommand @@ -72,10 +74,12 @@ DEBUG: assigned task 6 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57638 DEBUG: assigned task 4 to node localhost:57637 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Next test the first-replica task assignment policy SET citus.task_assignment_policy TO 'first-replica'; @@ -89,10 +93,12 @@ DEBUG: assigned task 6 to node localhost:57637 DEBUG: assigned task 4 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57638 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM task_assignment_test_table; DEBUG: StartTransactionCommand @@ -101,10 +107,12 @@ DEBUG: assigned task 6 to node localhost:57637 DEBUG: assigned task 4 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57638 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) -- Round-robin task assignment relies on the current jobId. We therefore need to -- ensure that jobIds start with an odd number here; this way, task assignment @@ -132,10 +140,12 @@ DEBUG: assigned task 6 to node localhost:57637 DEBUG: assigned task 4 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57638 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM task_assignment_test_table; DEBUG: StartTransactionCommand @@ -144,10 +154,12 @@ DEBUG: assigned task 6 to node localhost:57638 DEBUG: assigned task 4 to node localhost:57638 DEBUG: assigned task 2 to node localhost:57637 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) EXPLAIN SELECT count(*) FROM task_assignment_test_table; DEBUG: StartTransactionCommand @@ -156,10 +168,12 @@ DEBUG: assigned task 6 to node localhost:57637 DEBUG: assigned task 4 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57638 DEBUG: CommitTransactionCommand - QUERY PLAN ------------------------------------------------------------- - explain statements for distributed queries are not enabled -(1 row) + QUERY PLAN +----------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + explain statements for distributed queries are not enabled +(3 rows) RESET citus.task_assignment_policy; DEBUG: StartTransactionCommand diff --git a/src/test/regress/expected/multi_utility_statements.out b/src/test/regress/expected/multi_utility_statements.out index e4891e6c1..b7f4b3723 100644 --- a/src/test/regress/expected/multi_utility_statements.out +++ b/src/test/regress/expected/multi_utility_statements.out @@ -261,6 +261,12 @@ FETCH ABSOLUTE 5 FROM noHoldCursor; 1 | 5 | 24.00 | 0.10 (1 row) +FETCH BACKWARD noHoldCursor; + l_orderkey | l_linenumber | l_quantity | l_discount +------------+--------------+------------+------------ + 1 | 4 | 28.00 | 0.09 +(1 row) + COMMIT; FETCH ABSOLUTE 5 FROM noHoldCursor; ERROR: cursor "noholdcursor" does not exist diff --git a/src/test/regress/input/multi_alter_table_statements.source b/src/test/regress/input/multi_alter_table_statements.source index 8d38495b3..710abf9c0 100644 --- a/src/test/regress/input/multi_alter_table_statements.source +++ b/src/test/regress/input/multi_alter_table_statements.source @@ -283,12 +283,12 @@ DROP TABLE ddl_commands; -- Distributed SELECTs cannot appear after ALTER BEGIN; CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); -SELECT l_orderkey FROM lineitem_alter LIMIT 0; +SELECT count(*) FROM lineitem_alter; COMMIT; -- but are allowed before BEGIN; -SELECT l_orderkey FROM lineitem_alter LIMIT 0; +SELECT count(*) FROM lineitem_alter; CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); COMMIT; SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter'; diff --git a/src/test/regress/multi_task_tracker_extra_schedule b/src/test/regress/multi_task_tracker_extra_schedule index 1e76bb3f2..99f53d6d1 100644 --- a/src/test/regress/multi_task_tracker_extra_schedule +++ b/src/test/regress/multi_task_tracker_extra_schedule @@ -33,11 +33,9 @@ test: multi_agg_distinct multi_limit_clause multi_limit_clause_approximate test: multi_average_expression multi_working_columns test: multi_array_agg test: multi_agg_type_conversion multi_count_type_conversion -test: multi_partition_pruning -test: multi_join_pruning multi_hash_pruning +test: multi_hash_pruning test: multi_null_minmax_value_pruning test: multi_query_directory_cleanup -test: multi_task_assignment_policy test: multi_utility_statements test: multi_dropped_column_aliases @@ -52,7 +50,7 @@ test: multi_tpch_query7 multi_tpch_query7_nested # Parallel tests to check our join order planning logic. Note that we load data # below; and therefore these tests should come after the execution tests. # ---------- -test: multi_join_order_tpch_small multi_join_order_additional +test: multi_join_order_additional test: multi_load_more_data test: multi_join_order_tpch_large diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index b55fdfae7..b16ab3310 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -596,15 +596,16 @@ BEGIN; CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); NOTICE: using one-phase commit for distributed DDL commands HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc' -SELECT l_orderkey FROM lineitem_alter LIMIT 0; +SELECT count(*) FROM lineitem_alter; ERROR: cannot open new connections after the first modification command within a transaction COMMIT; -- but are allowed before BEGIN; -SELECT l_orderkey FROM lineitem_alter LIMIT 0; - l_orderkey ------------- -(0 rows) +SELECT count(*) FROM lineitem_alter; + count +------- + 18000 +(1 row) CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); COMMIT; diff --git a/src/test/regress/output/multi_subquery.source b/src/test/regress/output/multi_subquery.source index 0b981a1fc..4c5931f17 100644 --- a/src/test/regress/output/multi_subquery.source +++ b/src/test/regress/output/multi_subquery.source @@ -764,31 +764,28 @@ FROM GROUP BY tenant_id, user_id) AS subquery; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Distributed Query into pg_merge_job_270014 - Executor: Real-Time - Task Count: 2 - Tasks Shown: One of 2 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate (cost=40.01..40.02 rows=1 width=16) - -> GroupAggregate (cost=39.89..39.99 rows=1 width=48) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Merge Join (cost=39.89..39.97 rows=1 width=540) - Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) - -> Sort (cost=28.08..28.09 rows=6 width=32) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Sort (cost=11.81..11.82 rows=3 width=556) - Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) - -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556) - Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) - Master Query - -> Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Seq Scan on pg_merge_job_270014 (cost=0.00..0.00 rows=0 width=0) -(22 rows) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + Task Count: 2 + Tasks Shown: One of 2 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate (cost=40.01..40.02 rows=1 width=16) + -> GroupAggregate (cost=39.89..39.99 rows=1 width=48) + Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Merge Join (cost=39.89..39.97 rows=1 width=540) + Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) + -> Sort (cost=28.08..28.09 rows=6 width=32) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Sort (cost=11.81..11.82 rows=3 width=556) + Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) + -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556) + Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) +(19 rows) -- Union and left join subquery pushdown EXPLAIN SELECT @@ -853,52 +850,49 @@ FROM hasdone) AS subquery_top GROUP BY hasdone; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Distributed Query into pg_merge_job_270015 - Executor: Real-Time - Task Count: 2 - Tasks Shown: One of 2 - -> Task - Node: host=localhost port=57637 dbname=regression - -> GroupAggregate (cost=91.93..91.98 rows=2 width=48) - Group Key: subquery_top.hasdone - -> Sort (cost=91.93..91.93 rows=2 width=64) - Sort Key: subquery_top.hasdone - -> Subquery Scan on subquery_top (cost=91.85..91.92 rows=2 width=64) - -> GroupAggregate (cost=91.85..91.90 rows=2 width=112) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Sort (cost=91.85..91.85 rows=2 width=88) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Merge Left Join (cost=91.75..91.84 rows=2 width=88) - Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id))) - -> Unique (cost=79.46..79.48 rows=2 width=56) - -> Sort (cost=79.46..79.47 rows=2 width=56) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time - -> Append (cost=0.00..79.45 rows=2 width=56) - -> Nested Loop (cost=0.00..39.72 rows=1 width=56) - Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id)) - -> Seq Scan on events_270009 events (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'click'::text) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Nested Loop (cost=0.00..39.72 rows=1 width=56) - Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id)) - -> Seq Scan on events_270009 events_1 (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'submit'::text) - -> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Materialize (cost=12.29..12.31 rows=1 width=48) - -> Unique (cost=12.29..12.30 rows=1 width=80) - -> Sort (cost=12.29..12.29 rows=1 width=80) - Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) - -> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=80) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) - Master Query - -> HashAggregate (cost=0.00..0.00 rows=0 width=0) - Group Key: intermediate_column_270015_2 - -> Seq Scan on pg_merge_job_270015 (cost=0.00..0.00 rows=0 width=0) -(43 rows) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: hasdone + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + Task Count: 2 + Tasks Shown: One of 2 + -> Task + Node: host=localhost port=57637 dbname=regression + -> GroupAggregate (cost=91.93..91.98 rows=2 width=48) + Group Key: subquery_top.hasdone + -> Sort (cost=91.93..91.93 rows=2 width=64) + Sort Key: subquery_top.hasdone + -> Subquery Scan on subquery_top (cost=91.85..91.92 rows=2 width=64) + -> GroupAggregate (cost=91.85..91.90 rows=2 width=112) + Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) + -> Sort (cost=91.85..91.85 rows=2 width=88) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) + -> Merge Left Join (cost=91.75..91.84 rows=2 width=88) + Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id))) + -> Unique (cost=79.46..79.48 rows=2 width=56) + -> Sort (cost=79.46..79.47 rows=2 width=56) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time + -> Append (cost=0.00..79.45 rows=2 width=56) + -> Nested Loop (cost=0.00..39.72 rows=1 width=56) + Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id)) + -> Seq Scan on events_270009 events (cost=0.00..11.62 rows=1 width=40) + Filter: ((event_type)::text = 'click'::text) + -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Nested Loop (cost=0.00..39.72 rows=1 width=56) + Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id)) + -> Seq Scan on events_270009 events_1 (cost=0.00..11.62 rows=1 width=40) + Filter: ((event_type)::text = 'submit'::text) + -> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Materialize (cost=12.29..12.31 rows=1 width=48) + -> Unique (cost=12.29..12.30 rows=1 width=80) + -> Sort (cost=12.29..12.29 rows=1 width=80) + Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) + -> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=80) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) +(40 rows) -- Union, left join and having subquery pushdown EXPLAIN SELECT @@ -1022,37 +1016,34 @@ ORDER BY user_lastseen DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Distributed Query into pg_merge_job_270017 - Executor: Real-Time - Task Count: 2 - Tasks Shown: One of 2 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Limit (cost=100.43..100.44 rows=6 width=56) - -> Sort (cost=100.43..100.44 rows=6 width=56) - Sort Key: (max(users.lastseen)) DESC - -> GroupAggregate (cost=100.14..100.29 rows=6 width=56) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Sort (cost=100.14..100.16 rows=6 width=548) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548) - -> Limit (cost=28.08..28.09 rows=6 width=24) - -> Sort (cost=28.08..28.09 rows=6 width=24) - Sort Key: users.lastseen DESC - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=24) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Limit (cost=11.96..11.96 rows=1 width=524) - -> Sort (cost=11.96..11.96 rows=1 width=524) - Sort Key: events.event_time DESC - -> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) - Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id))) - Master Query - -> Limit (cost=0.00..0.00 rows=0 width=0) - -> Sort (cost=0.00..0.00 rows=0 width=0) - Sort Key: intermediate_column_270017_2 DESC - -> Seq Scan on pg_merge_job_270017 (cost=0.00..0.00 rows=0 width=0) -(29 rows) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit (cost=0.00..0.00 rows=0 width=0) + -> Sort (cost=0.00..0.00 rows=0 width=0) + Sort Key: user_lastseen DESC + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + Task Count: 2 + Tasks Shown: One of 2 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit (cost=100.43..100.44 rows=6 width=56) + -> Sort (cost=100.43..100.44 rows=6 width=56) + Sort Key: (max(users.lastseen)) DESC + -> GroupAggregate (cost=100.14..100.29 rows=6 width=56) + Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Sort (cost=100.14..100.16 rows=6 width=548) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548) + -> Limit (cost=28.08..28.09 rows=6 width=24) + -> Sort (cost=28.08..28.09 rows=6 width=24) + Sort Key: users.lastseen DESC + -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=24) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Limit (cost=11.96..11.96 rows=1 width=524) + -> Sort (cost=11.96..11.96 rows=1 width=524) + Sort Key: events.event_time DESC + -> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) + Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id))) +(26 rows) SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/output/multi_subquery_0.source b/src/test/regress/output/multi_subquery_0.source index ecc2ef7eb..096c204e4 100644 --- a/src/test/regress/output/multi_subquery_0.source +++ b/src/test/regress/output/multi_subquery_0.source @@ -764,31 +764,28 @@ FROM GROUP BY tenant_id, user_id) AS subquery; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Distributed Query into pg_merge_job_270014 - Executor: Real-Time - Task Count: 2 - Tasks Shown: One of 2 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Aggregate (cost=40.01..40.02 rows=1 width=32) - -> GroupAggregate (cost=39.89..39.99 rows=1 width=556) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Merge Join (cost=39.89..39.97 rows=1 width=556) - Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) - -> Sort (cost=28.08..28.09 rows=6 width=32) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Sort (cost=11.81..11.82 rows=3 width=556) - Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) - -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556) - Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) - Master Query - -> Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Seq Scan on pg_merge_job_270014 (cost=0.00..0.00 rows=0 width=0) -(22 rows) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Aggregate (cost=0.00..0.00 rows=0 width=0) + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + Task Count: 2 + Tasks Shown: One of 2 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Aggregate (cost=40.01..40.02 rows=1 width=32) + -> GroupAggregate (cost=39.89..39.99 rows=1 width=556) + Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Merge Join (cost=39.89..39.97 rows=1 width=556) + Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) + -> Sort (cost=28.08..28.09 rows=6 width=32) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Sort (cost=11.81..11.82 rows=3 width=556) + Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) + -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556) + Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) +(19 rows) -- Union and left join subquery pushdown EXPLAIN SELECT @@ -853,49 +850,46 @@ FROM hasdone) AS subquery_top GROUP BY hasdone; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Distributed Query into pg_merge_job_270015 - Executor: Real-Time - Task Count: 2 - Tasks Shown: One of 2 - -> Task - Node: host=localhost port=57637 dbname=regression - -> HashAggregate (cost=91.94..91.96 rows=2 width=64) - Group Key: COALESCE(('Has done paying'::text), 'Has not done paying'::text) - -> GroupAggregate (cost=91.85..91.90 rows=2 width=88) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Sort (cost=91.85..91.85 rows=2 width=88) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Merge Left Join (cost=91.75..91.84 rows=2 width=88) - Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id))) - -> Unique (cost=79.46..79.48 rows=2 width=40) - -> Sort (cost=79.46..79.47 rows=2 width=40) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time - -> Append (cost=0.00..79.45 rows=2 width=40) - -> Nested Loop (cost=0.00..39.72 rows=1 width=40) - Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id)) - -> Seq Scan on events_270009 events (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'click'::text) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Nested Loop (cost=0.00..39.72 rows=1 width=40) - Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id)) - -> Seq Scan on events_270009 events_1 (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'submit'::text) - -> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Materialize (cost=12.29..12.31 rows=1 width=48) - -> Unique (cost=12.29..12.30 rows=1 width=32) - -> Sort (cost=12.29..12.29 rows=1 width=32) - Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) - -> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) - Master Query - -> HashAggregate (cost=0.00..0.00 rows=0 width=0) - Group Key: intermediate_column_270015_2 - -> Seq Scan on pg_merge_job_270015 (cost=0.00..0.00 rows=0 width=0) -(40 rows) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + HashAggregate (cost=0.00..0.00 rows=0 width=0) + Group Key: hasdone + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + Task Count: 2 + Tasks Shown: One of 2 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate (cost=91.94..91.96 rows=2 width=64) + Group Key: COALESCE(('Has done paying'::text), 'Has not done paying'::text) + -> GroupAggregate (cost=91.85..91.90 rows=2 width=88) + Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) + -> Sort (cost=91.85..91.85 rows=2 width=88) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) + -> Merge Left Join (cost=91.75..91.84 rows=2 width=88) + Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id))) + -> Unique (cost=79.46..79.48 rows=2 width=40) + -> Sort (cost=79.46..79.47 rows=2 width=40) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time + -> Append (cost=0.00..79.45 rows=2 width=40) + -> Nested Loop (cost=0.00..39.72 rows=1 width=40) + Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id)) + -> Seq Scan on events_270009 events (cost=0.00..11.62 rows=1 width=40) + Filter: ((event_type)::text = 'click'::text) + -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Nested Loop (cost=0.00..39.72 rows=1 width=40) + Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id)) + -> Seq Scan on events_270009 events_1 (cost=0.00..11.62 rows=1 width=40) + Filter: ((event_type)::text = 'submit'::text) + -> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Materialize (cost=12.29..12.31 rows=1 width=48) + -> Unique (cost=12.29..12.30 rows=1 width=32) + -> Sort (cost=12.29..12.29 rows=1 width=32) + Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) + -> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=32) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) +(37 rows) -- Union, left join and having subquery pushdown EXPLAIN SELECT @@ -1019,37 +1013,34 @@ ORDER BY user_lastseen DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Distributed Query into pg_merge_job_270017 - Executor: Real-Time - Task Count: 2 - Tasks Shown: One of 2 - -> Task - Node: host=localhost port=57637 dbname=regression - -> Limit (cost=100.43..100.44 rows=6 width=56) - -> Sort (cost=100.43..100.44 rows=6 width=56) - Sort Key: (max(users.lastseen)) DESC - -> GroupAggregate (cost=100.14..100.29 rows=6 width=548) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Sort (cost=100.14..100.16 rows=6 width=548) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548) - -> Limit (cost=28.08..28.09 rows=6 width=40) - -> Sort (cost=28.08..28.09 rows=6 width=40) - Sort Key: users.lastseen DESC - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=40) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Limit (cost=11.96..11.96 rows=1 width=524) - -> Sort (cost=11.96..11.96 rows=1 width=524) - Sort Key: events.event_time DESC - -> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) - Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id))) - Master Query - -> Limit (cost=0.00..0.00 rows=0 width=0) - -> Sort (cost=0.00..0.00 rows=0 width=0) - Sort Key: intermediate_column_270017_2 DESC - -> Seq Scan on pg_merge_job_270017 (cost=0.00..0.00 rows=0 width=0) -(29 rows) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Limit (cost=0.00..0.00 rows=0 width=0) + -> Sort (cost=0.00..0.00 rows=0 width=0) + Sort Key: user_lastseen DESC + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) + Task Count: 2 + Tasks Shown: One of 2 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit (cost=100.43..100.44 rows=6 width=56) + -> Sort (cost=100.43..100.44 rows=6 width=56) + Sort Key: (max(users.lastseen)) DESC + -> GroupAggregate (cost=100.14..100.29 rows=6 width=548) + Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Sort (cost=100.14..100.16 rows=6 width=548) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548) + -> Limit (cost=28.08..28.09 rows=6 width=40) + -> Sort (cost=28.08..28.09 rows=6 width=40) + Sort Key: users.lastseen DESC + -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=40) + Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Limit (cost=11.96..11.96 rows=1 width=524) + -> Sort (cost=11.96..11.96 rows=1 width=524) + Sort Key: events.event_time DESC + -> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) + Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id))) +(26 rows) SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/sql/multi_join_order_additional.sql b/src/test/regress/sql/multi_join_order_additional.sql index 425e276fb..18d307017 100644 --- a/src/test/regress/sql/multi_join_order_additional.sql +++ b/src/test/regress/sql/multi_join_order_additional.sql @@ -11,6 +11,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 650000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; +SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise SET client_min_messages TO DEBUG2; -- Create new table definitions for use in testing in distributed planning and diff --git a/src/test/regress/sql/multi_join_order_tpch_large.sql b/src/test/regress/sql/multi_join_order_tpch_large.sql index 542b33385..20cf83dd0 100644 --- a/src/test/regress/sql/multi_join_order_tpch_large.sql +++ b/src/test/regress/sql/multi_join_order_tpch_large.sql @@ -11,6 +11,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 660000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; +SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise SET client_min_messages TO LOG; -- Change configuration to treat lineitem, orders, customer, and part tables as diff --git a/src/test/regress/sql/multi_mx_reference_table.sql b/src/test/regress/sql/multi_mx_reference_table.sql index c4843c077..31f89b60b 100644 --- a/src/test/regress/sql/multi_mx_reference_table.sql +++ b/src/test/regress/sql/multi_mx_reference_table.sql @@ -304,6 +304,7 @@ DECLARE test_cursor CURSOR FOR FETCH test_cursor; FETCH ALL test_cursor; FETCH test_cursor; -- fetch one row after the last +FETCH BACKWARD test_cursor; END; -- table creation queries inside can be router plannable diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index cbcc14d52..bdca55574 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -557,6 +557,7 @@ DECLARE test_cursor CURSOR FOR ORDER BY id; FETCH test_cursor; FETCH test_cursor; +FETCH BACKWARD test_cursor; END; -- queries inside copy can be router plannable diff --git a/src/test/regress/sql/multi_mx_schema_support.sql b/src/test/regress/sql/multi_mx_schema_support.sql index de631df22..b72a5ee04 100644 --- a/src/test/regress/sql/multi_mx_schema_support.sql +++ b/src/test/regress/sql/multi_mx_schema_support.sql @@ -21,6 +21,8 @@ DECLARE test_cursor CURSOR FOR FROM nation_hash WHERE n_nationkey = 1; FETCH test_cursor; +FETCH test_cursor; +FETCH BACKWARD test_cursor; END; -- test with search_path is set @@ -31,6 +33,8 @@ DECLARE test_cursor CURSOR FOR FROM nation_hash WHERE n_nationkey = 1; FETCH test_cursor; +FETCH test_cursor; +FETCH BACKWARD test_cursor; END; diff --git a/src/test/regress/sql/multi_query_directory_cleanup.sql b/src/test/regress/sql/multi_query_directory_cleanup.sql index 4192495e9..035d30263 100644 --- a/src/test/regress/sql/multi_query_directory_cleanup.sql +++ b/src/test/regress/sql/multi_query_directory_cleanup.sql @@ -46,25 +46,45 @@ SELECT pg_ls_dir('base/pgsql_job_cache'); -- transaction. BEGIN; DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_00; DECLARE c_01 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_01; DECLARE c_02 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_02; DECLARE c_03 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_03; DECLARE c_04 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_04; DECLARE c_05 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_05; DECLARE c_06 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_06; DECLARE c_07 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_07; DECLARE c_08 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_08; DECLARE c_09 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_09; DECLARE c_10 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_10; DECLARE c_11 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_11; DECLARE c_12 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_12; DECLARE c_13 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_13; DECLARE c_14 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_14; DECLARE c_15 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_15; DECLARE c_16 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_16; DECLARE c_17 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_17; DECLARE c_18 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_18; DECLARE c_19 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_19; SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f; -- close first, 17th (first after re-alloc) and last cursor. CLOSE c_00; diff --git a/src/test/regress/sql/multi_reference_table.sql b/src/test/regress/sql/multi_reference_table.sql index 96af1c710..d2a4476b9 100644 --- a/src/test/regress/sql/multi_reference_table.sql +++ b/src/test/regress/sql/multi_reference_table.sql @@ -335,6 +335,7 @@ DECLARE test_cursor CURSOR FOR FETCH test_cursor; FETCH ALL test_cursor; FETCH test_cursor; -- fetch one row after the last +FETCH BACKWARD test_cursor; END; -- table creation queries inside can be router plannable diff --git a/src/test/regress/sql/multi_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index 5f75b3d28..4d72d8e1f 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -917,6 +917,7 @@ DECLARE test_cursor CURSOR FOR FETCH test_cursor; FETCH ALL test_cursor; FETCH test_cursor; -- fetch one row after the last +FETCH BACKWARD test_cursor; END; -- queries inside copy can be router plannable @@ -988,14 +989,14 @@ $$ LANGUAGE plpgsql; SELECT * FROM author_articles_id_word_count(); -- materialized views can be created for router plannable queries -CREATE MATERIALIZED VIEW mv_articles_hash AS +CREATE MATERIALIZED VIEW mv_articles_hash_empty AS SELECT * FROM articles_hash WHERE author_id = 1; +SELECT * FROM mv_articles_hash_empty; -SELECT * FROM mv_articles_hash; - -CREATE MATERIALIZED VIEW mv_articles_hash_error AS +CREATE MATERIALIZED VIEW mv_articles_hash_data AS SELECT * FROM articles_hash WHERE author_id in (1,2); - +SELECT * FROM mv_articles_hash_data; + -- router planner/executor is now enabled for task-tracker executor SET citus.task_executor_type to 'task-tracker'; SELECT id @@ -1053,7 +1054,8 @@ DROP TABLE failure_test; DROP FUNCTION author_articles_max_id(); DROP FUNCTION author_articles_id_word_count(); -DROP MATERIALIZED VIEW mv_articles_hash; +DROP MATERIALIZED VIEW mv_articles_hash_empty; +DROP MATERIALIZED VIEW mv_articles_hash_data; DROP TABLE articles_hash; DROP TABLE articles_single_shard_hash; diff --git a/src/test/regress/sql/multi_schema_support.sql b/src/test/regress/sql/multi_schema_support.sql index 1b84e71ff..70feb03de 100644 --- a/src/test/regress/sql/multi_schema_support.sql +++ b/src/test/regress/sql/multi_schema_support.sql @@ -113,6 +113,8 @@ DECLARE test_cursor CURSOR FOR FROM test_schema_support.nation_append WHERE n_nationkey = 1; FETCH test_cursor; +FETCH test_cursor; +FETCH BACKWARD test_cursor; END; -- test with search_path is set @@ -123,6 +125,8 @@ DECLARE test_cursor CURSOR FOR FROM nation_append WHERE n_nationkey = 1; FETCH test_cursor; +FETCH test_cursor; +FETCH BACKWARD test_cursor; END; diff --git a/src/test/regress/sql/multi_utility_statements.sql b/src/test/regress/sql/multi_utility_statements.sql index a29842208..b1e8232c3 100644 --- a/src/test/regress/sql/multi_utility_statements.sql +++ b/src/test/regress/sql/multi_utility_statements.sql @@ -146,5 +146,6 @@ DECLARE noHoldCursor SCROLL CURSOR FOR ORDER BY l_orderkey, l_linenumber; FETCH ABSOLUTE 5 FROM noHoldCursor; +FETCH BACKWARD noHoldCursor; COMMIT; FETCH ABSOLUTE 5 FROM noHoldCursor;