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;