From 52358fe891b24da2fcbe8ca338e1795e5b6591a1 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Tue, 31 Jan 2017 20:28:22 -0800 Subject: [PATCH 1/2] Initial temp table removal implementation --- .../distributed/executor/multi_executor.c | 385 ++++---- .../executor/multi_router_executor.c | 294 ++---- .../distributed/executor/multi_utility.c | 45 - .../distributed/planner/multi_explain.c | 300 +----- .../planner/multi_master_planner.c | 168 +--- .../planner/multi_physical_planner.c | 1 + .../distributed/planner/multi_planner.c | 258 +++--- .../planner/multi_router_planner.c | 6 + src/backend/distributed/shared_library_init.c | 14 - .../distributed/utils/citus_outfuncs.c | 3 + .../distributed/utils/citus_readfuncs.c | 3 + src/include/distributed/multi_executor.h | 30 +- src/include/distributed/multi_explain.h | 3 - .../distributed/multi_master_planner.h | 7 +- .../distributed/multi_physical_planner.h | 4 + src/include/distributed/multi_planner.h | 4 +- .../distributed/multi_router_executor.h | 5 + src/test/regress/expected/multi_explain.out | 833 ++++++++--------- src/test/regress/expected/multi_explain_0.out | 785 ++++++++-------- .../expected/multi_join_order_additional.out | 80 +- .../expected/multi_join_order_tpch_large.out | 58 +- .../expected/multi_join_order_tpch_small.out | 46 +- .../regress/expected/multi_join_pruning.out | 30 +- .../regress/expected/multi_mx_explain.out | 856 +++++++++--------- .../regress/expected/multi_mx_explain_0.out | 808 ++++++++--------- .../multi_mx_repartition_udt_prepare.out | 10 +- .../expected/multi_mx_router_planner.out | 1 - .../expected/multi_partition_pruning.out | 30 +- .../multi_query_directory_cleanup.out | 120 +++ .../expected/multi_repartition_udt.out | 10 +- .../regress/expected/multi_router_planner.out | 26 +- .../expected/multi_task_assignment_policy.out | 70 +- .../input/multi_alter_table_statements.source | 4 +- .../multi_alter_table_statements.source | 11 +- src/test/regress/output/multi_subquery.source | 200 ++-- .../sql/multi_join_order_additional.sql | 1 + .../sql/multi_join_order_tpch_large.sql | 1 + .../sql/multi_query_directory_cleanup.sql | 20 + src/test/regress/sql/multi_router_planner.sql | 13 +- 39 files changed, 2553 insertions(+), 2990 deletions(-) diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 061f00619..502b8e862 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -24,63 +24,107 @@ #include "distributed/multi_utility.h" #include "distributed/worker_protocol.h" #include "executor/execdebug.h" +#include "executor/executor.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. + * FIXME: It'd probably be better to have different set of methods for: + * - router readonly queries + * - router modify + * - router insert ... select + * - real-time/task-tracker (no point in seperating those) * - * 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. + * I think it's better however to only have one type of CitusScanState, to + * allow to easily share code between routines. */ -void -multi_ExecutorStart(QueryDesc *queryDesc, int eflags) +static CustomExecMethods CitusCustomExecMethods = { + "CitusScan", + CitusBeginScan, + CitusExecScan, + CitusEndScan, + CitusReScan, +#if (PG_VERSION_NUM >= 90600) + NULL, /* NO EstimateDSMCustomScan callback */ + NULL, /* NO InitializeDSMCustomScan callback */ + NULL, /* NO InitializeWorkerCustomScan callback */ +#endif + NULL, + NULL, + CitusExplainScan +}; + + +Node * +CitusCreateScan(CustomScan *scan) { - PlannedStmt *planStatement = queryDesc->plannedstmt; + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); - if (HasCitusToplevelNode(planStatement)) + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->customScanState.methods = &CitusCustomExecMethods; + scanState->multiPlan = GetMultiPlan(scan); + scanState->executorType = JobExecutorType(scanState->multiPlan); + + return (Node *) scanState; +} + + +void +CitusBeginScan(CustomScanState *node, + EState *estate, + int eflags) +{ + CitusScanState *scanState = (CitusScanState *) node; + MultiPlan *multiPlan = scanState->multiPlan; + + Assert(IsA(scanState, CustomScanState)); + + /* ensure plan is executable */ + VerifyMultiPlanValidity(multiPlan); + + /* ExecCheckRTPerms(planStatement->rtable, true); */ + + if (scanState->executorType == MULTI_EXECUTOR_ROUTER) { - MultiPlan *multiPlan = GetMultiPlan(planStatement); - MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST; - Job *workerJob = multiPlan->workerJob; + RouterBeginScan(scanState); + } +} - /* ensure plan is executable */ - VerifyMultiPlanValidity(multiPlan); - ExecCheckRTPerms(planStatement->rtable, true); +TupleTableSlot * +CitusExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + MultiPlan *multiPlan = scanState->multiPlan; - executorType = JobExecutorType(multiPlan); - if (executorType == MULTI_EXECUTOR_ROUTER) + if (scanState->executorType == MULTI_EXECUTOR_ROUTER) + { + return RouterExecScan(scanState); + } + else + { + TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot; + + if (!scanState->finishedUnderlyingScan) { - 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); - } - else - { - PlannedStmt *masterSelectPlan = MasterNodeSelectPlan(multiPlan); - CreateStmt *masterCreateStmt = MasterNodeCreateStatement(multiPlan); - List *masterCopyStmtList = MasterNodeCopyStatementList(multiPlan); - RangeTblEntry *masterRangeTableEntry = NULL; + Job *workerJob = multiPlan->workerJob; StringInfo jobDirectoryName = NULL; + EState *executorState = scanState->customScanState.ss.ps.state; + List *workerTaskList = workerJob->taskList; + ListCell *workerTaskCell = NULL; + TupleDesc tupleDescriptor = NULL; + Relation fakeRel = NULL; + MemoryContext executorTupleContext = GetPerTupleMemoryContext(executorState); + ExprContext *executorExpressionContext = + GetPerTupleExprContext(executorState); + uint32 columnCount = 0; + Datum *columnValues = NULL; + bool *columnNulls = NULL; /* * We create a directory on the master node to keep task execution results. @@ -93,204 +137,133 @@ multi_ExecutorStart(QueryDesc *queryDesc, int eflags) ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId); /* pick distributed executor to use */ - if (eflags & EXEC_FLAG_EXPLAIN_ONLY) + if (executorState->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY) { /* skip distributed query execution for EXPLAIN commands */ } - else if (executorType == MULTI_EXECUTOR_REAL_TIME) + else if (scanState->executorType == MULTI_EXECUTOR_REAL_TIME) { MultiRealTimeExecute(workerJob); } - else if (executorType == MULTI_EXECUTOR_TASK_TRACKER) + else if (scanState->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); + tupleDescriptor = node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; - /* make the temporary table visible */ - CommandCounterIncrement(); + /* + * Load data, collected by Multi*Execute() above, into a + * tuplestore. For that first create a tuplestore, and then copy + * the files one-by-one. + * + * FIXME: Should probably be in a separate routine. + * + * Long term it'd be a lot better if Multi*Execute() directly + * filled the tuplestores, but that's a fair bit of work. + */ - if (!(eflags & EXEC_FLAG_EXPLAIN_ONLY)) + /* + * To be able to use copy.c, we need a Relation descriptor. As + * there's no relation corresponding to the data loaded from + * workers, fake one. We just need the bare minimal set of fields + * accessed by BeginCopyFrom(). + * + * FIXME: should be abstracted into a separate function. + */ + fakeRel = palloc0(sizeof(RelationData)); + fakeRel->rd_att = tupleDescriptor; + fakeRel->rd_rel = palloc0(sizeof(FormData_pg_class)); + fakeRel->rd_rel->relkind = RELKIND_RELATION; + + columnCount = tupleDescriptor->natts; + columnValues = palloc0(columnCount * sizeof(Datum)); + columnNulls = palloc0(columnCount * sizeof(bool)); + + Assert(scanState->tuplestorestate == NULL); + scanState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem); + + foreach(workerTaskCell, workerTaskList) { - CopyQueryResults(masterCopyStmtList); + Task *workerTask = (Task *) lfirst(workerTaskCell); + StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId); + StringInfo taskFilename = + TaskFilename(jobDirectoryName, workerTask->taskId); + List *copyOptions = NIL; + CopyState copyState = NULL; + + if (BinaryMasterCopyFormat) + { + DefElem *copyOption = makeDefElem("format", + (Node *) makeString("binary")); + copyOptions = lappend(copyOptions, copyOption); + } + copyState = BeginCopyFrom(fakeRel, 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(scanState->tuplestorestate, + tupleDescriptor, + columnValues, columnNulls); + MemoryContextSwitchTo(oldContext); + } } - /* - * 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->finishedUnderlyingScan = true; } - } - /* 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)) - { - standard_ExecutorStart(queryDesc, eflags); + if (scanState->tuplestorestate != NULL) + { + Tuplestorestate *tupleStore = scanState->tuplestorestate; + tuplestore_gettupleslot(tupleStore, true, false, resultSlot); + + return resultSlot; + } + + return NULL; } } -/* - * CopyQueryResults executes the commands that copy query results into a - * temporary table. - */ -static void -CopyQueryResults(List *masterCopyStmtList) -{ - ListCell *masterCopyStmtCell = NULL; - - /* 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(); -} - - -/* Execute query plan. */ void -multi_ExecutorRun(QueryDesc *queryDesc, ScanDirection direction, tuplecount_t count) +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 */ - RouterExecutorRun(queryDesc, direction, count); - } - else - { - /* drop into the standard executor */ - standard_ExecutorRun(queryDesc, direction, count); + tuplestore_end(scanState->tuplestorestate); + scanState->tuplestorestate = NULL; } } -/* Perform actions, like e.g. firing triggers, after the query has run. */ void -multi_ExecutorFinish(QueryDesc *queryDesc) +CitusReScan(CustomScanState *node) { - int eflags = queryDesc->estate->es_top_eflags; + CitusScanState *scanState = (CitusScanState *) node; - if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR) - { - /* drop into the router executor */ - RouterExecutorFinish(queryDesc); - } - else - { - /* drop into the standard executor */ - standard_ExecutorFinish(queryDesc); - } -} - - -/* - * 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. - */ -void -multi_ExecutorEnd(QueryDesc *queryDesc) -{ - int eflags = queryDesc->estate->es_top_eflags; - - if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR) - { - /* drop into the router executor */ - RouterExecutorEnd(queryDesc); - } - else - { - /* drop into the standard executor */ - standard_ExecutorEnd(queryDesc); - } + scanState->tuplestorestate = NULL; + scanState->finishedUnderlyingScan = true; /* - * 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. + * XXX: this probably already works, but if not should be easily + * supportable - probably hard to exercise right now though. */ - 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; - } + elog(WARNING, "unsupported at this point"); } diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index ceed971b2..6233b0d90 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -74,83 +74,33 @@ 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 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, + CitusScanState *scanState, TupleDesc tupleDescriptor); 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 ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterTypes, const char ***parameterValues); static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query, ParamListInfo paramListInfo); -static bool StoreQueryResult(MaterialState *routerState, MultiConnection *connection, +static bool StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, TupleDesc tupleDescriptor, 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. @@ -457,73 +407,47 @@ RequiresConsistentSnapshot(Task *task) } -/* - * RouterExecutorRun actually executes a single task on a worker. - */ void -RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count) +RouterBeginScan(CitusScanState *scanState) { - PlannedStmt *planStatement = queryDesc->plannedstmt; - MultiPlan *multiPlan = GetMultiPlan(planStatement); + 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) - { - bool isModificationQuery = false; - bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation; + ReacquireMetadataLocks(taskList); +} + +TupleTableSlot * +RouterExecScan(CitusScanState *scanState) +{ + MultiPlan *multiPlan = scanState->multiPlan; + TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot; + + if (!scanState->finishedUnderlyingScan) + { + Job *workerJob = multiPlan->workerJob; + List *taskList = workerJob->taskList; + bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation; + bool isModificationQuery = false; + CmdType operation = multiPlan->operation; + + /* should use IsModificationStmt or such */ 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))); - } if (requiresMasterEvaluation) { @@ -539,59 +463,42 @@ RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count) if (isModificationQuery) { - ExecuteSingleModifyTask(queryDesc, task, sendTuples); + bool sendTuples = multiPlan->hasReturning; + ExecuteSingleModifyTask(scanState, task, sendTuples); } else { - ExecuteSingleSelectTask(queryDesc, task); + ExecuteSingleSelectTask(scanState, task); } } else { - ExecuteMultipleTasks(queryDesc, taskList, isModificationQuery, + bool sendTuples = multiPlan->hasReturning; + ExecuteMultipleTasks(scanState, taskList, isModificationQuery, sendTuples); } /* mark underlying query as having executed */ - routerState->eof_underlying = true; + scanState->finishedUnderlyingScan = true; } /* if the underlying query produced output, return it */ - if (routerState->tuplestorestate != NULL) + + /* + * FIXME: centralize this into function to be shared between router and + * other executors? + */ + if (scanState->tuplestorestate != NULL) { - TupleDesc resultTupleDescriptor = queryDesc->tupDesc; - int64 returnedRows = 0; + Tuplestorestate *tupleStore = scanState->tuplestorestate; - /* return rows from the tuplestore */ - returnedRows = ReturnRowsFromTuplestore(count, resultTupleDescriptor, - destination, - routerState->tuplestorestate); + /* XXX: could trivially support backward scans here */ + tuplestore_gettupleslot(tupleStore, true, false, resultSlot); - /* - * 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; - } + return resultSlot; } -out: - - /* shutdown tuple receiver, if we started it */ - if (sendTuples) - { - (*destination->rShutdown)(destination); - } - - if (queryDesc->totaltime != NULL) - { - InstrStopNode(queryDesc->totaltime, estate->es_processed); - } - - MemoryContextSwitchTo(oldcontext); + return NULL; } @@ -603,11 +510,12 @@ 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; + TupleDesc tupleDescriptor = + scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; + ParamListInfo paramListInfo = + scanState->customScanState.ss.ps.state->es_param_list_info; List *taskPlacementList = task->taskPlacementList; ListCell *taskPlacementCell = NULL; char *queryString = task->queryString; @@ -639,7 +547,7 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task) continue; } - queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, + queryOK = StoreQueryResult(scanState, connection, tupleDescriptor, dontFailOnError, ¤tAffectedTupleCount); if (queryOK) { @@ -661,14 +569,14 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task) * framework), or errors out (failed on all placements). */ static void -ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, +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; + CmdType operation = scanState->multiPlan->operation; + TupleDesc tupleDescriptor = + scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; + EState *executorState = scanState->customScanState.ss.ps.state; + ParamListInfo paramListInfo = executorState->es_param_list_info; bool resultsOK = false; List *taskPlacementList = task->taskPlacementList; List *connectionList = NIL; @@ -761,7 +669,7 @@ ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, */ if (!gotResults && expectResults) { - queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, + queryOK = StoreQueryResult(scanState, connection, tupleDescriptor, failOnError, ¤tAffectedTupleCount); } else @@ -893,20 +801,21 @@ 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; + TupleDesc tupleDescriptor = + scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; + 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); + /* XXX: Seems very redundant to pass both scanState and tupleDescriptor */ affectedTupleCount = ExecuteModifyTasks(taskList, expectResults, paramListInfo, - routerState, tupleDescriptor); + scanState, tupleDescriptor); executorState->es_processed = affectedTupleCount; } @@ -936,7 +845,7 @@ ExecuteModifyTasksWithoutResults(List *taskList) */ static int64 ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListInfo, - MaterialState *routerState, TupleDesc tupleDescriptor) + CitusScanState *scanState, TupleDesc tupleDescriptor) { int64 totalAffectedTupleCount = 0; ListCell *taskCell = NULL; @@ -1066,9 +975,9 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn */ if (placementIndex == 0 && expectResults) { - Assert(routerState != NULL && tupleDescriptor != NULL); + Assert(scanState != NULL && tupleDescriptor != NULL); - queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, + queryOK = StoreQueryResult(scanState, connection, tupleDescriptor, failOnError, ¤tAffectedTupleCount); } else @@ -1149,50 +1058,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,12 +1183,13 @@ ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterT * the connection. */ static bool -StoreQueryResult(MaterialState *routerState, MultiConnection *connection, +StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, TupleDesc tupleDescriptor, bool failOnError, int64 *rows) { 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 *)); bool commandFailed = false; MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext, @@ -1333,17 +1199,17 @@ 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(false, false, 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 (;;) { 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..97ef0ee7b 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, @@ -90,290 +84,62 @@ static void ExplainXMLTag(const char *tagname, int flags, ExplainState *es); static void ExplainJSONLineEnding(ExplainState *es); 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. - */ 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; + const char *executorName = NULL; 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); - - if (es->format == EXPLAIN_FORMAT_TEXT) - { - appendStringInfoSpaces(es->str, es->indent * 2); - appendStringInfo(es->str, "Distributed Query"); - - 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; - } + /* + * XXX: can we get by without the open/close group somehow - then we'd not + * copy any code from explain.c? Seems unlikely. + */ + ExplainOpenGroup("Distributed Query", "Distributed Query", true, es); /* - * Use a snapshot with an updated command ID to ensure this query sees - * results of any previously executed queries. + * XXX: might be worthwhile to put this somewhere central, e.g. for + * debugging output. */ - 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) + switch (scanState->executorType) { - 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) + case MULTI_EXECUTOR_ROUTER: { - dir = NoMovementScanDirection; + executorName = "Router"; } - else + break; + + case MULTI_EXECUTOR_REAL_TIME: { - dir = ForwardScanDirection; + executorName = "Real-Time"; } + break; - /* run the plan */ - ExecutorRun(queryDesc, dir, 0L); + case MULTI_EXECUTOR_TASK_TRACKER: + { + executorName = "Task-Tracker"; + } + break; - /* run cleanup too */ - ExecutorFinish(queryDesc); + default: + { + executorName = "Other"; + } + break; } + ExplainPropertyText("Executor", executorName, es); - /* - * 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); + ExplainJob(multiPlan->workerJob, es); - /* - * 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..facff1f9d 100644 --- a/src/backend/distributed/planner/multi_master_planner.c +++ b/src/backend/distributed/planner/multi_master_planner.c @@ -34,7 +34,7 @@ * a target target list for the master node. This master target list keeps the * temporary table's columns on the master node. */ -static List * +List * MasterTargetList(List *workerTargetList) { List *masterTargetList = NIL; @@ -67,49 +67,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 @@ -213,14 +170,22 @@ BuildAggregatePlan(Query *masterQuery, Plan *subPlan) */ static PlannedStmt * BuildSelectStatement(Query *masterQuery, char *masterTableName, - List *masterTargetList) + List *masterTargetList, CustomScan *dataScan) { PlannedStmt *selectStatement = NULL; RangeTblEntry *rangeTableEntry = NULL; RangeTblEntry *queryRangeTableEntry = NULL; - SeqScan *sequentialScan = NULL; Agg *aggregationPlan = NULL; Plan *topLevelPlan = NULL; + ListCell *lc = NULL; + List *columnNames = NULL; + + /* (0) compute column names */ + foreach(lc, masterTargetList) + { + TargetEntry *te = lfirst(lc); + columnNames = lappend(columnNames, makeString(te->resname)); + } /* (1) make PlannedStmt and set basic information */ selectStatement = makeNode(PlannedStmt); @@ -233,9 +198,8 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName, 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->rtekind = RTE_VALUES; /* can't look up relation */ + rangeTableEntry->eref = makeAlias("remote scan", columnNames); rangeTableEntry->inh = false; rangeTableEntry->inFromCl = true; @@ -243,22 +207,21 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName, selectStatement->rtable = list_make1(rangeTableEntry); /* (2) build and initialize sequential scan node */ - sequentialScan = makeNode(SeqScan); - sequentialScan->scanrelid = 1; /* always one */ + /* Gone */ /* (3) add an aggregation plan if needed */ if (masterQuery->hasAggs || masterQuery->groupClause) { - sequentialScan->plan.targetlist = masterTargetList; + dataScan->scan.plan.targetlist = masterTargetList; - aggregationPlan = BuildAggregatePlan(masterQuery, (Plan *) sequentialScan); + aggregationPlan = BuildAggregatePlan(masterQuery, &dataScan->scan.plan); topLevelPlan = (Plan *) aggregationPlan; } else { /* otherwise set the final projections on the scan plan directly */ - sequentialScan->plan.targetlist = masterQuery->targetList; - topLevelPlan = (Plan *) sequentialScan; + dataScan->scan.plan.targetlist = masterQuery->targetList; + topLevelPlan = &dataScan->scan.plan; } /* (4) add a sorting plan if needed */ @@ -303,54 +266,6 @@ 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 @@ -358,7 +273,7 @@ MasterNodeCreateStatement(MultiPlan *multiPlan) * retrieved from worker nodes and are merged into a temporary table. */ PlannedStmt * -MasterNodeSelectPlan(MultiPlan *multiPlan) +MasterNodeSelectPlan(MultiPlan *multiPlan, CustomScan *dataScan) { Query *masterQuery = multiPlan->masterQuery; char *tableName = multiPlan->masterTableName; @@ -368,49 +283,8 @@ MasterNodeSelectPlan(MultiPlan *multiPlan) List *workerTargetList = workerJob->jobQuery->targetList; List *masterTargetList = MasterTargetList(workerTargetList); - masterSelectPlan = BuildSelectStatement(masterQuery, tableName, masterTargetList); + masterSelectPlan = + BuildSelectStatement(masterQuery, tableName, masterTargetList, dataScan); 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..a4a87c372 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -221,6 +221,7 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) 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..bc0c1335f 100644 --- a/src/backend/distributed/planner/multi_planner.c +++ b/src/backend/distributed/planner/multi_planner.c @@ -16,10 +16,12 @@ #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" @@ -37,7 +39,6 @@ static List *relationRestrictionContextList = NIL; /* local function forward declarations */ static void CheckNodeIsDumpable(Node *node); -static char * GetMultiPlanString(PlannedStmt *result); static PlannedStmt * MultiQueryContainerNode(PlannedStmt *result, struct MultiPlan *multiPlan); static struct PlannedStmt * CreateDistributedPlan(PlannedStmt *localPlan, @@ -293,19 +294,25 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query } +static CustomScanMethods CitusCustomScanMethods = { + "CitusScan", + CitusCreateScan +}; + + /* - * 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) { - char *serializedMultiPlan = NULL; MultiPlan *multiPlan = NULL; - serializedMultiPlan = GetMultiPlanString(result); - multiPlan = (MultiPlan *) CitusStringToNode(serializedMultiPlan); - Assert(CitusIsA(multiPlan, MultiPlan)); + Assert(IsA(customScan, CustomScan)); + Assert(customScan->methods == &CitusCustomScanMethods); + Assert(list_length(customScan->custom_private) == 1); + + multiPlan = DeSerializeMultiPlan(linitial(customScan->custom_private)); return multiPlan; } @@ -315,24 +322,49 @@ GetMultiPlan(PlannedStmt *result) bool HasCitusToplevelNode(PlannedStmt *result) { - /* - * 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()) - { - return false; - } + elog(ERROR, "gone"); +} - if (GetMultiPlanString(result) == NULL) - { - return false; - } - else - { - return true; - } + +Node * +SerializableMultiPlan(MultiPlan *multiPlan) +{ + /* + * FIXME: 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. + */ + char *serializedPlan = NULL; + Const *multiPlanData = NULL; + + serializedPlan = CitusNodeToString(multiPlan); + + multiPlanData = makeNode(Const); + multiPlanData->consttype = CSTRINGOID; + multiPlanData->constlen = strlen(serializedPlan); + multiPlanData->constvalue = CStringGetDatum(serializedPlan); + multiPlanData->constbyval = false; + multiPlanData->location = -1; + + return (Node *) multiPlanData; +} + + +MultiPlan * +DeSerializeMultiPlan(Node *node) +{ + Const *multiPlanData = NULL; + char *serializedMultiPlan = NULL; + MultiPlan *multiPlan = NULL; + + Assert(IsA(node, Const)); + multiPlanData = (Const *) node; + serializedMultiPlan = DatumGetCString(multiPlanData->constvalue); + + multiPlan = (MultiPlan *) CitusStringToNode(serializedMultiPlan); + Assert(CitusIsA(multiPlan, MultiPlan)); + + return multiPlan; } @@ -346,124 +378,98 @@ HasCitusToplevelNode(PlannedStmt *result) * which should not be referred to outside this file, as it's likely to become * version dependant. Use GetMultiPlan() and HasCitusToplevelNode() to access. * + * FIXME + * * 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. */ PlannedStmt * -MultiQueryContainerNode(PlannedStmt *result, MultiPlan *multiPlan) +MultiQueryContainerNode(PlannedStmt *originalPlan, MultiPlan *multiPlan) { - FunctionScan *fauxFunctionScan = NULL; - RangeTblFunction *fauxFunction = NULL; - FuncExpr *fauxFuncExpr = NULL; - Const *multiPlanData = NULL; - char *serializedPlan = NULL; + PlannedStmt *resultPlan = NULL; + CustomScan *customScan = makeNode(CustomScan); + Node *multiPlanData = SerializableMultiPlan(multiPlan); - /* 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; + customScan->methods = &CitusCustomScanMethods; + customScan->custom_private = list_make1(multiPlanData); - 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); + /* FIXME: This probably ain't correct */ + if (ExecSupportsBackwardScan(originalPlan->planTree)) + { + customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN; + } /* - * 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. + * FIXME: these two branches/pieces of code should probably be moved into + * router / logical planner code respectively. */ - if (!ExecSupportsBackwardScan(result->planTree)) + if (multiPlan->masterQuery) { - FuncExpr *funcExpr = makeNode(FuncExpr); - TargetEntry *targetEntry = NULL; - bool resjunkAttribute = true; + resultPlan = MasterNodeSelectPlan(multiPlan, customScan); + resultPlan->queryId = originalPlan->queryId; + resultPlan->utilityStmt = originalPlan->utilityStmt; + } + else + { + ListCell *lc = NULL; + List *targetList = NIL; + bool foundJunk = false; + RangeTblEntry *rangeTableEntry = NULL; + List *columnNames = NIL; + int newRTI = list_length(originalPlan->rtable) + 1; - funcExpr->funcretset = true; + /* + * XXX: This basically just builds a targetlist to "read" from the + * custom scan output. + */ + foreach(lc, originalPlan->planTree->targetlist) + { + TargetEntry *te = lfirst(lc); + Var *newVar = NULL; + TargetEntry *newTargetEntry = NULL; - targetEntry = makeTargetEntry((Expr *) funcExpr, InvalidAttrNumber, NULL, - resjunkAttribute); + Assert(IsA(te, TargetEntry)); - fauxFunctionScan->scan.plan.targetlist = - lappend(fauxFunctionScan->scan.plan.targetlist, - targetEntry); + /* + * XXX: I can't think of a case where we'd need resjunk stuff at + * the toplevel of a router query - all things needing it have + * been pushed down. + */ + if (te->resjunk) + { + foundJunk = true; + continue; + } + + if (foundJunk) + { + ereport(ERROR, (errmsg("unexpected !junk entry after resjunk entry"))); + } + + /* build TE pointing to custom scan */ + newVar = makeVarFromTargetEntry(newRTI, te); + newTargetEntry = flatCopyTargetEntry(te); + newTargetEntry->expr = (Expr *) newVar; + targetList = lappend(targetList, newTargetEntry); + + columnNames = lappend(columnNames, makeString(te->resname)); + } + + /* XXX: can't think of a better RTE type than VALUES */ + rangeTableEntry = makeNode(RangeTblEntry); + rangeTableEntry->rtekind = RTE_VALUES; /* can't look up relation */ + rangeTableEntry->eref = makeAlias("remote_scan", columnNames); + rangeTableEntry->inh = false; + rangeTableEntry->inFromCl = true; + + resultPlan = originalPlan; + resultPlan->planTree = (Plan *) customScan; + resultPlan->rtable = lappend(resultPlan->rtable, rangeTableEntry); + customScan->scan.plan.targetlist = targetList; } - result->planTree = (Plan *) fauxFunctionScan; - - return result; -} - - -/* - * GetMultiPlanString returns either NULL, if the plan is not a distributed - * one, or the string representing the distributed plan. - */ -static char * -GetMultiPlanString(PlannedStmt *result) -{ - FunctionScan *fauxFunctionScan = NULL; - RangeTblFunction *fauxFunction = NULL; - FuncExpr *fauxFuncExpr = NULL; - Const *multiPlanData = NULL; - - if (!IsA(result->planTree, FunctionScan)) - { - return NULL; - } - - fauxFunctionScan = (FunctionScan *) result->planTree; - - if (list_length(fauxFunctionScan->functions) != 1) - { - return NULL; - } - - fauxFunction = linitial(fauxFunctionScan->functions); - - if (!IsA(fauxFunction->funcexpr, FuncExpr)) - { - return NULL; - } - - fauxFuncExpr = (FuncExpr *) fauxFunction->funcexpr; - - if (fauxFuncExpr->funcid != CitusExtraDataContainerFuncId()) - { - return NULL; - } - - 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); - - return DatumGetCString(multiPlanData->constvalue); + return resultPlan; } diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 368a96094..11212ec81 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) { @@ -236,6 +238,7 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query, multiPlan->masterQuery = NULL; multiPlan->masterTableName = NULL; multiPlan->routerExecutable = true; + multiPlan->hasReturning = list_length(originalQuery->returningList) > 0; return multiPlan; } @@ -264,6 +267,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. @@ -319,6 +324,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery, multiPlan->masterTableName = NULL; multiPlan->masterQuery = NULL; multiPlan->routerExecutable = true; + multiPlan->hasReturning = list_length(originalQuery->returningList) > 0; return multiPlan; } diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index c400e1e42..05f92f486 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" @@ -117,10 +116,6 @@ _PG_init(void) */ if (planner_hook != NULL || ExplainOneQuery_hook != NULL || - ExecutorStart_hook != NULL || - ExecutorRun_hook != NULL || - ExecutorFinish_hook != NULL || - ExecutorEnd_hook != NULL || ProcessUtility_hook != NULL) { ereport(ERROR, (errmsg("Citus has to be loaded first"), @@ -147,15 +142,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..8da52e506 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -276,6 +276,9 @@ 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); diff --git a/src/backend/distributed/utils/citus_readfuncs.c b/src/backend/distributed/utils/citus_readfuncs.c index 4d51ae86d..fca056e4e 100644 --- a/src/backend/distributed/utils/citus_readfuncs.c +++ b/src/backend/distributed/utils/citus_readfuncs.c @@ -183,6 +183,9 @@ 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); diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 995d9d525..53fc327b9 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -12,6 +12,10 @@ #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 @@ -23,10 +27,26 @@ #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; + MultiPlan *multiPlan; + MultiExecutorType executorType; + + /* state for router */ + bool finishedUnderlyingScan; + Tuplestorestate *tuplestorestate; +} CitusScanState; + +Node * CitusCreateScan(CustomScan *scan); +extern void CitusBeginScan(CustomScanState *node, + EState *estate, + int eflags); +extern TupleTableSlot * CitusExecScan(CustomScanState *node); +extern void CitusEndScan(CustomScanState *node); +extern void CitusReScan(CustomScanState *node); +extern void CitusExplainScan(CustomScanState *node, List *ancestors, + struct ExplainState *es); #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..c58b42717 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); +extern List * MasterTargetList(List *workerTargetList); #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..ccf3b44a2 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -213,6 +213,10 @@ typedef struct JoinSequenceNode typedef struct MultiPlan { CitusNode type; + + CmdType operation; + bool hasReturning; + Job *workerJob; Query *masterQuery; char *masterTableName; diff --git a/src/include/distributed/multi_planner.h b/src/include/distributed/multi_planner.h index 1c1d7a337..e2f255082 100644 --- a/src/include/distributed/multi_planner.h +++ b/src/include/distributed/multi_planner.h @@ -53,7 +53,9 @@ extern PlannedStmt * multi_planner(Query *parse, int cursorOptions, extern bool HasCitusToplevelNode(PlannedStmt *planStatement); struct MultiPlan; -extern struct MultiPlan * GetMultiPlan(PlannedStmt *planStatement); +extern struct MultiPlan * GetMultiPlan(CustomScan *node); +extern Node * SerializableMultiPlan(struct MultiPlan *multiPlan); +extern struct MultiPlan * DeSerializeMultiPlan(Node *node); extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index index, RangeTblEntry *rte); extern bool IsModifyCommand(Query *query); diff --git a/src/include/distributed/multi_router_executor.h b/src/include/distributed/multi_router_executor.h index e369d0f1f..b42f6e002 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,6 +35,9 @@ typedef struct XactShardConnSet extern bool AllModificationsCommutative; extern bool EnableDeadlockPrevention; +extern void RouterBeginScan(CitusScanState *scanState); + +extern TupleTableSlot * RouterExecScan(CitusScanState *scanState); extern void RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList); extern void RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count); diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index bafe813bb..cffad3c26 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -39,88 +39,83 @@ $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 (CitusScan) + 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 -- 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": "CitusScan", + "Parallel Aware": false, + "Distributed Query": { + "Executor": "Real-Time", + "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 +128,72 @@ 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 + CitusScan false - - intermediate_column_570003_0 - - - - Seq Scan - Outer - false - pg_merge_job_570003 - pg_merge_job_570003 - - + + 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 + + + + + + + + + + - - - + + + -- Validate XML format SELECT true AS valid FROM explain_xml($$ @@ -212,114 +204,107 @@ 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: "CitusScan" 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: + 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" + -- 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 (CitusScan) + 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 -- 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 (CitusScan) + Output: "?column?", "?column?_1", "?column?_2" + 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 -- 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 (CitusScan) + 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 -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem VALUES(1,0); -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -332,7 +317,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -348,7 +333,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -363,7 +348,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 +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -383,89 +368,81 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem; -Distributed Query into pg_merge_job_570012 +Custom Scan (CitusScan) Executor: 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 (CitusScan) + Output: "?column?", "?column?_1", "?column?_2", worker_column_4 + 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 -- 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 (CitusScan) + Output: l_quantity, worker_column_2 + 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 -- 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 (CitusScan) + 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) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$); t @@ -477,18 +454,16 @@ 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 (CitusScan) + 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) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -497,19 +472,17 @@ 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 (CitusScan) + Executor: 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 +491,39 @@ 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": "CitusScan", + "Parallel Aware": false, + "Distributed Query": { + "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", - "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 +540,41 @@ 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 + CitusScan false - - - Seq Scan - Outer - false - pg_merge_job_570030 - pg_merge_job_570030 - - + + Task-Tracker + + 1 + None, not supported for re-partition queries + + + 1 + 1 + + + 8 + 1 + + + + + + - - - + + + SELECT true AS valid FROM explain_xml($$ SELECT count(*) @@ -635,28 +602,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: 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: "CitusScan" 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: + 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 -- test parallel aggregates SET parallel_setup_cost=0; SET parallel_tuple_cost=0; @@ -672,37 +638,33 @@ 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 (CitusScan) + 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 -- 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 (CitusScan) + 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) 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 +Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) Executor: Router Task Count: 1 Tasks Shown: All @@ -715,22 +677,27 @@ 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 (CitusScan) + 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) -- 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 (CitusScan) (cost=0.00..0.00 rows=0 width=0) + Executor: Router + 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..be57df725 100644 --- a/src/test/regress/expected/multi_explain_0.out +++ b/src/test/regress/expected/multi_explain_0.out @@ -39,81 +39,76 @@ $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 (CitusScan) + 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 -- 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": "CitusScan", + "Distributed Query": { + "Executor": "Real-Time", + "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 +121,65 @@ 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 - - + CitusScan + + 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,107 +190,100 @@ 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: "CitusScan" + Distributed Query: + 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" + -- 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 (CitusScan) + 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 -- 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 (CitusScan) + Output: "?column?", "?column?_1", "?column?_2" + 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 -- 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 (CitusScan) + 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 -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem VALUES(1,0); -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -311,7 +296,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -327,7 +312,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -342,7 +327,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 +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -362,89 +347,81 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem; -Distributed Query into pg_merge_job_570012 +Custom Scan (CitusScan) Executor: 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 (CitusScan) + Output: "?column?", "?column?_1", "?column?_2", worker_column_4 + 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 -- 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 (CitusScan) + Output: l_quantity, worker_column_2 + 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 -- 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 (CitusScan) + 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) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$); t @@ -456,18 +433,16 @@ 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 (CitusScan) + 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) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -476,19 +451,17 @@ 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 (CitusScan) + Executor: 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 +470,36 @@ 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": "CitusScan", + "Distributed Query": { + "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_570024", - "Alias": "pg_merge_job_570024" - } - ] - } - } - ] + } } ] SELECT true AS valid FROM explain_json($$ @@ -546,41 +516,38 @@ 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 + CitusScan + + 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 +575,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: 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: "CitusScan" + Distributed Query: + 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 -- test parallel aggregates SET parallel_setup_cost=0; ERROR: unrecognized configuration parameter "parallel_setup_cost" @@ -643,37 +609,33 @@ 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 (CitusScan) + 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 -- 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 (CitusScan) + 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) 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 +Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) Executor: Router Task Count: 1 Tasks Shown: All @@ -686,22 +648,27 @@ 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 (CitusScan) + 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) -- 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 (CitusScan) (cost=0.00..0.00 rows=0 width=0) + Executor: Router + 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..84b3d55e3 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 otherwhise 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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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..8fe01093c 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 otherwhise 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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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..66bcca2f4 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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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..60caa5865 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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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..75a3c880d 100644 --- a/src/test/regress/expected/multi_mx_explain.out +++ b/src/test/regress/expected/multi_mx_explain.out @@ -61,88 +61,83 @@ $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 (CitusScan) + 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 -- 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": "CitusScan", + "Parallel Aware": false, + "Distributed Query": { + "Executor": "Real-Time", + "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 +151,72 @@ 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 + CitusScan false - - intermediate_column_60130862144_0 - - - - Seq Scan - Outer - false - pg_merge_job_60130862144 - pg_merge_job_60130862144 - - + + 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 + + + + + + + + + + - - - + + + -- Validate XML format SELECT true AS valid FROM explain_xml($$ @@ -235,115 +227,108 @@ 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: "CitusScan" 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: + 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" + -- 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 (CitusScan) + 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 \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 (CitusScan) + Output: "?column?", "?column?_1", "?column?_2" + 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 -- 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 (CitusScan) + 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 -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem_mx VALUES(1,0); -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -356,7 +341,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem_mx SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -370,7 +355,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem_mx WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -383,7 +368,7 @@ Distributed Query -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -403,106 +388,102 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem_mx; -Distributed Query into pg_merge_job_68720796741 +Custom Scan (CitusScan) Executor: 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 (CitusScan) + 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) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); t @@ -514,18 +495,16 @@ 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 (CitusScan) + 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) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -534,25 +513,23 @@ 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 (CitusScan) + Executor: 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 +538,47 @@ 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": "CitusScan", + "Parallel Aware": false, + "Distributed Query": { + "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 + } + ] } - ] - }, - { - "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 +595,49 @@ 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 + CitusScan false - - - Seq Scan - Outer - false - pg_merge_job_68720796765 - pg_merge_job_68720796765 - - + + 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(*) @@ -681,29 +652,28 @@ 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: "CitusScan" 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: + 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 diff --git a/src/test/regress/expected/multi_mx_explain_0.out b/src/test/regress/expected/multi_mx_explain_0.out index 489e4c5ca..07f2c51b2 100644 --- a/src/test/regress/expected/multi_mx_explain_0.out +++ b/src/test/regress/expected/multi_mx_explain_0.out @@ -61,81 +61,76 @@ $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 (CitusScan) + 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 -- 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": "CitusScan", + "Distributed Query": { + "Executor": "Real-Time", + "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 +144,65 @@ 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 - - + CitusScan + + 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,108 +213,101 @@ 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: "CitusScan" + Distributed Query: + 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" + -- 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 (CitusScan) + 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 \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 (CitusScan) + Output: "?column?", "?column?_1", "?column?_2" + 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 -- 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 (CitusScan) + 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 -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem_mx VALUES(1,0); -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -335,7 +320,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem_mx SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -349,7 +334,7 @@ Distributed Query EXPLAIN (COSTS FALSE) DELETE FROM lineitem_mx WHERE l_orderkey = 1 AND l_partkey = 0; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -362,7 +347,7 @@ Distributed Query -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; -Distributed Query +Custom Scan (CitusScan) Executor: Router Task Count: 1 Tasks Shown: All @@ -382,106 +367,102 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem_mx; -Distributed Query into pg_merge_job_68720796741 +Custom Scan (CitusScan) Executor: 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 (CitusScan) + 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) SELECT true AS valid FROM explain_xml($$ SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); t @@ -493,18 +474,16 @@ 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 (CitusScan) + 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) -- Test re-partition join SET citus.large_table_shard_count TO 1; EXPLAIN (COSTS FALSE) @@ -513,25 +492,23 @@ 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 (CitusScan) + Executor: 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 +517,44 @@ 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": "CitusScan", + "Distributed Query": { + "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 + } + ] } - ] - }, - { - "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 +571,46 @@ 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 + CitusScan + + 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 +625,25 @@ 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: "CitusScan" + Distributed Query: + 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 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..401f74d12 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -173,9 +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 + QUERY PLAN +------------------------------------------------------------- + Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) Executor: Task-Tracker Task Count: 4 Tasks Shown: None, not supported for re-partition queries @@ -185,9 +185,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) +(10 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..8179be5eb 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -1489,7 +1489,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_partition_pruning.out b/src/test/regress/expected/multi_partition_pruning.out index 446b67cfc..28ea9dc13 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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index 065b6d19d..585d09c27 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -181,9 +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 + QUERY PLAN +------------------------------------------------------------- + Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) Executor: Task-Tracker Task Count: 4 Tasks Shown: None, not supported for re-partition queries @@ -193,9 +193,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) +(10 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..04e82a6b4 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -2210,12 +2210,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 +2225,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 +2359,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_task_assignment_policy.out b/src/test/regress/expected/multi_task_assignment_policy.out index f571238ad..fcca06718 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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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 (CitusScan) (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/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/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..11d00f8dc 100644 --- a/src/test/regress/output/multi_subquery.source +++ b/src/test/regress/output/multi_subquery.source @@ -764,31 +764,29 @@ 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 (CitusScan) (cost=0.00..0.00 rows=0 width=0) + 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[])) +(20 rows) -- Union and left join subquery pushdown EXPLAIN SELECT @@ -853,52 +851,50 @@ 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 (CitusScan) (cost=0.00..0.00 rows=0 width=0) + 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)) +(41 rows) -- Union, left join and having subquery pushdown EXPLAIN SELECT @@ -1022,37 +1018,35 @@ 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 (CitusScan) (cost=0.00..0.00 rows=0 width=0) + 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))) +(27 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..3abf122b1 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 otherwhise 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..12fc81c8b 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 otherwhise 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_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_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index 5f75b3d28..fdea0f4a6 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -988,14 +988,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 +1053,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; From 1f838199f8786977a1483be6906a3fe320a67fcf Mon Sep 17 00:00:00 2001 From: Metin Doslu Date: Wed, 15 Feb 2017 18:00:54 +0200 Subject: [PATCH 2/2] Use CustomScan API for query execution Custom Scan is a node in the planned statement which helps external providers to abstract data scan not just for foreign data wrappers but also for regular relations so you can benefit your version of caching or hardware optimizations. This sounds like only an abstraction on the data scan layer, but we can use it as an abstraction for our distributed queries. The only thing we need to do is to find distributable parts of the query, plan for them and replace them with a Citus Custom Scan. Then, whenever PostgreSQL hits this custom scan node in its Vulcano style execution, it will call our callback functions which run distributed plan and provides tuples to the upper node as it scans a regular relation. This means fewer code changes, fewer bugs and more supported features for us! First, in the distributed query planner phase, we create a Custom Scan which wraps the distributed plan. For real-time and task-tracker executors, we add this custom plan under the master query plan. For router executor, we directly pass the custom plan because there is not any master query. Then, we simply let the PostgreSQL executor run this plan. When it hits the custom scan node, we call the related executor parts for distributed plan, fill the tuple store in the custom scan and return results to PostgreSQL executor in Vulcano style, a tuple per XXX_ExecScan() call. * Modify planner to utilize Custom Scan node. * Create different scan methods for different executors. * Use native PostgreSQL Explain for master part of queries. --- .../distributed/executor/multi_executor.c | 525 ++++++++++++------ .../executor/multi_router_executor.c | 246 ++++---- .../distributed/planner/multi_explain.c | 43 +- .../planner/multi_master_planner.c | 79 ++- .../planner/multi_physical_planner.c | 7 - .../distributed/planner/multi_planner.c | 358 +++++++----- .../planner/multi_router_planner.c | 16 +- src/backend/distributed/shared_library_init.c | 4 +- .../distributed/utils/citus_outfuncs.c | 1 - .../distributed/utils/citus_readfuncs.c | 1 - src/include/distributed/multi_executor.h | 34 +- .../distributed/multi_master_planner.h | 2 +- .../distributed/multi_physical_planner.h | 4 +- src/include/distributed/multi_planner.h | 7 +- .../distributed/multi_router_executor.h | 13 +- src/test/regress/expected/multi_explain.out | 81 +-- src/test/regress/expected/multi_explain_0.out | 81 +-- .../expected/multi_join_order_additional.out | 50 +- .../expected/multi_join_order_tpch_large.out | 28 +- .../expected/multi_join_order_tpch_small.out | 20 +- .../regress/expected/multi_join_pruning.out | 18 +- .../regress/expected/multi_mx_explain.out | 54 +- .../regress/expected/multi_mx_explain_0.out | 54 +- .../expected/multi_mx_reference_table.out | 6 + .../multi_mx_repartition_udt_prepare.out | 9 +- .../expected/multi_mx_router_planner.out | 11 +- .../expected/multi_mx_schema_support.out | 22 + .../expected/multi_partition_pruning.out | 18 +- .../expected/multi_reference_table.out | 6 + .../expected/multi_repartition_udt.out | 9 +- .../regress/expected/multi_router_planner.out | 11 +- .../regress/expected/multi_schema_support.out | 24 + .../expected/multi_task_assignment_policy.out | 42 +- .../expected/multi_utility_statements.out | 6 + .../regress/multi_task_tracker_extra_schedule | 6 +- src/test/regress/output/multi_subquery.source | 15 +- .../regress/output/multi_subquery_0.source | 191 +++---- .../sql/multi_join_order_additional.sql | 2 +- .../sql/multi_join_order_tpch_large.sql | 2 +- .../regress/sql/multi_mx_reference_table.sql | 1 + .../regress/sql/multi_mx_router_planner.sql | 1 + .../regress/sql/multi_mx_schema_support.sql | 4 + .../regress/sql/multi_reference_table.sql | 1 + src/test/regress/sql/multi_router_planner.sql | 1 + src/test/regress/sql/multi_schema_support.sql | 4 + .../regress/sql/multi_utility_statements.sql | 1 + 46 files changed, 1155 insertions(+), 964 deletions(-) diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 502b8e862..dd2c26733 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -24,7 +24,6 @@ #include "distributed/multi_utility.h" #include "distributed/worker_protocol.h" #include "executor/execdebug.h" -#include "executor/executor.h" #include "commands/copy.h" #include "nodes/makefuncs.h" #include "storage/lmgr.h" @@ -34,212 +33,389 @@ /* - * FIXME: It'd probably be better to have different set of methods for: - * - router readonly queries - * - router modify - * - router insert ... select - * - real-time/task-tracker (no point in seperating those) - * - * I think it's better however to only have one type of CitusScanState, to - * allow to easily share code between routines. + * Define executor methods for the different executor types. */ -static CustomExecMethods CitusCustomExecMethods = { - "CitusScan", - CitusBeginScan, - CitusExecScan, - CitusEndScan, - CitusReScan, -#if (PG_VERSION_NUM >= 90600) - NULL, /* NO EstimateDSMCustomScan callback */ - NULL, /* NO InitializeDSMCustomScan callback */ - NULL, /* NO InitializeWorkerCustomScan callback */ -#endif - NULL, - NULL, - CitusExplainScan +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 * -CitusCreateScan(CustomScan *scan) +RealTimeCreateScan(CustomScan *scan) { CitusScanState *scanState = palloc0(sizeof(CitusScanState)); + scanState->executorType = MULTI_EXECUTOR_REAL_TIME; scanState->customScanState.ss.ps.type = T_CustomScanState; - scanState->customScanState.methods = &CitusCustomExecMethods; scanState->multiPlan = GetMultiPlan(scan); - scanState->executorType = JobExecutorType(scanState->multiPlan); + + scanState->customScanState.methods = &RealTimeCustomExecMethods; return (Node *) scanState; } -void -CitusBeginScan(CustomScanState *node, - EState *estate, - int eflags) +/* + * TaskTrackerCreateScan creates the scan state for task-tracker executor queries. + */ +Node * +TaskTrackerCreateScan(CustomScan *scan) { - CitusScanState *scanState = (CitusScanState *) node; - MultiPlan *multiPlan = scanState->multiPlan; + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); - Assert(IsA(scanState, CustomScanState)); + scanState->executorType = MULTI_EXECUTOR_TASK_TRACKER; + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->multiPlan = GetMultiPlan(scan); - /* ensure plan is executable */ - VerifyMultiPlanValidity(multiPlan); + scanState->customScanState.methods = &TaskTrackerCustomExecMethods; - /* ExecCheckRTPerms(planStatement->rtable, true); */ - - if (scanState->executorType == MULTI_EXECUTOR_ROUTER) - { - RouterBeginScan(scanState); - } + return (Node *) scanState; } -TupleTableSlot * -CitusExecScan(CustomScanState *node) +/* + * RouterCreateScan creates the scan state for router executor queries. + */ +Node * +RouterCreateScan(CustomScan *scan) { - CitusScanState *scanState = (CitusScanState *) node; - MultiPlan *multiPlan = scanState->multiPlan; + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); + MultiPlan *multiPlan = NULL; + Job *workerJob = NULL; + List *taskList = NIL; + bool isModificationQuery = false; - if (scanState->executorType == MULTI_EXECUTOR_ROUTER) + 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) { - return RouterExecScan(scanState); + if (isModificationQuery) + { + scanState->customScanState.methods = &RouterSingleModifyCustomExecMethods; + } + else + { + scanState->customScanState.methods = &RouterSelectCustomExecMethods; + } } else { - TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot; + Assert(isModificationQuery); + scanState->customScanState.methods = &RouterMultiModifyCustomExecMethods; + } - if (!scanState->finishedUnderlyingScan) + return (Node *) scanState; +} + + +/* + * 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 +PrepareMasterJobDirectory(Job *workerJob) +{ + StringInfo jobDirectoryName = MasterJobDirectoryName(workerJob->jobId); + CreateDirectory(jobDirectoryName); + + ResourceOwnerEnlargeJobDirectories(CurrentResourceOwner); + ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId); +} + + +/* + * 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) +{ + 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; + + 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) + { + DefElem *copyOption = makeDefElem("format", (Node *) makeString("binary")); + copyOptions = lappend(copyOptions, copyOption); + } + + foreach(workerTaskCell, workerTaskList) + { + 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) { - Job *workerJob = multiPlan->workerJob; - StringInfo jobDirectoryName = NULL; - EState *executorState = scanState->customScanState.ss.ps.state; - List *workerTaskList = workerJob->taskList; - ListCell *workerTaskCell = NULL; - TupleDesc tupleDescriptor = NULL; - Relation fakeRel = NULL; - MemoryContext executorTupleContext = GetPerTupleMemoryContext(executorState); - ExprContext *executorExpressionContext = - GetPerTupleExprContext(executorState); - uint32 columnCount = 0; - Datum *columnValues = NULL; - bool *columnNulls = NULL; + MemoryContext oldContext = NULL; + bool nextRowFound = false; - /* - * 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); + ResetPerTupleExprContext(executorState); + oldContext = MemoryContextSwitchTo(executorTupleContext); - ResourceOwnerEnlargeJobDirectories(CurrentResourceOwner); - ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId); - - /* pick distributed executor to use */ - if (executorState->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY) + nextRowFound = NextCopyFrom(copyState, executorExpressionContext, + columnValues, columnNulls, NULL); + if (!nextRowFound) { - /* skip distributed query execution for EXPLAIN commands */ - } - else if (scanState->executorType == MULTI_EXECUTOR_REAL_TIME) - { - MultiRealTimeExecute(workerJob); - } - else if (scanState->executorType == MULTI_EXECUTOR_TASK_TRACKER) - { - MultiTaskTrackerExecute(workerJob); + MemoryContextSwitchTo(oldContext); + break; } - tupleDescriptor = node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; - - /* - * Load data, collected by Multi*Execute() above, into a - * tuplestore. For that first create a tuplestore, and then copy - * the files one-by-one. - * - * FIXME: Should probably be in a separate routine. - * - * Long term it'd be a lot better if Multi*Execute() directly - * filled the tuplestores, but that's a fair bit of work. - */ - - /* - * To be able to use copy.c, we need a Relation descriptor. As - * there's no relation corresponding to the data loaded from - * workers, fake one. We just need the bare minimal set of fields - * accessed by BeginCopyFrom(). - * - * FIXME: should be abstracted into a separate function. - */ - fakeRel = palloc0(sizeof(RelationData)); - fakeRel->rd_att = tupleDescriptor; - fakeRel->rd_rel = palloc0(sizeof(FormData_pg_class)); - fakeRel->rd_rel->relkind = RELKIND_RELATION; - - columnCount = tupleDescriptor->natts; - columnValues = palloc0(columnCount * sizeof(Datum)); - columnNulls = palloc0(columnCount * sizeof(bool)); - - Assert(scanState->tuplestorestate == NULL); - scanState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem); - - foreach(workerTaskCell, workerTaskList) - { - Task *workerTask = (Task *) lfirst(workerTaskCell); - StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId); - StringInfo taskFilename = - TaskFilename(jobDirectoryName, workerTask->taskId); - List *copyOptions = NIL; - CopyState copyState = NULL; - - if (BinaryMasterCopyFormat) - { - DefElem *copyOption = makeDefElem("format", - (Node *) makeString("binary")); - copyOptions = lappend(copyOptions, copyOption); - } - copyState = BeginCopyFrom(fakeRel, 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(scanState->tuplestorestate, - tupleDescriptor, - columnValues, columnNulls); - MemoryContextSwitchTo(oldContext); - } - } - - scanState->finishedUnderlyingScan = true; + tuplestore_putvalues(citusScanState->tuplestorestate, tupleDescriptor, + columnValues, columnNulls); + MemoryContextSwitchTo(oldContext); } - if (scanState->tuplestorestate != NULL) - { - Tuplestorestate *tupleStore = scanState->tuplestorestate; - tuplestore_gettupleslot(tupleStore, true, false, resultSlot); - - return resultSlot; - } - - return NULL; + EndCopyFrom(copyState); } } +/* + * 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 CitusEndScan(CustomScanState *node) { @@ -253,17 +429,14 @@ CitusEndScan(CustomScanState *node) } +/* + * 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) { - CitusScanState *scanState = (CitusScanState *) node; - - scanState->tuplestorestate = NULL; - scanState->finishedUnderlyingScan = true; - - /* - * XXX: this probably already works, but if not should be easily - * supportable - probably hard to exercise right now though. - */ - elog(WARNING, "unsupported at this point"); + 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 6233b0d90..9a26090f8 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -77,26 +77,24 @@ static void ReacquireMetadataLocks(List *taskList); static void ExecuteSingleModifyTask(CitusScanState *scanState, Task *task, bool expectResults); static void ExecuteSingleSelectTask(CitusScanState *scanState, Task *task); -static List * GetModifyConnections(List *taskPlacementList, - bool markCritical, +static List * GetModifyConnections(List *taskPlacementList, bool markCritical, bool startedInTransaction); static void ExecuteMultipleTasks(CitusScanState *scanState, List *taskList, bool isModificationQuery, bool expectResults); static int64 ExecuteModifyTasks(List *taskList, bool expectResults, - ParamListInfo paramListInfo, - CitusScanState *scanState, - 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 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(CitusScanState *scanState, MultiConnection *connection, - TupleDesc tupleDescriptor, bool failOnError, int64 *rows); + bool failOnError, int64 *rows); static bool ConsumeQueryResult(MultiConnection *connection, bool failOnError, int64 *rows); @@ -407,9 +405,14 @@ RequiresConsistentSnapshot(Task *task) } +/* + * 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 -RouterBeginScan(CitusScanState *scanState) +CitusModifyBeginScan(CustomScanState *node, EState *estate, int eflags) { + CitusScanState *scanState = (CitusScanState *) node; MultiPlan *multiPlan = scanState->multiPlan; Job *workerJob = multiPlan->workerJob; List *taskList = workerJob->taskList; @@ -428,77 +431,115 @@ RouterBeginScan(CitusScanState *scanState) } +/* + * RouterSingleModifyExecScan executes a single modification query on a + * distributed plan and returns results if there is any. + */ TupleTableSlot * -RouterExecScan(CitusScanState *scanState) +RouterSingleModifyExecScan(CustomScanState *node) { - MultiPlan *multiPlan = scanState->multiPlan; - TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot; + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; - if (!scanState->finishedUnderlyingScan) + if (!scanState->finishedRemoteScan) { + MultiPlan *multiPlan = scanState->multiPlan; + bool hasReturning = multiPlan->hasReturning; Job *workerJob = multiPlan->workerJob; List *taskList = workerJob->taskList; - bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation; - bool isModificationQuery = false; - CmdType operation = multiPlan->operation; + Task *task = (Task *) linitial(taskList); - /* should use IsModificationStmt or such */ - if (operation == CMD_INSERT || operation == CMD_UPDATE || - operation == CMD_DELETE) - { - isModificationQuery = true; - } + 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) - { - bool sendTuples = multiPlan->hasReturning; - ExecuteSingleModifyTask(scanState, task, sendTuples); - } - else - { - ExecuteSingleSelectTask(scanState, task); - } - } - else - { - bool sendTuples = multiPlan->hasReturning; - ExecuteMultipleTasks(scanState, taskList, isModificationQuery, - sendTuples); - } - - /* mark underlying query as having executed */ - scanState->finishedUnderlyingScan = true; + scanState->finishedRemoteScan = true; } - /* if the underlying query produced output, return it */ + resultSlot = ReturnTupleFromTuplestore(scanState); - /* - * FIXME: centralize this into function to be shared between router and - * other executors? - */ - if (scanState->tuplestorestate != NULL) + return resultSlot; +} + + +/* + * ProcessMasterEvaluableFunctions executes evaluable functions and rebuilds + * the query strings in task lists. + */ +static void +ProcessMasterEvaluableFunctions(Job *workerJob) +{ + if (workerJob->requiresMasterEvaluation) { - Tuplestorestate *tupleStore = scanState->tuplestorestate; + Query *jobQuery = workerJob->jobQuery; + List *taskList = workerJob->taskList; - /* XXX: could trivially support backward scans here */ - tuplestore_gettupleslot(tupleStore, true, false, resultSlot); + ExecuteMasterEvaluableFunctions(jobQuery); + RebuildQueryStrings(jobQuery, taskList); + } +} - return resultSlot; + +/* + * 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; } - return NULL; + resultSlot = ReturnTupleFromTuplestore(scanState); + + 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) + { + 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; + } + + resultSlot = ReturnTupleFromTuplestore(scanState); + + return resultSlot; } @@ -512,8 +553,6 @@ RouterExecScan(CitusScanState *scanState) static void ExecuteSingleSelectTask(CitusScanState *scanState, Task *task) { - TupleDesc tupleDescriptor = - scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; ParamListInfo paramListInfo = scanState->customScanState.ss.ps.state->es_param_list_info; List *taskPlacementList = task->taskPlacementList; @@ -547,8 +586,8 @@ ExecuteSingleSelectTask(CitusScanState *scanState, Task *task) continue; } - queryOK = StoreQueryResult(scanState, connection, tupleDescriptor, - dontFailOnError, ¤tAffectedTupleCount); + queryOK = StoreQueryResult(scanState, connection, dontFailOnError, + ¤tAffectedTupleCount); if (queryOK) { return; @@ -569,21 +608,19 @@ ExecuteSingleSelectTask(CitusScanState *scanState, Task *task) * framework), or errors out (failed on all placements). */ static void -ExecuteSingleModifyTask(CitusScanState *scanState, Task *task, - bool expectResults) +ExecuteSingleModifyTask(CitusScanState *scanState, Task *task, bool expectResults) { CmdType operation = scanState->multiPlan->operation; - TupleDesc tupleDescriptor = - scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; EState *executorState = scanState->customScanState.ss.ps.state; ParamListInfo paramListInfo = executorState->es_param_list_info; - bool resultsOK = false; 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 = @@ -669,8 +706,8 @@ ExecuteSingleModifyTask(CitusScanState *scanState, Task *task, */ if (!gotResults && expectResults) { - queryOK = StoreQueryResult(scanState, connection, tupleDescriptor, - failOnError, ¤tAffectedTupleCount); + queryOK = StoreQueryResult(scanState, connection, failOnError, + ¤tAffectedTupleCount); } else { @@ -804,8 +841,6 @@ static void ExecuteMultipleTasks(CitusScanState *scanState, List *taskList, bool isModificationQuery, bool expectResults) { - TupleDesc tupleDescriptor = - scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; EState *executorState = scanState->customScanState.ss.ps.state; ParamListInfo paramListInfo = executorState->es_param_list_info; int64 affectedTupleCount = -1; @@ -813,9 +848,8 @@ ExecuteMultipleTasks(CitusScanState *scanState, List *taskList, /* can only support modifications right now */ Assert(isModificationQuery); - /* XXX: Seems very redundant to pass both scanState and tupleDescriptor */ affectedTupleCount = ExecuteModifyTasks(taskList, expectResults, paramListInfo, - scanState, tupleDescriptor); + scanState); executorState->es_processed = affectedTupleCount; } @@ -831,7 +865,7 @@ ExecuteMultipleTasks(CitusScanState *scanState, List *taskList, int64 ExecuteModifyTasksWithoutResults(List *taskList) { - return ExecuteModifyTasks(taskList, false, NULL, NULL, NULL); + return ExecuteModifyTasks(taskList, false, NULL, NULL); } @@ -845,7 +879,7 @@ ExecuteModifyTasksWithoutResults(List *taskList) */ static int64 ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListInfo, - CitusScanState *scanState, TupleDesc tupleDescriptor) + CitusScanState *scanState) { int64 totalAffectedTupleCount = 0; ListCell *taskCell = NULL; @@ -929,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) @@ -975,10 +1008,10 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn */ if (placementIndex == 0 && expectResults) { - Assert(scanState != NULL && tupleDescriptor != NULL); + Assert(scanState != NULL); - queryOK = StoreQueryResult(scanState, connection, tupleDescriptor, - failOnError, ¤tAffectedTupleCount); + queryOK = StoreQueryResult(scanState, connection, failOnError, + ¤tAffectedTupleCount); } else { @@ -1184,13 +1217,17 @@ ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterT */ static bool StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, - TupleDesc tupleDescriptor, bool failOnError, int64 *rows) + bool failOnError, int64 *rows) { + TupleDesc tupleDescriptor = + scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor; AttInMetadata *attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor); - Tuplestorestate *tupleStore = NULL; 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", @@ -1201,7 +1238,8 @@ StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, if (scanState->tuplestorestate == NULL) { - scanState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem); + scanState->tuplestorestate = + tuplestore_begin_heap(randomAccess, interTransactions, work_mem); } else if (!failOnError) { @@ -1403,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/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index 97ef0ee7b..25fa2ab9c 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -84,12 +84,17 @@ static void ExplainXMLTag(const char *tagname, int flags, ExplainState *es); static void ExplainJSONLineEnding(ExplainState *es); static void ExplainYAMLLineStarting(ExplainState *es); + +/* + * 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 CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es) { CitusScanState *scanState = (CitusScanState *) node; MultiPlan *multiPlan = scanState->multiPlan; - const char *executorName = NULL; if (!ExplainDistributedQueries) { @@ -99,44 +104,8 @@ CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es return; } - /* - * XXX: can we get by without the open/close group somehow - then we'd not - * copy any code from explain.c? Seems unlikely. - */ ExplainOpenGroup("Distributed Query", "Distributed Query", true, es); - /* - * XXX: might be worthwhile to put this somewhere central, e.g. for - * debugging output. - */ - switch (scanState->executorType) - { - case MULTI_EXECUTOR_ROUTER: - { - executorName = "Router"; - } - break; - - case MULTI_EXECUTOR_REAL_TIME: - { - executorName = "Real-Time"; - } - break; - - case MULTI_EXECUTOR_TASK_TRACKER: - { - executorName = "Task-Tracker"; - } - break; - - default: - { - executorName = "Other"; - } - break; - } - ExplainPropertyText("Executor", executorName, es); - ExplainJob(multiPlan->workerJob, es); 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 facff1f9d..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" @@ -34,7 +35,7 @@ * a target target list for the master node. This master target list keeps the * temporary table's columns on the master node. */ -List * +static List * MasterTargetList(List *workerTargetList) { List *masterTargetList = NIL; @@ -164,67 +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, CustomScan *dataScan) +BuildSelectStatement(Query *masterQuery, List *masterTargetList, CustomScan *remoteScan) { PlannedStmt *selectStatement = NULL; - RangeTblEntry *rangeTableEntry = NULL; - RangeTblEntry *queryRangeTableEntry = NULL; + RangeTblEntry *customScanRangeTableEntry = NULL; Agg *aggregationPlan = NULL; Plan *topLevelPlan = NULL; - ListCell *lc = NULL; - List *columnNames = NULL; - - /* (0) compute column names */ - foreach(lc, masterTargetList) - { - TargetEntry *te = lfirst(lc); - columnNames = lappend(columnNames, makeString(te->resname)); - } + 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_VALUES; /* can't look up relation */ - rangeTableEntry->eref = makeAlias("remote scan", columnNames); - 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 */ - /* Gone */ - - /* (3) add an aggregation plan if needed */ + /* (2) add an aggregation plan if needed */ if (masterQuery->hasAggs || masterQuery->groupClause) { - dataScan->scan.plan.targetlist = masterTargetList; + remoteScan->scan.plan.targetlist = masterTargetList; - aggregationPlan = BuildAggregatePlan(masterQuery, &dataScan->scan.plan); + aggregationPlan = BuildAggregatePlan(masterQuery, &remoteScan->scan.plan); topLevelPlan = (Plan *) aggregationPlan; } else { /* otherwise set the final projections on the scan plan directly */ - dataScan->scan.plan.targetlist = masterQuery->targetList; - topLevelPlan = &dataScan->scan.plan; + 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; @@ -242,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; @@ -259,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; @@ -267,24 +258,24 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName, /* - * 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, CustomScan *dataScan) +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, dataScan); + masterSelectPlan = BuildSelectStatement(masterQuery, masterTargetList, remoteScan); return masterSelectPlan; } diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index a4a87c372..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,7 +213,6 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) multiPlan = CitusMakeNode(MultiPlan); multiPlan->workerJob = workerJob; multiPlan->masterQuery = masterQuery; - multiPlan->masterTableName = jobSchemaName->data; multiPlan->routerExecutable = MultiPlanRouterExecutable(multiPlan); multiPlan->operation = CMD_SELECT; diff --git a/src/backend/distributed/planner/multi_planner.c b/src/backend/distributed/planner/multi_planner.c index bc0c1335f..7839c7cfc 100644 --- a/src/backend/distributed/planner/multi_planner.c +++ b/src/backend/distributed/planner/multi_planner.c @@ -23,30 +23,48 @@ #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 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); @@ -144,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; } @@ -274,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,12 +311,6 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query } -static CustomScanMethods CitusCustomScanMethods = { - "CitusScan", - CitusCreateScan -}; - - /* * GetMultiPlan returns the associated MultiPlan for a CustomScan. */ @@ -308,41 +319,34 @@ GetMultiPlan(CustomScan *customScan) { MultiPlan *multiPlan = NULL; - Assert(IsA(customScan, CustomScan)); - Assert(customScan->methods == &CitusCustomScanMethods); Assert(list_length(customScan->custom_private) == 1); - multiPlan = DeSerializeMultiPlan(linitial(customScan->custom_private)); + multiPlan = DeserializeMultiPlan(linitial(customScan->custom_private)); return multiPlan; } -/* Does the passed in statement require distributed execution? */ -bool -HasCitusToplevelNode(PlannedStmt *result) +/* + * 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) { - elog(ERROR, "gone"); -} - - -Node * -SerializableMultiPlan(MultiPlan *multiPlan) -{ - /* - * FIXME: 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. - */ - char *serializedPlan = NULL; + char *serializedMultiPlan = NULL; Const *multiPlanData = NULL; - serializedPlan = CitusNodeToString(multiPlan); + serializedMultiPlan = CitusNodeToString(multiPlan); multiPlanData = makeNode(Const); multiPlanData->consttype = CSTRINGOID; - multiPlanData->constlen = strlen(serializedPlan); - multiPlanData->constvalue = CStringGetDatum(serializedPlan); + multiPlanData->constlen = strlen(serializedMultiPlan); + multiPlanData->constvalue = CStringGetDatum(serializedMultiPlan); multiPlanData->constbyval = false; multiPlanData->location = -1; @@ -350,8 +354,12 @@ SerializableMultiPlan(MultiPlan *multiPlan) } -MultiPlan * -DeSerializeMultiPlan(Node *node) +/* + * 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; @@ -369,107 +377,171 @@ DeSerializeMultiPlan(Node *node) /* - * 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. - * - * FIXME - * - * 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. + * FinalizePlan combines local plan with distributed plan and creates a plan + * which can be run by the PostgreSQL executor. */ -PlannedStmt * -MultiQueryContainerNode(PlannedStmt *originalPlan, MultiPlan *multiPlan) +static PlannedStmt * +FinalizePlan(PlannedStmt *localPlan, MultiPlan *multiPlan) { - PlannedStmt *resultPlan = NULL; + PlannedStmt *finalPlan = NULL; CustomScan *customScan = makeNode(CustomScan); - Node *multiPlanData = SerializableMultiPlan(multiPlan); + Node *multiPlanData = NULL; + MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST; - customScan->methods = &CitusCustomScanMethods; - customScan->custom_private = list_make1(multiPlanData); - - /* FIXME: This probably ain't correct */ - if (ExecSupportsBackwardScan(originalPlan->planTree)) + if (!multiPlan->planningError) { - customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN; + executorType = JobExecutorType(multiPlan); } - /* - * FIXME: these two branches/pieces of code should probably be moved into - * router / logical planner code respectively. - */ + switch (executorType) + { + 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) { - resultPlan = MasterNodeSelectPlan(multiPlan, customScan); - resultPlan->queryId = originalPlan->queryId; - resultPlan->utilityStmt = originalPlan->utilityStmt; + finalPlan = FinalizeNonRouterPlan(localPlan, multiPlan, customScan); } else { - ListCell *lc = NULL; - List *targetList = NIL; - bool foundJunk = false; - RangeTblEntry *rangeTableEntry = NULL; - List *columnNames = NIL; - int newRTI = list_length(originalPlan->rtable) + 1; - - /* - * XXX: This basically just builds a targetlist to "read" from the - * custom scan output. - */ - foreach(lc, originalPlan->planTree->targetlist) - { - TargetEntry *te = lfirst(lc); - Var *newVar = NULL; - TargetEntry *newTargetEntry = NULL; - - Assert(IsA(te, TargetEntry)); - - /* - * XXX: I can't think of a case where we'd need resjunk stuff at - * the toplevel of a router query - all things needing it have - * been pushed down. - */ - if (te->resjunk) - { - foundJunk = true; - continue; - } - - if (foundJunk) - { - ereport(ERROR, (errmsg("unexpected !junk entry after resjunk entry"))); - } - - /* build TE pointing to custom scan */ - newVar = makeVarFromTargetEntry(newRTI, te); - newTargetEntry = flatCopyTargetEntry(te); - newTargetEntry->expr = (Expr *) newVar; - targetList = lappend(targetList, newTargetEntry); - - columnNames = lappend(columnNames, makeString(te->resname)); - } - - /* XXX: can't think of a better RTE type than VALUES */ - rangeTableEntry = makeNode(RangeTblEntry); - rangeTableEntry->rtekind = RTE_VALUES; /* can't look up relation */ - rangeTableEntry->eref = makeAlias("remote_scan", columnNames); - rangeTableEntry->inh = false; - rangeTableEntry->inFromCl = true; - - resultPlan = originalPlan; - resultPlan->planTree = (Plan *) customScan; - resultPlan->rtable = lappend(resultPlan->rtable, rangeTableEntry); - customScan->scan.plan.targetlist = targetList; + finalPlan = FinalizeRouterPlan(localPlan, customScan); } - return resultPlan; + return finalPlan; +} + + +/* + * 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. + */ +static PlannedStmt * +FinalizeNonRouterPlan(PlannedStmt *localPlan, MultiPlan *multiPlan, + CustomScan *customScan) +{ + PlannedStmt *finalPlan = NULL; + + finalPlan = MasterNodeSelectPlan(multiPlan, customScan); + finalPlan->queryId = localPlan->queryId; + finalPlan->utilityStmt = localPlan->utilityStmt; + + return finalPlan; +} + + +/* + * 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 PlannedStmt * +FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan) +{ + PlannedStmt *routerPlan = NULL; + RangeTblEntry *remoteScanRangeTableEntry = NULL; + ListCell *targetEntryCell = NULL; + List *targetList = NIL; + List *columnNameList = NIL; + + /* 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) + { + 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); + } + + customScan->scan.plan.targetlist = targetList; + + routerPlan = makeNode(PlannedStmt); + routerPlan->planTree = (Plan *) customScan; + + remoteScanRangeTableEntry = RemoteScanRangeTableEntry(columnNameList); + routerPlan->rtable = list_make1(remoteScanRangeTableEntry); + + routerPlan->canSetTag = true; + routerPlan->relationOids = NIL; + + routerPlan->queryId = localPlan->queryId; + routerPlan->utilityStmt = localPlan->utilityStmt; + routerPlan->commandType = localPlan->commandType; + routerPlan->hasReturning = localPlan->hasReturning; + + return routerPlan; +} + + +/* + * RemoteScanRangeTableEntry creates a range table entry from given column name + * list to represent a remote scan. + */ +RangeTblEntry * +RemoteScanRangeTableEntry(List *columnNameList) +{ + RangeTblEntry *remoteScanRangeTableEntry = makeNode(RangeTblEntry); + + /* 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 11212ec81..39bf01782 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -236,9 +236,13 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query, multiPlan->workerJob = job; multiPlan->masterQuery = NULL; - multiPlan->masterTableName = NULL; multiPlan->routerExecutable = true; - multiPlan->hasReturning = list_length(originalQuery->returningList) > 0; + multiPlan->hasReturning = false; + + if (list_length(originalQuery->returningList) > 0) + { + multiPlan->hasReturning = true; + } return multiPlan; } @@ -321,10 +325,14 @@ CreateInsertSelectRouterPlan(Query *originalQuery, /* and finally the multi plan */ multiPlan->workerJob = workerJob; - multiPlan->masterTableName = NULL; multiPlan->masterQuery = NULL; multiPlan->routerExecutable = true; - multiPlan->hasReturning = list_length(originalQuery->returningList) > 0; + 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 05f92f486..689296b08 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -114,9 +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 || - 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 " diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 8da52e506..8aa485857 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -281,7 +281,6 @@ OutMultiPlan(OUTFUNC_ARGS) 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 fca056e4e..37c9245fa 100644 --- a/src/backend/distributed/utils/citus_readfuncs.c +++ b/src/backend/distributed/utils/citus_readfuncs.c @@ -188,7 +188,6 @@ ReadMultiPlan(READFUNC_ARGS) 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 53fc327b9..e3b53a327 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -17,9 +17,6 @@ #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 @@ -30,23 +27,26 @@ typedef struct CitusScanState { - CustomScanState customScanState; - MultiPlan *multiPlan; - MultiExecutorType executorType; - - /* state for router */ - bool finishedUnderlyingScan; - Tuplestorestate *tuplestorestate; + 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; -Node * CitusCreateScan(CustomScan *scan); -extern void CitusBeginScan(CustomScanState *node, - EState *estate, - int eflags); -extern TupleTableSlot * CitusExecScan(CustomScanState *node); + +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 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_master_planner.h b/src/include/distributed/multi_master_planner.h index c58b42717..9873ce4ff 100644 --- a/src/include/distributed/multi_master_planner.h +++ b/src/include/distributed/multi_master_planner.h @@ -24,6 +24,6 @@ struct MultiPlan; struct CustomScan; extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan, struct CustomScan *dataScan); -extern List * MasterTargetList(List *workerTargetList); + #endif /* MULTI_MASTER_PLANNER_H */ diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index ccf3b44a2..0cf340899 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -213,13 +213,11 @@ typedef struct JoinSequenceNode typedef struct MultiPlan { CitusNode type; - CmdType operation; - bool hasReturning; + 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 e2f255082..8a24b2efa 100644 --- a/src/include/distributed/multi_planner.h +++ b/src/include/distributed/multi_planner.h @@ -51,14 +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(CustomScan *node); -extern Node * SerializableMultiPlan(struct MultiPlan *multiPlan); -extern struct MultiPlan * DeSerializeMultiPlan(Node *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 b42f6e002..8c9eafb7d 100644 --- a/src/include/distributed/multi_router_executor.h +++ b/src/include/distributed/multi_router_executor.h @@ -35,15 +35,12 @@ typedef struct XactShardConnSet extern bool AllModificationsCommutative; extern bool EnableDeadlockPrevention; -extern void RouterBeginScan(CitusScanState *scanState); - -extern TupleTableSlot * RouterExecScan(CitusScanState *scanState); - -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 cffad3c26..e2ef27342 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -43,8 +43,7 @@ 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 (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -74,10 +73,9 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Real-Time", "Parallel Aware": false, "Distributed Query": { - "Executor": "Real-Time", "Job": { "Task Count": 8, "Tasks Shown": "One of 8", @@ -150,10 +148,9 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Real-Time false - Real-Time 8 One of 8 @@ -221,10 +218,9 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Real-Time" Parallel Aware: false Distributed Query: - Executor: "Real-Time" Job: Task Count: 8 Tasks Shown: "One of 8" @@ -253,8 +249,7 @@ 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 (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -267,9 +262,8 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem; Aggregate Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2"))) - -> Custom Scan (CitusScan) + -> Custom Scan (Citus Real-Time) Output: "?column?", "?column?_1", "?column?_2" - Executor: Real-Time Task Count: 8 Tasks Shown: One of 8 -> Task @@ -286,8 +280,7 @@ EXPLAIN (COSTS FALSE) Limit -> Sort Sort Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -304,8 +297,7 @@ Limit -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem VALUES(1,0); -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -317,8 +309,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -333,8 +324,7 @@ Custom Scan (CitusScan) EXPLAIN (COSTS FALSE) DELETE FROM lineitem WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -348,8 +338,7 @@ Custom Scan (CitusScan) -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -368,8 +357,7 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem; -Custom Scan (CitusScan) - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -381,10 +369,9 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) HAVING sum(l_quantity) > 100; Aggregate Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2"))) - Filter: (sum("remote scan".worker_column_4) > '100'::numeric) - -> Custom Scan (CitusScan) + Filter: (sum(remote_scan.worker_column_4) > '100'::numeric) + -> Custom Scan (Citus Real-Time) Output: "?column?", "?column?_1", "?column?_2", worker_column_4 - Executor: Real-Time Task Count: 8 Tasks Shown: One of 8 -> Task @@ -400,11 +387,10 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) HAVING l_quantity > (100 * random()); HashAggregate Output: l_quantity - Group Key: "remote scan".l_quantity - Filter: (("remote scan".worker_column_2)::double precision > ('100'::double precision * random())) - -> Custom Scan (CitusScan) + 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 - Executor: Real-Time Task Count: 8 Tasks Shown: One of 8 -> Task @@ -419,8 +405,7 @@ SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 4 Tasks Shown: All -> Task @@ -455,8 +440,7 @@ SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -473,8 +457,7 @@ EXPLAIN (COSTS FALSE) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 1 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -500,10 +483,9 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Task-Tracker", "Parallel Aware": false, "Distributed Query": { - "Executor": "Task-Tracker", "Job": { "Task Count": 1, "Tasks Shown": "None, not supported for re-partition queries", @@ -550,10 +532,9 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Task-Tracker false - Task-Tracker 1 None, not supported for re-partition queries @@ -610,10 +591,9 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Task-Tracker" Parallel Aware: false Distributed Query: - Executor: "Task-Tracker" Job: Task Count: 1 Tasks Shown: "None, not supported for re-partition queries" @@ -639,8 +619,7 @@ Finalize Aggregate -- ensure distributed plans don't break EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -652,8 +631,7 @@ PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -664,8 +642,7 @@ Aggregate 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; -Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Router +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) Task Count: 1 Tasks Shown: All -> Task @@ -678,8 +655,7 @@ PREPARE real_time_executor_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query; Aggregate - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -691,8 +667,7 @@ Aggregate -- 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); -Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Router +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) Task Count: 1 Tasks Shown: All -> Task diff --git a/src/test/regress/expected/multi_explain_0.out b/src/test/regress/expected/multi_explain_0.out index be57df725..af839b514 100644 --- a/src/test/regress/expected/multi_explain_0.out +++ b/src/test/regress/expected/multi_explain_0.out @@ -43,8 +43,7 @@ Sort Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity -> HashAggregate Group Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -71,9 +70,8 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Real-Time", "Distributed Query": { - "Executor": "Real-Time", "Job": { "Task Count": 8, "Tasks Shown": "One of 8", @@ -140,9 +138,8 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Real-Time - Real-Time 8 One of 8 @@ -204,9 +201,8 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Real-Time" Distributed Query: - Executor: "Real-Time" Job: Task Count: 8 Tasks Shown: "One of 8" @@ -232,8 +228,7 @@ Sort Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity -> HashAggregate Group Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -246,9 +241,8 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem; Aggregate Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2"))) - -> Custom Scan (CitusScan) + -> Custom Scan (Citus Real-Time) Output: "?column?", "?column?_1", "?column?_2" - Executor: Real-Time Task Count: 8 Tasks Shown: One of 8 -> Task @@ -265,8 +259,7 @@ EXPLAIN (COSTS FALSE) Limit -> Sort Sort Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -283,8 +276,7 @@ Limit -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem VALUES(1,0); -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -296,8 +288,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -312,8 +303,7 @@ Custom Scan (CitusScan) EXPLAIN (COSTS FALSE) DELETE FROM lineitem WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -327,8 +317,7 @@ Custom Scan (CitusScan) -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -347,8 +336,7 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem; -Custom Scan (CitusScan) - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -360,10 +348,9 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) HAVING sum(l_quantity) > 100; Aggregate Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2"))) - Filter: (sum("remote scan".worker_column_4) > '100'::numeric) - -> Custom Scan (CitusScan) + Filter: (sum(remote_scan.worker_column_4) > '100'::numeric) + -> Custom Scan (Citus Real-Time) Output: "?column?", "?column?_1", "?column?_2", worker_column_4 - Executor: Real-Time Task Count: 8 Tasks Shown: One of 8 -> Task @@ -379,11 +366,10 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) HAVING l_quantity > (100 * random()); HashAggregate Output: l_quantity - Group Key: "remote scan".l_quantity - Filter: (("remote scan".worker_column_2)::double precision > ('100'::double precision * random())) - -> Custom Scan (CitusScan) + 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 - Executor: Real-Time Task Count: 8 Tasks Shown: One of 8 -> Task @@ -398,8 +384,7 @@ SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 4 Tasks Shown: All -> Task @@ -434,8 +419,7 @@ SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -452,8 +436,7 @@ EXPLAIN (COSTS FALSE) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 1 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -477,9 +460,8 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Task-Tracker", "Distributed Query": { - "Executor": "Task-Tracker", "Job": { "Task Count": 1, "Tasks Shown": "None, not supported for re-partition queries", @@ -524,9 +506,8 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Task-Tracker - Task-Tracker 1 None, not supported for re-partition queries @@ -581,9 +562,8 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Task-Tracker" Distributed Query: - Executor: "Task-Tracker" Job: Task Count: 1 Tasks Shown: "None, not supported for re-partition queries" @@ -610,8 +590,7 @@ Aggregate -- ensure distributed plans don't break EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 8 Tasks Shown: One of 8 -> Task @@ -623,8 +602,7 @@ PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -635,8 +613,7 @@ Aggregate 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; -Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Router +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) Task Count: 1 Tasks Shown: All -> Task @@ -649,8 +626,7 @@ PREPARE real_time_executor_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query; Aggregate - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -662,8 +638,7 @@ Aggregate -- 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); -Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Router +Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0) Task Count: 1 Tasks Shown: All -> Task diff --git a/src/test/regress/expected/multi_join_order_additional.out b/src/test/regress/expected/multi_join_order_additional.out index 84b3d55e3..da8ddd2f5 100644 --- a/src/test/regress/expected/multi_join_order_additional.out +++ b/src/test/regress/expected/multi_join_order_additional.out @@ -6,7 +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 otherwhise +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. @@ -140,9 +140,9 @@ 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 --------------------------------------------------------------- - Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) + 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) @@ -156,10 +156,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -177,10 +177,10 @@ BEGIN; EXPLAIN SELECT count(*) FROM orders, lineitem_hash WHERE o_orderkey = l_orderkey; LOG: join order: [ "orders" ][ single partition join "lineitem_hash" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -188,10 +188,10 @@ LOG: join order: [ "orders" ][ single partition join "lineitem_hash" ] 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 --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -199,10 +199,10 @@ LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ] EXPLAIN SELECT count(*) FROM customer_hash, nation WHERE c_nationkey = n_nationkey; LOG: join order: [ "customer_hash" ][ broadcast join "nation" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -213,10 +213,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -225,10 +225,10 @@ LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition EXPLAIN SELECT count(*) FROM orders, customer_hash WHERE c_custkey = o_custkey; LOG: join order: [ "orders" ][ dual partition join "customer_hash" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -237,10 +237,10 @@ LOG: join order: [ "orders" ][ dual partition join "customer_hash" ] EXPLAIN SELECT count(*) FROM orders_hash, customer WHERE c_custkey = o_custkey; LOG: join order: [ "orders_hash" ][ single partition join "customer" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) 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 8fe01093c..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,7 +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 otherwhise +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 @@ -24,10 +24,10 @@ WHERE and l_discount between 0.06 - 0.01 and 0.06 + 0.01 and l_quantity < 24; LOG: join order: [ "lineitem" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -55,13 +55,13 @@ ORDER BY revenue DESC, o_orderdate; LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ] - QUERY PLAN --------------------------------------------------------------------------- + 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 (CitusScan) (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 (6 rows) @@ -104,7 +104,7 @@ LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partit 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 (CitusScan) (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 (6 rows) @@ -139,10 +139,10 @@ WHERE AND l_shipinstruct = 'DELIVER IN PERSON' ); LOG: join order: [ "lineitem" ][ single partition join "part" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -158,11 +158,11 @@ WHERE GROUP BY l_partkey; LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single partition join "part" ][ single partition join "customer" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------- HashAggregate (cost=0.00..0.00 rows=0 width=0) Group Key: l_partkey - -> Custom Scan (CitusScan) (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 (4 rows) 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 66bcca2f4..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,10 @@ WHERE and l_discount between 0.06 - 0.01 and 0.06 + 0.01 and l_quantity < 24; LOG: join order: [ "lineitem" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -49,13 +49,13 @@ ORDER BY revenue DESC, o_orderdate; LOG: join order: [ "orders" ][ broadcast join "customer" ][ local partition join "lineitem" ] - QUERY PLAN --------------------------------------------------------------------------- + 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 (CitusScan) (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 (6 rows) @@ -98,7 +98,7 @@ LOG: join order: [ "orders" ][ broadcast join "customer" ][ broadcast join "nat 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 (CitusScan) (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 (6 rows) @@ -133,10 +133,10 @@ WHERE AND l_shipinstruct = 'DELIVER IN PERSON' ); LOG: join order: [ "lineitem" ][ broadcast join "part" ] - QUERY PLAN --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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_join_pruning.out b/src/test/regress/expected/multi_join_pruning.out index 60caa5865..e04799723 100644 --- a/src/test/regress/expected/multi_join_pruning.out +++ b/src/test/regress/expected/multi_join_pruning.out @@ -100,10 +100,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -112,10 +112,10 @@ EXPLAIN SELECT count(*) 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -125,10 +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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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 75a3c880d..02dfbee99 100644 --- a/src/test/regress/expected/multi_mx_explain.out +++ b/src/test/regress/expected/multi_mx_explain.out @@ -65,8 +65,7 @@ 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 (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -96,10 +95,9 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Real-Time", "Parallel Aware": false, "Distributed Query": { - "Executor": "Real-Time", "Job": { "Task Count": 16, "Tasks Shown": "One of 16", @@ -173,10 +171,9 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Real-Time false - Real-Time 16 One of 16 @@ -244,10 +241,9 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Real-Time" Parallel Aware: false Distributed Query: - Executor: "Real-Time" Job: Task Count: 16 Tasks Shown: "One of 16" @@ -276,8 +272,7 @@ 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 (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -291,9 +286,8 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx; Aggregate Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2"))) - -> Custom Scan (CitusScan) + -> Custom Scan (Citus Real-Time) Output: "?column?", "?column?_1", "?column?_2" - Executor: Real-Time Task Count: 16 Tasks Shown: One of 16 -> Task @@ -310,8 +304,7 @@ EXPLAIN (COSTS FALSE) Limit -> Sort Sort Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -328,8 +321,7 @@ Limit -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem_mx VALUES(1,0); -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -341,8 +333,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem_mx SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -355,8 +346,7 @@ Custom Scan (CitusScan) EXPLAIN (COSTS FALSE) DELETE FROM lineitem_mx WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -368,8 +358,7 @@ Custom Scan (CitusScan) -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -388,8 +377,7 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem_mx; -Custom Scan (CitusScan) - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -400,8 +388,7 @@ SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: All -> Task @@ -496,8 +483,7 @@ SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -514,8 +500,7 @@ EXPLAIN (COSTS FALSE) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -547,10 +532,9 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Task-Tracker", "Parallel Aware": false, "Distributed Query": { - "Executor": "Task-Tracker", "Job": { "Task Count": 4, "Tasks Shown": "None, not supported for re-partition queries", @@ -605,10 +589,9 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Task-Tracker false - Task-Tracker 4 None, not supported for re-partition queries @@ -660,10 +643,9 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Task-Tracker" Parallel Aware: false Distributed Query: - Executor: "Task-Tracker" Job: Task Count: 4 Tasks Shown: "None, not supported for re-partition queries" diff --git a/src/test/regress/expected/multi_mx_explain_0.out b/src/test/regress/expected/multi_mx_explain_0.out index 07f2c51b2..0a74d3001 100644 --- a/src/test/regress/expected/multi_mx_explain_0.out +++ b/src/test/regress/expected/multi_mx_explain_0.out @@ -65,8 +65,7 @@ Sort Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity -> HashAggregate Group Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -93,9 +92,8 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Real-Time", "Distributed Query": { - "Executor": "Real-Time", "Job": { "Task Count": 16, "Tasks Shown": "One of 16", @@ -163,9 +161,8 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Real-Time - Real-Time 16 One of 16 @@ -227,9 +224,8 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Real-Time" Distributed Query: - Executor: "Real-Time" Job: Task Count: 16 Tasks Shown: "One of 16" @@ -255,8 +251,7 @@ Sort Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity -> HashAggregate Group Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -270,9 +265,8 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx; Aggregate Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2"))) - -> Custom Scan (CitusScan) + -> Custom Scan (Citus Real-Time) Output: "?column?", "?column?_1", "?column?_2" - Executor: Real-Time Task Count: 16 Tasks Shown: One of 16 -> Task @@ -289,8 +283,7 @@ EXPLAIN (COSTS FALSE) Limit -> Sort Sort Key: l_quantity - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -307,8 +300,7 @@ Limit -- Test insert EXPLAIN (COSTS FALSE) INSERT INTO lineitem_mx VALUES(1,0); -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -320,8 +312,7 @@ EXPLAIN (COSTS FALSE) UPDATE lineitem_mx SET l_suppkey = 12 WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -334,8 +325,7 @@ Custom Scan (CitusScan) EXPLAIN (COSTS FALSE) DELETE FROM lineitem_mx WHERE l_orderkey = 1 AND l_partkey = 0; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -347,8 +337,7 @@ Custom Scan (CitusScan) -- Test single-shard SELECT EXPLAIN (COSTS FALSE) SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; -Custom Scan (CitusScan) - Executor: Router +Custom Scan (Citus Router) Task Count: 1 Tasks Shown: All -> Task @@ -367,8 +356,7 @@ t EXPLAIN (COSTS FALSE) CREATE TABLE explain_result AS SELECT * FROM lineitem_mx; -Custom Scan (CitusScan) - Executor: Real-Time +Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -379,8 +367,7 @@ SET citus.explain_all_tasks TO on; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) Task Count: 16 Tasks Shown: All -> Task @@ -475,8 +462,7 @@ SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -493,8 +479,7 @@ EXPLAIN (COSTS FALSE) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; Aggregate - -> Custom Scan (CitusScan) - Executor: Task-Tracker + -> Custom Scan (Citus Task-Tracker) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -524,9 +509,8 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "CitusScan", + "Custom Plan Provider": "Citus Task-Tracker", "Distributed Query": { - "Executor": "Task-Tracker", "Job": { "Task Count": 4, "Tasks Shown": "None, not supported for re-partition queries", @@ -579,9 +563,8 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - CitusScan + Citus Task-Tracker - Task-Tracker 4 None, not supported for re-partition queries @@ -631,9 +614,8 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "CitusScan" + Custom Plan Provider: "Citus Task-Tracker" Distributed Query: - Executor: "Task-Tracker" Job: Task Count: 4 Tasks Shown: "None, not supported for re-partition queries" 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 401f74d12..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 -------------------------------------------------------------- - Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - 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,7 +184,7 @@ LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_ot -> MapMergeJob Map Task Count: 5 Merge Task Count: 4 -(10 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 8179be5eb..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 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 28ea9dc13..65034ae48 100644 --- a/src/test/regress/expected/multi_partition_pruning.out +++ b/src/test/regress/expected/multi_partition_pruning.out @@ -170,30 +170,30 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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_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 585d09c27..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 -------------------------------------------------------------- - Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - 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,7 +192,7 @@ LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_ot -> MapMergeJob Map Task Count: 5 Merge Task Count: 4 -(10 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 04e82a6b4..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 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 fcca06718..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,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -74,10 +74,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -93,10 +93,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -107,10 +107,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -140,10 +140,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -154,10 +154,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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) @@ -168,10 +168,10 @@ 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 --------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (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_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/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_subquery.source b/src/test/regress/output/multi_subquery.source index 11d00f8dc..4c5931f17 100644 --- a/src/test/regress/output/multi_subquery.source +++ b/src/test/regress/output/multi_subquery.source @@ -767,8 +767,7 @@ FROM QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Aggregate (cost=0.00..0.00 rows=0 width=0) - -> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) Task Count: 2 Tasks Shown: One of 2 -> Task @@ -786,7 +785,7 @@ FROM 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[])) -(20 rows) +(19 rows) -- Union and left join subquery pushdown EXPLAIN SELECT @@ -855,8 +854,7 @@ GROUP BY ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- HashAggregate (cost=0.00..0.00 rows=0 width=0) Group Key: hasdone - -> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) Task Count: 2 Tasks Shown: One of 2 -> Task @@ -894,7 +892,7 @@ GROUP BY 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)) -(41 rows) +(40 rows) -- Union, left join and having subquery pushdown EXPLAIN SELECT @@ -1023,8 +1021,7 @@ LIMIT 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 (CitusScan) (cost=0.00..0.00 rows=0 width=0) - Executor: Real-Time + -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) Task Count: 2 Tasks Shown: One of 2 -> Task @@ -1047,6 +1044,6 @@ LIMIT 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))) -(27 rows) +(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 3abf122b1..18d307017 100644 --- a/src/test/regress/sql/multi_join_order_additional.sql +++ b/src/test/regress/sql/multi_join_order_additional.sql @@ -11,7 +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 otherwhise +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 12fc81c8b..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,7 +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 otherwhise +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_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 fdea0f4a6..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 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;