Initial temp table removal implementation

pull/1185/head
Andres Freund 2017-01-31 20:28:22 -08:00 committed by Metin Doslu
parent 6f4886cd11
commit 52358fe891
39 changed files with 2553 additions and 2990 deletions

View File

@ -24,63 +24,107 @@
#include "distributed/multi_utility.h" #include "distributed/multi_utility.h"
#include "distributed/worker_protocol.h" #include "distributed/worker_protocol.h"
#include "executor/execdebug.h" #include "executor/execdebug.h"
#include "executor/executor.h"
#include "commands/copy.h"
#include "nodes/makefuncs.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "tcop/utility.h" #include "tcop/utility.h"
#include "utils/snapmgr.h" #include "utils/snapmgr.h"
#include "utils/memutils.h"
static void CopyQueryResults(List *masterCopyStmtList);
/* /*
* multi_ExecutorStart is a hook called at at the beginning of any execution * FIXME: It'd probably be better to have different set of methods for:
* of any query plan. * - 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 * I think it's better however to only have one type of CitusScanState, to
* checks. If a legal statement, start the distributed execution. After that * allow to easily share code between routines.
* the to-be-executed query is replaced with the portion executing solely on
* the master.
*/ */
void static CustomExecMethods CitusCustomExecMethods = {
multi_ExecutorStart(QueryDesc *queryDesc, int eflags) "CitusScan",
{ CitusBeginScan,
PlannedStmt *planStatement = queryDesc->plannedstmt; CitusExecScan,
CitusEndScan,
CitusReScan,
#if (PG_VERSION_NUM >= 90600)
NULL, /* NO EstimateDSMCustomScan callback */
NULL, /* NO InitializeDSMCustomScan callback */
NULL, /* NO InitializeWorkerCustomScan callback */
#endif
NULL,
NULL,
CitusExplainScan
};
if (HasCitusToplevelNode(planStatement))
{ Node *
MultiPlan *multiPlan = GetMultiPlan(planStatement); CitusCreateScan(CustomScan *scan)
MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST; {
Job *workerJob = multiPlan->workerJob; CitusScanState *scanState = palloc0(sizeof(CitusScanState));
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 */ /* ensure plan is executable */
VerifyMultiPlanValidity(multiPlan); VerifyMultiPlanValidity(multiPlan);
ExecCheckRTPerms(planStatement->rtable, true); /* ExecCheckRTPerms(planStatement->rtable, true); */
executorType = JobExecutorType(multiPlan); if (scanState->executorType == MULTI_EXECUTOR_ROUTER)
if (executorType == MULTI_EXECUTOR_ROUTER)
{ {
List *taskList = workerJob->taskList; RouterBeginScan(scanState);
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 */ TupleTableSlot *
queryDesc->tupDesc = tupleDescriptor; CitusExecScan(CustomScanState *node)
{
CitusScanState *scanState = (CitusScanState *) node;
MultiPlan *multiPlan = scanState->multiPlan;
/* drop into the router executor */ if (scanState->executorType == MULTI_EXECUTOR_ROUTER)
RouterExecutorStart(queryDesc, eflags, taskList); {
return RouterExecScan(scanState);
} }
else else
{ {
PlannedStmt *masterSelectPlan = MasterNodeSelectPlan(multiPlan); TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot;
CreateStmt *masterCreateStmt = MasterNodeCreateStatement(multiPlan);
List *masterCopyStmtList = MasterNodeCopyStatementList(multiPlan); if (!scanState->finishedUnderlyingScan)
RangeTblEntry *masterRangeTableEntry = NULL; {
Job *workerJob = multiPlan->workerJob;
StringInfo jobDirectoryName = NULL; 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. * 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); ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId);
/* pick distributed executor to use */ /* 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 */ /* skip distributed query execution for EXPLAIN commands */
} }
else if (executorType == MULTI_EXECUTOR_REAL_TIME) else if (scanState->executorType == MULTI_EXECUTOR_REAL_TIME)
{ {
MultiRealTimeExecute(workerJob); MultiRealTimeExecute(workerJob);
} }
else if (executorType == MULTI_EXECUTOR_TASK_TRACKER) else if (scanState->executorType == MULTI_EXECUTOR_TASK_TRACKER)
{ {
MultiTaskTrackerExecute(workerJob); MultiTaskTrackerExecute(workerJob);
} }
/* then create the result relation */ tupleDescriptor = node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
ProcessUtility((Node *) masterCreateStmt,
"(temp table creation)",
PROCESS_UTILITY_QUERY,
NULL,
None_Receiver,
NULL);
/* make the temporary table visible */
CommandCounterIncrement();
if (!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
{
CopyQueryResults(masterCopyStmtList);
}
/* /*
* Update the QueryDesc's snapshot so it sees the table. That's not * Load data, collected by Multi*Execute() above, into a
* particularly pretty, but we don't have much of a choice. One might * tuplestore. For that first create a tuplestore, and then copy
* think we could unregister the snapshot, push a new active one, * the files one-by-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 * FIXME: Should probably be in a separate routine.
* 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;
}
}
/* 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);
}
}
/*
* 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)
{
int eflags = queryDesc->estate->es_top_eflags;
if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR)
{
/* drop into the router executor */
RouterExecutorRun(queryDesc, direction, count);
}
else
{
/* drop into the standard executor */
standard_ExecutorRun(queryDesc, direction, count);
}
}
/* Perform actions, like e.g. firing triggers, after the query has run. */
void
multi_ExecutorFinish(QueryDesc *queryDesc)
{
int eflags = queryDesc->estate->es_top_eflags;
if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR)
{
/* 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 * Long term it'd be a lot better if Multi*Execute() directly
* master, remove the resources that were needed for distributed execution. * filled the tuplestores, but that's a fair bit of work.
*/ */
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);
}
/* /*
* Final step of a distributed query is executing the master node select * To be able to use copy.c, we need a Relation descriptor. As
* query. We clean up the temp tables after executing it, if we already created it. * 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.
*/ */
if (eflags & EXEC_FLAG_CITUS_MASTER_SELECT) 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)
{ {
PlannedStmt *planStatement = queryDesc->plannedstmt; Task *workerTask = (Task *) lfirst(workerTaskCell);
int savedLogMinMessages = 0; StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId);
int savedClientMinMessages = 0; StringInfo taskFilename =
TaskFilename(jobDirectoryName, workerTask->taskId);
List *copyOptions = NIL;
CopyState copyState = NULL;
RangeTblEntry *rangeTableEntry = linitial(planStatement->rtable); if (BinaryMasterCopyFormat)
Oid masterTableRelid = rangeTableEntry->relid; {
DefElem *copyOption = makeDefElem("format",
(Node *) makeString("binary"));
copyOptions = lappend(copyOptions, copyOption);
}
copyState = BeginCopyFrom(fakeRel, taskFilename->data, false, NULL,
copyOptions);
ObjectAddress masterTableObject = { InvalidOid, InvalidOid, 0 }; while (true)
{
MemoryContext oldContext = NULL;
bool nextRowFound = false;
masterTableObject.classId = RelationRelationId; ResetPerTupleExprContext(executorState);
masterTableObject.objectId = masterTableRelid; oldContext = MemoryContextSwitchTo(executorTupleContext);
masterTableObject.objectSubId = 0;
/* nextRowFound = NextCopyFrom(copyState, executorExpressionContext,
* Temporarily change logging level to avoid DEBUG2 logging output by columnValues, columnNulls, NULL);
* performDeletion. This avoids breaking the regression tests which if (!nextRowFound)
* use DEBUG2 logging. {
*/ MemoryContextSwitchTo(oldContext);
savedLogMinMessages = log_min_messages; break;
savedClientMinMessages = client_min_messages; }
log_min_messages = INFO; tuplestore_putvalues(scanState->tuplestorestate,
client_min_messages = INFO; tupleDescriptor,
columnValues, columnNulls);
MemoryContextSwitchTo(oldContext);
}
}
performDeletion(&masterTableObject, DROP_RESTRICT, PERFORM_DELETION_INTERNAL); scanState->finishedUnderlyingScan = true;
}
log_min_messages = savedLogMinMessages; if (scanState->tuplestorestate != NULL)
client_min_messages = savedClientMinMessages; {
Tuplestorestate *tupleStore = scanState->tuplestorestate;
tuplestore_gettupleslot(tupleStore, true, false, resultSlot);
return resultSlot;
}
return NULL;
} }
} }
void
CitusEndScan(CustomScanState *node)
{
CitusScanState *scanState = (CitusScanState *) node;
if (scanState->tuplestorestate)
{
tuplestore_end(scanState->tuplestorestate);
scanState->tuplestorestate = NULL;
}
}
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");
}

View File

@ -74,83 +74,33 @@ bool EnableDeadlockPrevention = true;
/* functions needed during run phase */ /* functions needed during run phase */
static void ReacquireMetadataLocks(List *taskList); static void ReacquireMetadataLocks(List *taskList);
static void ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, static void ExecuteSingleModifyTask(CitusScanState *scanState, Task *task,
bool expectResults); bool expectResults);
static void ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task); static void ExecuteSingleSelectTask(CitusScanState *scanState, Task *task);
static List * GetModifyConnections(List *taskPlacementList, static List * GetModifyConnections(List *taskPlacementList,
bool markCritical, bool markCritical,
bool startedInTransaction); bool startedInTransaction);
static void ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList, static void ExecuteMultipleTasks(CitusScanState *scanState, List *taskList,
bool isModificationQuery, bool expectResults); bool isModificationQuery, bool expectResults);
static int64 ExecuteModifyTasks(List *taskList, bool expectResults, static int64 ExecuteModifyTasks(List *taskList, bool expectResults,
ParamListInfo paramListInfo, ParamListInfo paramListInfo,
MaterialState *routerState, CitusScanState *scanState,
TupleDesc tupleDescriptor); TupleDesc tupleDescriptor);
static List * TaskShardIntervalList(List *taskList); static List * TaskShardIntervalList(List *taskList);
static void AcquireExecutorShardLock(Task *task, CmdType commandType); static void AcquireExecutorShardLock(Task *task, CmdType commandType);
static void AcquireExecutorMultiShardLocks(List *taskList); static void AcquireExecutorMultiShardLocks(List *taskList);
static bool RequiresConsistentSnapshot(Task *task); static bool RequiresConsistentSnapshot(Task *task);
static uint64 ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor,
DestReceiver *destination,
Tuplestorestate *tupleStore);
static void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, static void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo,
Oid **parameterTypes, Oid **parameterTypes,
const char ***parameterValues); const char ***parameterValues);
static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query, static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query,
ParamListInfo paramListInfo); ParamListInfo paramListInfo);
static bool StoreQueryResult(MaterialState *routerState, MultiConnection *connection, static bool StoreQueryResult(CitusScanState *scanState, MultiConnection *connection,
TupleDesc tupleDescriptor, bool failOnError, int64 *rows); TupleDesc tupleDescriptor, bool failOnError, int64 *rows);
static bool ConsumeQueryResult(MultiConnection *connection, bool failOnError, static bool ConsumeQueryResult(MultiConnection *connection, bool failOnError,
int64 *rows); 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 * ReacquireMetadataLocks re-acquires the metadata locks that are normally
* acquired during planning. * acquired during planning.
@ -457,73 +407,47 @@ RequiresConsistentSnapshot(Task *task)
} }
/*
* RouterExecutorRun actually executes a single task on a worker.
*/
void void
RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count) RouterBeginScan(CitusScanState *scanState)
{ {
PlannedStmt *planStatement = queryDesc->plannedstmt; MultiPlan *multiPlan = scanState->multiPlan;
MultiPlan *multiPlan = GetMultiPlan(planStatement);
Job *workerJob = multiPlan->workerJob; Job *workerJob = multiPlan->workerJob;
List *taskList = workerJob->taskList; 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 * If we are executing a prepared statement, then we may not yet have obtained
* query might already have been executed is cursors. * the metadata locks in this transaction. To prevent a concurrent shard copy,
* we re-obtain them here or error out if a shard copy has already started.
*
* If a shard copy finishes in between fetching a plan from cache and
* re-acquiring the locks, then we might still run a stale plan, which could
* cause shard placements to diverge. To minimize this window, we take the
* locks as early as possible.
*/ */
if (!routerState->eof_underlying) ReacquireMetadataLocks(taskList);
{ }
bool isModificationQuery = false;
bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation;
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 || if (operation == CMD_INSERT || operation == CMD_UPDATE ||
operation == CMD_DELETE) operation == CMD_DELETE)
{ {
isModificationQuery = true; isModificationQuery = true;
} }
else if (operation != CMD_SELECT)
{
ereport(ERROR, (errmsg("unrecognized operation code: %d",
(int) operation)));
}
if (requiresMasterEvaluation) if (requiresMasterEvaluation)
{ {
@ -539,59 +463,42 @@ RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count)
if (isModificationQuery) if (isModificationQuery)
{ {
ExecuteSingleModifyTask(queryDesc, task, sendTuples); bool sendTuples = multiPlan->hasReturning;
ExecuteSingleModifyTask(scanState, task, sendTuples);
} }
else else
{ {
ExecuteSingleSelectTask(queryDesc, task); ExecuteSingleSelectTask(scanState, task);
} }
} }
else else
{ {
ExecuteMultipleTasks(queryDesc, taskList, isModificationQuery, bool sendTuples = multiPlan->hasReturning;
ExecuteMultipleTasks(scanState, taskList, isModificationQuery,
sendTuples); sendTuples);
} }
/* mark underlying query as having executed */ /* mark underlying query as having executed */
routerState->eof_underlying = true; scanState->finishedUnderlyingScan = true;
} }
/* if the underlying query produced output, return it */ /* if the underlying query produced output, return it */
if (routerState->tuplestorestate != NULL)
{
TupleDesc resultTupleDescriptor = queryDesc->tupDesc;
int64 returnedRows = 0;
/* return rows from the tuplestore */
returnedRows = ReturnRowsFromTuplestore(count, resultTupleDescriptor,
destination,
routerState->tuplestorestate);
/* /*
* Count tuples processed, if this is a SELECT. (For modifications * FIXME: centralize this into function to be shared between router and
* it'll already have been increased, as we want the number of * other executors?
* modified tuples, not the number of RETURNed tuples.)
*/ */
if (operation == CMD_SELECT) if (scanState->tuplestorestate != NULL)
{ {
estate->es_processed += returnedRows; Tuplestorestate *tupleStore = scanState->tuplestorestate;
}
/* XXX: could trivially support backward scans here */
tuplestore_gettupleslot(tupleStore, true, false, resultSlot);
return resultSlot;
} }
out: return NULL;
/* shutdown tuple receiver, if we started it */
if (sendTuples)
{
(*destination->rShutdown)(destination);
}
if (queryDesc->totaltime != NULL)
{
InstrStopNode(queryDesc->totaltime, estate->es_processed);
}
MemoryContextSwitchTo(oldcontext);
} }
@ -603,11 +510,12 @@ out:
* other placements or errors out if the query fails on all placements. * other placements or errors out if the query fails on all placements.
*/ */
static void static void
ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task) ExecuteSingleSelectTask(CitusScanState *scanState, Task *task)
{ {
TupleDesc tupleDescriptor = queryDesc->tupDesc; TupleDesc tupleDescriptor =
MaterialState *routerState = (MaterialState *) queryDesc->planstate; scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
ParamListInfo paramListInfo = queryDesc->params; ParamListInfo paramListInfo =
scanState->customScanState.ss.ps.state->es_param_list_info;
List *taskPlacementList = task->taskPlacementList; List *taskPlacementList = task->taskPlacementList;
ListCell *taskPlacementCell = NULL; ListCell *taskPlacementCell = NULL;
char *queryString = task->queryString; char *queryString = task->queryString;
@ -639,7 +547,7 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task)
continue; continue;
} }
queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, queryOK = StoreQueryResult(scanState, connection, tupleDescriptor,
dontFailOnError, &currentAffectedTupleCount); dontFailOnError, &currentAffectedTupleCount);
if (queryOK) if (queryOK)
{ {
@ -661,14 +569,14 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task)
* framework), or errors out (failed on all placements). * framework), or errors out (failed on all placements).
*/ */
static void static void
ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task, ExecuteSingleModifyTask(CitusScanState *scanState, Task *task,
bool expectResults) bool expectResults)
{ {
CmdType operation = queryDesc->operation; CmdType operation = scanState->multiPlan->operation;
TupleDesc tupleDescriptor = queryDesc->tupDesc; TupleDesc tupleDescriptor =
EState *executorState = queryDesc->estate; scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
MaterialState *routerState = (MaterialState *) queryDesc->planstate; EState *executorState = scanState->customScanState.ss.ps.state;
ParamListInfo paramListInfo = queryDesc->params; ParamListInfo paramListInfo = executorState->es_param_list_info;
bool resultsOK = false; bool resultsOK = false;
List *taskPlacementList = task->taskPlacementList; List *taskPlacementList = task->taskPlacementList;
List *connectionList = NIL; List *connectionList = NIL;
@ -761,7 +669,7 @@ ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task,
*/ */
if (!gotResults && expectResults) if (!gotResults && expectResults)
{ {
queryOK = StoreQueryResult(routerState, connection, tupleDescriptor, queryOK = StoreQueryResult(scanState, connection, tupleDescriptor,
failOnError, &currentAffectedTupleCount); failOnError, &currentAffectedTupleCount);
} }
else else
@ -893,20 +801,21 @@ GetModifyConnections(List *taskPlacementList, bool markCritical, bool noNewTrans
* commits. * commits.
*/ */
static void static void
ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList, ExecuteMultipleTasks(CitusScanState *scanState, List *taskList,
bool isModificationQuery, bool expectResults) bool isModificationQuery, bool expectResults)
{ {
TupleDesc tupleDescriptor = queryDesc->tupDesc; TupleDesc tupleDescriptor =
EState *executorState = queryDesc->estate; scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
MaterialState *routerState = (MaterialState *) queryDesc->planstate; EState *executorState = scanState->customScanState.ss.ps.state;
ParamListInfo paramListInfo = queryDesc->params; ParamListInfo paramListInfo = executorState->es_param_list_info;
int64 affectedTupleCount = -1; int64 affectedTupleCount = -1;
/* can only support modifications right now */ /* can only support modifications right now */
Assert(isModificationQuery); Assert(isModificationQuery);
/* XXX: Seems very redundant to pass both scanState and tupleDescriptor */
affectedTupleCount = ExecuteModifyTasks(taskList, expectResults, paramListInfo, affectedTupleCount = ExecuteModifyTasks(taskList, expectResults, paramListInfo,
routerState, tupleDescriptor); scanState, tupleDescriptor);
executorState->es_processed = affectedTupleCount; executorState->es_processed = affectedTupleCount;
} }
@ -936,7 +845,7 @@ ExecuteModifyTasksWithoutResults(List *taskList)
*/ */
static int64 static int64
ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListInfo, ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListInfo,
MaterialState *routerState, TupleDesc tupleDescriptor) CitusScanState *scanState, TupleDesc tupleDescriptor)
{ {
int64 totalAffectedTupleCount = 0; int64 totalAffectedTupleCount = 0;
ListCell *taskCell = NULL; ListCell *taskCell = NULL;
@ -1066,9 +975,9 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn
*/ */
if (placementIndex == 0 && expectResults) 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, &currentAffectedTupleCount); failOnError, &currentAffectedTupleCount);
} }
else 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 * SendQueryInSingleRowMode sends the given query on the connection in an
* asynchronous way. The function also sets the single-row mode on the * asynchronous way. The function also sets the single-row mode on the
@ -1318,12 +1183,13 @@ ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterT
* the connection. * the connection.
*/ */
static bool static bool
StoreQueryResult(MaterialState *routerState, MultiConnection *connection, StoreQueryResult(CitusScanState *scanState, MultiConnection *connection,
TupleDesc tupleDescriptor, bool failOnError, int64 *rows) TupleDesc tupleDescriptor, bool failOnError, int64 *rows)
{ {
AttInMetadata *attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor); AttInMetadata *attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor);
Tuplestorestate *tupleStore = NULL; 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 *)); char **columnArray = (char **) palloc0(expectedColumnCount * sizeof(char *));
bool commandFailed = false; bool commandFailed = false;
MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext, MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext,
@ -1333,17 +1199,17 @@ StoreQueryResult(MaterialState *routerState, MultiConnection *connection,
ALLOCSET_DEFAULT_MAXSIZE); ALLOCSET_DEFAULT_MAXSIZE);
*rows = 0; *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) else if (!failOnError)
{ {
/* might have failed query execution on another placement before */ /* might have failed query execution on another placement before */
tuplestore_clear(routerState->tuplestorestate); tuplestore_clear(scanState->tuplestorestate);
} }
tupleStore = routerState->tuplestorestate; tupleStore = scanState->tuplestorestate;
for (;;) for (;;)
{ {

View File

@ -369,51 +369,6 @@ multi_ProcessUtility(Node *parsetree,
" necessary users and roles."))); " 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) if (commandMustRunAsOwner)
{ {
GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); GetUserIdAndSecContext(&savedUserId, &savedSecurityContext);

View File

@ -65,9 +65,6 @@ typedef struct RemoteExplainPlan
/* Explain functions for distributed queries */ /* 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 ExplainJob(Job *job, ExplainState *es);
static void ExplainMapMergeJob(MapMergeJob *mapMergeJob, ExplainState *es); static void ExplainMapMergeJob(MapMergeJob *mapMergeJob, ExplainState *es);
static void ExplainTaskList(List *taskList, 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, static void ExplainTaskPlacement(ShardPlacement *taskPlacement, List *explainOutputList,
ExplainState *es); ExplainState *es);
static StringInfo BuildRemoteExplainQuery(char *queryString, 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 Explain functions copied from explain.c */
static void ExplainOpenGroup(const char *objtype, const char *labelname, 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 ExplainJSONLineEnding(ExplainState *es);
static void ExplainYAMLLineStarting(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 void
MultiExplainOneQuery(Query *query, IntoClause *into, ExplainState *es, CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es)
const char *queryString, ParamListInfo params)
{ {
instr_time planStart; CitusScanState *scanState = (CitusScanState *) node;
instr_time planDuration; MultiPlan *multiPlan = scanState->multiPlan;
int cursorOptions = 0; const char *executorName = NULL;
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);
if (!ExplainDistributedQueries) if (!ExplainDistributedQueries)
{ {
appendStringInfoSpaces(es->str, es->indent * 2);
appendStringInfo(es->str, "explain statements for distributed queries "); appendStringInfo(es->str, "explain statements for distributed queries ");
appendStringInfo(es->str, "are not enabled\n"); appendStringInfo(es->str, "are not enabled\n");
return; return;
} }
ExplainOpenGroup("Distributed Query", NULL, true, es); /*
* 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);
if (es->format == EXPLAIN_FORMAT_TEXT) /*
* XXX: might be worthwhile to put this somewhere central, e.g. for
* debugging output.
*/
switch (scanState->executorType)
{ {
appendStringInfoSpaces(es->str, es->indent * 2); case MULTI_EXECUTOR_ROUTER:
appendStringInfo(es->str, "Distributed Query");
if (multiPlan->masterTableName != NULL)
{ {
appendStringInfo(es->str, " into %s", multiPlan->masterTableName); executorName = "Router";
} }
break;
appendStringInfo(es->str, "\n");
es->indent += 1;
}
routerExecutablePlan = multiPlan->routerExecutable;
if (routerExecutablePlan)
{
ExplainPropertyText("Executor", "Router", es);
}
else
{
switch (TaskExecutorType)
{
case MULTI_EXECUTOR_REAL_TIME: case MULTI_EXECUTOR_REAL_TIME:
{ {
ExplainPropertyText("Executor", "Real-Time", es); executorName = "Real-Time";
} }
break; break;
case MULTI_EXECUTOR_TASK_TRACKER: case MULTI_EXECUTOR_TASK_TRACKER:
{ {
ExplainPropertyText("Executor", "Task-Tracker", es); executorName = "Task-Tracker";
} }
break; break;
default: default:
{ {
ExplainPropertyText("Executor", "Other", es); executorName = "Other";
} }
break; break;
} }
} ExplainPropertyText("Executor", executorName, es);
workerJob = multiPlan->workerJob; ExplainJob(multiPlan->workerJob, es);
ExplainJob(workerJob, es);
if (es->format == EXPLAIN_FORMAT_TEXT) ExplainCloseGroup("Distributed Query", "Distributed Query", true, es);
{
es->indent -= 1;
}
if (!routerExecutablePlan)
{
if (es->format == EXPLAIN_FORMAT_TEXT)
{
appendStringInfoSpaces(es->str, es->indent * 2);
appendStringInfo(es->str, "Master Query\n");
es->indent += 1;
}
ExplainOpenGroup("Master Query", "Master Query", false, es);
ExplainMasterPlan(plan, into, es, queryString, params, planDuration);
ExplainCloseGroup("Master Query", "Master Query", false, es);
if (es->format == EXPLAIN_FORMAT_TEXT)
{
es->indent -= 1;
}
}
ExplainCloseGroup("Distributed Query", NULL, true, es);
}
/*
* ExplainMasterPlan generates EXPLAIN output for the master query that merges results.
* When using EXPLAIN ANALYZE, this function shows the execution time of the master query
* in isolation. Calling ExplainOnePlan directly would show the overall execution time of
* the distributed query, which makes it hard to determine how much time the master query
* took.
*
* Parts of this function are copied directly from ExplainOnePlan.
*/
static void
ExplainMasterPlan(PlannedStmt *masterPlan, IntoClause *into,
ExplainState *es, const char *queryString,
ParamListInfo params, const instr_time *planDuration)
{
DestReceiver *dest = NULL;
int eflags = 0;
QueryDesc *queryDesc = NULL;
int instrument_option = 0;
if (es->analyze && es->timing)
{
instrument_option |= INSTRUMENT_TIMER;
}
else if (es->analyze)
{
instrument_option |= INSTRUMENT_ROWS;
}
if (es->buffers)
{
instrument_option |= INSTRUMENT_BUFFERS;
}
/*
* Use a snapshot with an updated command ID to ensure this query sees
* results of any previously executed queries.
*/
PushCopiedSnapshot(GetActiveSnapshot());
UpdateActiveSnapshotCommandId();
/*
* Normally we discard the query's output, but if explaining CREATE TABLE
* AS, we'd better use the appropriate tuple receiver.
*/
if (into)
{
dest = CreateIntoRelDestReceiver(into);
}
else
{
dest = None_Receiver;
}
/* Create a QueryDesc for the query */
queryDesc = CreateQueryDesc(masterPlan, queryString,
GetActiveSnapshot(), InvalidSnapshot,
dest, params, instrument_option);
/* Select execution options */
if (es->analyze)
{
eflags = 0; /* default run-to-completion flags */
}
else
{
eflags = EXEC_FLAG_EXPLAIN_ONLY;
}
if (into)
{
eflags |= GetIntoRelEFlags(into);
}
/*
* ExecutorStart creates the merge table. If using ANALYZE, it also executes the
* worker job and populates the merge table.
*/
ExecutorStart(queryDesc, eflags);
if (es->analyze)
{
ScanDirection dir;
/* if using analyze, then finish query execution */
/* EXPLAIN ANALYZE CREATE TABLE AS WITH NO DATA is weird */
if (into && into->skipData)
{
dir = NoMovementScanDirection;
}
else
{
dir = ForwardScanDirection;
}
/* run the plan */
ExecutorRun(queryDesc, dir, 0L);
/* run cleanup too */
ExecutorFinish(queryDesc);
}
/*
* ExplainOnePlan executes the master query again, which ensures that the execution
* time only shows the execution time of the master query itself, instead of the
* overall execution time.
*/
ExplainOnePlan(queryDesc->plannedstmt, into, es, queryString, params, planDuration);
/*
* ExecutorEnd for the distributed query is deferred until after the master query
* is executed again, otherwise the merge table would be dropped.
*/
ExecutorEnd(queryDesc);
FreeQueryDesc(queryDesc);
PopActiveSnapshot();
} }

View File

@ -34,7 +34,7 @@
* a target target list for the master node. This master target list keeps the * a target target list for the master node. This master target list keeps the
* temporary table's columns on the master node. * temporary table's columns on the master node.
*/ */
static List * List *
MasterTargetList(List *workerTargetList) MasterTargetList(List *workerTargetList)
{ {
List *masterTargetList = NIL; 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 * BuildAggregatePlan creates and returns an aggregate plan. This aggregate plan
* builds aggreation and grouping operators (if any) that are to be executed on * builds aggreation and grouping operators (if any) that are to be executed on
@ -213,14 +170,22 @@ BuildAggregatePlan(Query *masterQuery, Plan *subPlan)
*/ */
static PlannedStmt * static PlannedStmt *
BuildSelectStatement(Query *masterQuery, char *masterTableName, BuildSelectStatement(Query *masterQuery, char *masterTableName,
List *masterTargetList) List *masterTargetList, CustomScan *dataScan)
{ {
PlannedStmt *selectStatement = NULL; PlannedStmt *selectStatement = NULL;
RangeTblEntry *rangeTableEntry = NULL; RangeTblEntry *rangeTableEntry = NULL;
RangeTblEntry *queryRangeTableEntry = NULL; RangeTblEntry *queryRangeTableEntry = NULL;
SeqScan *sequentialScan = NULL;
Agg *aggregationPlan = NULL; Agg *aggregationPlan = NULL;
Plan *topLevelPlan = 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 */ /* (1) make PlannedStmt and set basic information */
selectStatement = makeNode(PlannedStmt); selectStatement = makeNode(PlannedStmt);
@ -233,9 +198,8 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName,
queryRangeTableEntry = (RangeTblEntry *) linitial(masterQuery->rtable); queryRangeTableEntry = (RangeTblEntry *) linitial(masterQuery->rtable);
rangeTableEntry = copyObject(queryRangeTableEntry); rangeTableEntry = copyObject(queryRangeTableEntry);
rangeTableEntry->rtekind = RTE_RELATION; rangeTableEntry->rtekind = RTE_VALUES; /* can't look up relation */
rangeTableEntry->eref = makeAlias(masterTableName, NIL); rangeTableEntry->eref = makeAlias("remote scan", columnNames);
rangeTableEntry->relid = 0; /* to be filled in exec_Start */
rangeTableEntry->inh = false; rangeTableEntry->inh = false;
rangeTableEntry->inFromCl = true; rangeTableEntry->inFromCl = true;
@ -243,22 +207,21 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName,
selectStatement->rtable = list_make1(rangeTableEntry); selectStatement->rtable = list_make1(rangeTableEntry);
/* (2) build and initialize sequential scan node */ /* (2) build and initialize sequential scan node */
sequentialScan = makeNode(SeqScan); /* Gone */
sequentialScan->scanrelid = 1; /* always one */
/* (3) add an aggregation plan if needed */ /* (3) add an aggregation plan if needed */
if (masterQuery->hasAggs || masterQuery->groupClause) 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; topLevelPlan = (Plan *) aggregationPlan;
} }
else else
{ {
/* otherwise set the final projections on the scan plan directly */ /* otherwise set the final projections on the scan plan directly */
sequentialScan->plan.targetlist = masterQuery->targetList; dataScan->scan.plan.targetlist = masterQuery->targetList;
topLevelPlan = (Plan *) sequentialScan; topLevelPlan = &dataScan->scan.plan;
} }
/* (4) add a sorting plan if needed */ /* (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 * 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 * 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. * retrieved from worker nodes and are merged into a temporary table.
*/ */
PlannedStmt * PlannedStmt *
MasterNodeSelectPlan(MultiPlan *multiPlan) MasterNodeSelectPlan(MultiPlan *multiPlan, CustomScan *dataScan)
{ {
Query *masterQuery = multiPlan->masterQuery; Query *masterQuery = multiPlan->masterQuery;
char *tableName = multiPlan->masterTableName; char *tableName = multiPlan->masterTableName;
@ -368,49 +283,8 @@ MasterNodeSelectPlan(MultiPlan *multiPlan)
List *workerTargetList = workerJob->jobQuery->targetList; List *workerTargetList = workerJob->jobQuery->targetList;
List *masterTargetList = MasterTargetList(workerTargetList); List *masterTargetList = MasterTargetList(workerTargetList);
masterSelectPlan = BuildSelectStatement(masterQuery, tableName, masterTargetList); masterSelectPlan =
BuildSelectStatement(masterQuery, tableName, masterTargetList, dataScan);
return masterSelectPlan; 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;
}

View File

@ -221,6 +221,7 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree)
multiPlan->masterQuery = masterQuery; multiPlan->masterQuery = masterQuery;
multiPlan->masterTableName = jobSchemaName->data; multiPlan->masterTableName = jobSchemaName->data;
multiPlan->routerExecutable = MultiPlanRouterExecutable(multiPlan); multiPlan->routerExecutable = MultiPlanRouterExecutable(multiPlan);
multiPlan->operation = CMD_SELECT;
return multiPlan; return multiPlan;
} }

View File

@ -16,10 +16,12 @@
#include "distributed/citus_nodefuncs.h" #include "distributed/citus_nodefuncs.h"
#include "distributed/citus_nodes.h" #include "distributed/citus_nodes.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_planner.h" #include "distributed/multi_planner.h"
#include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_optimizer.h"
#include "distributed/multi_logical_planner.h" #include "distributed/multi_logical_planner.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
#include "distributed/multi_master_planner.h"
#include "distributed/multi_router_planner.h" #include "distributed/multi_router_planner.h"
#include "executor/executor.h" #include "executor/executor.h"
@ -37,7 +39,6 @@ static List *relationRestrictionContextList = NIL;
/* local function forward declarations */ /* local function forward declarations */
static void CheckNodeIsDumpable(Node *node); static void CheckNodeIsDumpable(Node *node);
static char * GetMultiPlanString(PlannedStmt *result);
static PlannedStmt * MultiQueryContainerNode(PlannedStmt *result, static PlannedStmt * MultiQueryContainerNode(PlannedStmt *result,
struct MultiPlan *multiPlan); struct MultiPlan *multiPlan);
static struct PlannedStmt * CreateDistributedPlan(PlannedStmt *localPlan, 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 * GetMultiPlan returns the associated MultiPlan for a CustomScan.
* statement requires distributed execution, NULL otherwise.
*/ */
MultiPlan * MultiPlan *
GetMultiPlan(PlannedStmt *result) GetMultiPlan(CustomScan *customScan)
{ {
char *serializedMultiPlan = NULL;
MultiPlan *multiPlan = NULL; MultiPlan *multiPlan = NULL;
serializedMultiPlan = GetMultiPlanString(result); Assert(IsA(customScan, CustomScan));
multiPlan = (MultiPlan *) CitusStringToNode(serializedMultiPlan); Assert(customScan->methods == &CitusCustomScanMethods);
Assert(CitusIsA(multiPlan, MultiPlan)); Assert(list_length(customScan->custom_private) == 1);
multiPlan = DeSerializeMultiPlan(linitial(customScan->custom_private));
return multiPlan; return multiPlan;
} }
@ -315,24 +322,49 @@ GetMultiPlan(PlannedStmt *result)
bool bool
HasCitusToplevelNode(PlannedStmt *result) HasCitusToplevelNode(PlannedStmt *result)
{ {
/* elog(ERROR, "gone");
* 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;
}
if (GetMultiPlanString(result) == NULL)
{ Node *
return false; SerializableMultiPlan(MultiPlan *multiPlan)
} {
else /*
{ * FIXME: This should be improved for 9.6+, we we can copy trees
return true; * 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 * which should not be referred to outside this file, as it's likely to become
* version dependant. Use GetMultiPlan() and HasCitusToplevelNode() to access. * version dependant. Use GetMultiPlan() and HasCitusToplevelNode() to access.
* *
* FIXME
*
* Internally the data is stored as arguments to a 'citus_extradata_container' * 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 * function, which has to be removed from the really executed plan tree before
* query execution. * query execution.
*/ */
PlannedStmt * PlannedStmt *
MultiQueryContainerNode(PlannedStmt *result, MultiPlan *multiPlan) MultiQueryContainerNode(PlannedStmt *originalPlan, MultiPlan *multiPlan)
{ {
FunctionScan *fauxFunctionScan = NULL; PlannedStmt *resultPlan = NULL;
RangeTblFunction *fauxFunction = NULL; CustomScan *customScan = makeNode(CustomScan);
FuncExpr *fauxFuncExpr = NULL; Node *multiPlanData = SerializableMultiPlan(multiPlan);
Const *multiPlanData = NULL;
char *serializedPlan = NULL;
/* pass multiPlan serialized as a constant function argument */ customScan->methods = &CitusCustomScanMethods;
serializedPlan = CitusNodeToString(multiPlan); customScan->custom_private = list_make1(multiPlanData);
multiPlanData = makeNode(Const);
multiPlanData->consttype = CSTRINGOID;
multiPlanData->constlen = strlen(serializedPlan);
multiPlanData->constvalue = CStringGetDatum(serializedPlan);
multiPlanData->constbyval = false;
multiPlanData->location = -1;
fauxFuncExpr = makeNode(FuncExpr); /* FIXME: This probably ain't correct */
fauxFuncExpr->funcid = CitusExtraDataContainerFuncId(); if (ExecSupportsBackwardScan(originalPlan->planTree))
fauxFuncExpr->funcretset = true; {
fauxFuncExpr->location = -1; customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN;
}
fauxFuncExpr->args = list_make1(multiPlanData);
fauxFunction = makeNode(RangeTblFunction);
fauxFunction->funcexpr = (Node *) fauxFuncExpr;
fauxFunctionScan = makeNode(FunctionScan);
fauxFunctionScan->functions = lappend(fauxFunctionScan->functions, fauxFunction);
/* copy original targetlist, accessed for RETURNING queries */
fauxFunctionScan->scan.plan.targetlist = copyObject(result->planTree->targetlist);
/* /*
* Add set returning function to target list if the original (postgres * FIXME: these two branches/pieces of code should probably be moved into
* created) plan doesn't support backward scans; doing so prevents * router / logical planner code respectively.
* backward scans being supported by the new plantree as well. This is
* ugly as hell, but until we can rely on custom scans (which can signal
* this via CUSTOMPATH_SUPPORT_BACKWARD_SCAN), there's not really a pretty
* method to achieve this.
*
* FIXME: This should really be done on the master select plan.
*/ */
if (!ExecSupportsBackwardScan(result->planTree)) if (multiPlan->masterQuery)
{ {
FuncExpr *funcExpr = makeNode(FuncExpr); resultPlan = MasterNodeSelectPlan(multiPlan, customScan);
TargetEntry *targetEntry = NULL; resultPlan->queryId = originalPlan->queryId;
bool resjunkAttribute = true; resultPlan->utilityStmt = originalPlan->utilityStmt;
funcExpr->funcretset = true;
targetEntry = makeTargetEntry((Expr *) funcExpr, InvalidAttrNumber, NULL,
resjunkAttribute);
fauxFunctionScan->scan.plan.targetlist =
lappend(fauxFunctionScan->scan.plan.targetlist,
targetEntry);
} }
else
{
ListCell *lc = NULL;
List *targetList = NIL;
bool foundJunk = false;
RangeTblEntry *rangeTableEntry = NULL;
List *columnNames = NIL;
int newRTI = list_length(originalPlan->rtable) + 1;
result->planTree = (Plan *) fauxFunctionScan; /*
* XXX: This basically just builds a targetlist to "read" from the
return result; * custom scan output.
}
/*
* GetMultiPlanString returns either NULL, if the plan is not a distributed
* one, or the string representing the distributed plan.
*/ */
static char * foreach(lc, originalPlan->planTree->targetlist)
GetMultiPlanString(PlannedStmt *result)
{
FunctionScan *fauxFunctionScan = NULL;
RangeTblFunction *fauxFunction = NULL;
FuncExpr *fauxFuncExpr = NULL;
Const *multiPlanData = NULL;
if (!IsA(result->planTree, FunctionScan))
{ {
return NULL; 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;
} }
fauxFunctionScan = (FunctionScan *) result->planTree; if (foundJunk)
if (list_length(fauxFunctionScan->functions) != 1)
{ {
return NULL; ereport(ERROR, (errmsg("unexpected !junk entry after resjunk entry")));
} }
fauxFunction = linitial(fauxFunctionScan->functions); /* build TE pointing to custom scan */
newVar = makeVarFromTargetEntry(newRTI, te);
newTargetEntry = flatCopyTargetEntry(te);
newTargetEntry->expr = (Expr *) newVar;
targetList = lappend(targetList, newTargetEntry);
if (!IsA(fauxFunction->funcexpr, FuncExpr)) columnNames = lappend(columnNames, makeString(te->resname));
{
return NULL;
} }
fauxFuncExpr = (FuncExpr *) fauxFunction->funcexpr; /* 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;
if (fauxFuncExpr->funcid != CitusExtraDataContainerFuncId()) resultPlan = originalPlan;
{ resultPlan->planTree = (Plan *) customScan;
return NULL; resultPlan->rtable = lappend(resultPlan->rtable, rangeTableEntry);
customScan->scan.plan.targetlist = targetList;
} }
if (list_length(fauxFuncExpr->args) != 1) return resultPlan;
{
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);
} }

View File

@ -195,6 +195,8 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query,
List *placementList = NIL; List *placementList = NIL;
MultiPlan *multiPlan = CitusMakeNode(MultiPlan); MultiPlan *multiPlan = CitusMakeNode(MultiPlan);
multiPlan->operation = query->commandType;
if (commandType == CMD_INSERT || commandType == CMD_UPDATE || if (commandType == CMD_INSERT || commandType == CMD_UPDATE ||
commandType == CMD_DELETE) commandType == CMD_DELETE)
{ {
@ -236,6 +238,7 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query,
multiPlan->masterQuery = NULL; multiPlan->masterQuery = NULL;
multiPlan->masterTableName = NULL; multiPlan->masterTableName = NULL;
multiPlan->routerExecutable = true; multiPlan->routerExecutable = true;
multiPlan->hasReturning = list_length(originalQuery->returningList) > 0;
return multiPlan; return multiPlan;
} }
@ -264,6 +267,8 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
int shardCount = targetCacheEntry->shardIntervalArrayLength; int shardCount = targetCacheEntry->shardIntervalArrayLength;
bool allReferenceTables = restrictionContext->allReferenceTables; bool allReferenceTables = restrictionContext->allReferenceTables;
multiPlan->operation = originalQuery->commandType;
/* /*
* Error semantics for INSERT ... SELECT queries are different than regular * Error semantics for INSERT ... SELECT queries are different than regular
* modify queries. Thus, handle separately. * modify queries. Thus, handle separately.
@ -319,6 +324,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
multiPlan->masterTableName = NULL; multiPlan->masterTableName = NULL;
multiPlan->masterQuery = NULL; multiPlan->masterQuery = NULL;
multiPlan->routerExecutable = true; multiPlan->routerExecutable = true;
multiPlan->hasReturning = list_length(originalQuery->returningList) > 0;
return multiPlan; return multiPlan;
} }

View File

@ -24,7 +24,6 @@
#include "distributed/master_metadata_utility.h" #include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h" #include "distributed/master_protocol.h"
#include "distributed/multi_copy.h" #include "distributed/multi_copy.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_explain.h" #include "distributed/multi_explain.h"
#include "distributed/multi_join_order.h" #include "distributed/multi_join_order.h"
#include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_optimizer.h"
@ -117,10 +116,6 @@ _PG_init(void)
*/ */
if (planner_hook != NULL || if (planner_hook != NULL ||
ExplainOneQuery_hook != NULL || ExplainOneQuery_hook != NULL ||
ExecutorStart_hook != NULL ||
ExecutorRun_hook != NULL ||
ExecutorFinish_hook != NULL ||
ExecutorEnd_hook != NULL ||
ProcessUtility_hook != NULL) ProcessUtility_hook != NULL)
{ {
ereport(ERROR, (errmsg("Citus has to be loaded first"), ereport(ERROR, (errmsg("Citus has to be loaded first"),
@ -147,15 +142,6 @@ _PG_init(void)
/* intercept planner */ /* intercept planner */
planner_hook = multi_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 */ /* register utility hook */
ProcessUtility_hook = multi_ProcessUtility; ProcessUtility_hook = multi_ProcessUtility;

View File

@ -276,6 +276,9 @@ OutMultiPlan(OUTFUNC_ARGS)
WRITE_NODE_TYPE("MULTIPLAN"); WRITE_NODE_TYPE("MULTIPLAN");
WRITE_INT_FIELD(operation);
WRITE_BOOL_FIELD(hasReturning);
WRITE_NODE_FIELD(workerJob); WRITE_NODE_FIELD(workerJob);
WRITE_NODE_FIELD(masterQuery); WRITE_NODE_FIELD(masterQuery);
WRITE_STRING_FIELD(masterTableName); WRITE_STRING_FIELD(masterTableName);

View File

@ -183,6 +183,9 @@ ReadMultiPlan(READFUNC_ARGS)
{ {
READ_LOCALS(MultiPlan); READ_LOCALS(MultiPlan);
READ_INT_FIELD(operation);
READ_BOOL_FIELD(hasReturning);
READ_NODE_FIELD(workerJob); READ_NODE_FIELD(workerJob);
READ_NODE_FIELD(masterQuery); READ_NODE_FIELD(masterQuery);
READ_STRING_FIELD(masterTableName); READ_STRING_FIELD(masterTableName);

View File

@ -12,6 +12,10 @@
#include "executor/execdesc.h" #include "executor/execdesc.h"
#include "nodes/parsenodes.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 */ /* signal currently executed statement is a master select statement or router execution */
#define EXEC_FLAG_CITUS_MASTER_SELECT 0x100 #define EXEC_FLAG_CITUS_MASTER_SELECT 0x100
@ -23,10 +27,26 @@
#define tuplecount_t long #define tuplecount_t long
#endif #endif
extern void multi_ExecutorStart(QueryDesc *queryDesc, int eflags);
extern void multi_ExecutorRun(QueryDesc *queryDesc, typedef struct CitusScanState
ScanDirection direction, tuplecount_t count); {
extern void multi_ExecutorFinish(QueryDesc *queryDesc); CustomScanState customScanState;
extern void multi_ExecutorEnd(QueryDesc *queryDesc); 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 */ #endif /* MULTI_EXECUTOR_H */

View File

@ -16,7 +16,4 @@
extern bool ExplainDistributedQueries; extern bool ExplainDistributedQueries;
extern bool ExplainAllTasks; extern bool ExplainAllTasks;
extern void MultiExplainOneQuery(Query *query, IntoClause *into, ExplainState *es,
const char *queryString, ParamListInfo params);
#endif /* MULTI_EXPLAIN_H */ #endif /* MULTI_EXPLAIN_H */

View File

@ -21,8 +21,9 @@
/* Function declarations for building local plans on the master node */ /* Function declarations for building local plans on the master node */
struct MultiPlan; struct MultiPlan;
extern CreateStmt * MasterNodeCreateStatement(struct MultiPlan *multiPlan); struct CustomScan;
extern List * MasterNodeCopyStatementList(struct MultiPlan *multiPlan); extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan,
extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan); struct CustomScan *dataScan);
extern List * MasterTargetList(List *workerTargetList);
#endif /* MULTI_MASTER_PLANNER_H */ #endif /* MULTI_MASTER_PLANNER_H */

View File

@ -213,6 +213,10 @@ typedef struct JoinSequenceNode
typedef struct MultiPlan typedef struct MultiPlan
{ {
CitusNode type; CitusNode type;
CmdType operation;
bool hasReturning;
Job *workerJob; Job *workerJob;
Query *masterQuery; Query *masterQuery;
char *masterTableName; char *masterTableName;

View File

@ -53,7 +53,9 @@ extern PlannedStmt * multi_planner(Query *parse, int cursorOptions,
extern bool HasCitusToplevelNode(PlannedStmt *planStatement); extern bool HasCitusToplevelNode(PlannedStmt *planStatement);
struct MultiPlan; 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, extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
Index index, RangeTblEntry *rte); Index index, RangeTblEntry *rte);
extern bool IsModifyCommand(Query *query); extern bool IsModifyCommand(Query *query);

View File

@ -12,8 +12,10 @@
#include "c.h" #include "c.h"
#include "access/sdir.h" #include "access/sdir.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
#include "executor/execdesc.h" #include "executor/execdesc.h"
#include "executor/tuptable.h"
#include "nodes/pg_list.h" #include "nodes/pg_list.h"
@ -33,6 +35,9 @@ typedef struct XactShardConnSet
extern bool AllModificationsCommutative; extern bool AllModificationsCommutative;
extern bool EnableDeadlockPrevention; extern bool EnableDeadlockPrevention;
extern void RouterBeginScan(CitusScanState *scanState);
extern TupleTableSlot * RouterExecScan(CitusScanState *scanState);
extern void RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList); extern void RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList);
extern void RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count); extern void RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count);

View File

@ -39,7 +39,11 @@ $BODY$ LANGUAGE plpgsql;
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -48,18 +52,31 @@ Distributed Query into pg_merge_job_570000
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_290001 lineitem -> 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
-- Test JSON format -- Test JSON format
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
[ [
{ {
"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,
"Group Key": ["l_quantity"],
"Plans": [
{
"Node Type": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "CitusScan",
"Parallel Aware": false,
"Distributed Query": {
"Executor": "Real-Time", "Executor": "Real-Time",
"Job": { "Job": {
"Task Count": 8, "Task Count": 8,
@ -92,36 +109,14 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
] ]
} }
] ]
}, }
"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 -- Validate JSON format
SELECT true AS valid FROM explain_json($$ SELECT true AS valid FROM explain_json($$
@ -133,6 +128,30 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Parallel-Aware>false</Parallel-Aware>
<Sort-Key>
<Item>COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>l_quantity</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Group-Key>
<Item>l_quantity</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Parallel-Aware>false</Parallel-Aware>
<Distributed-Query> <Distributed-Query>
<Executor>Real-Time</Executor> <Executor>Real-Time</Executor>
<Job> <Job>
@ -168,40 +187,13 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</Task> </Task>
</Tasks> </Tasks>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Parallel-Aware>false</Parallel-Aware>
<Sort-Key>
<Item>COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570003_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>intermediate_column_570003_0</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Group-Key>
<Item>intermediate_column_570003_0</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Relation-Name>pg_merge_job_570003</Relation-Name>
<Alias>pg_merge_job_570003</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
-- Validate XML format -- Validate XML format
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
@ -212,7 +204,27 @@ t
EXPLAIN (COSTS FALSE, FORMAT YAML) EXPLAIN (COSTS FALSE, FORMAT YAML)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
- Executor: "Real-Time" - 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
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: Job:
Task Count: 8 Task Count: 8
Tasks Shown: "One of 8" Tasks Shown: "One of 8"
@ -233,32 +245,15 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Relation Name: "lineitem_290001" Relation Name: "lineitem_290001"
Alias: "lineitem" Alias: "lineitem"
Master Query:
- Plan:
Node Type: "Sort"
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"
Plans:
- Node Type: "Aggregate"
Strategy: "Hashed"
Partial Mode: "Simple"
Parent Relationship: "Outer"
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"
-- Test Text format -- Test Text format
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -267,16 +262,13 @@ Distributed Query into pg_merge_job_570006
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_290001 lineitem -> 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
-- Test verbose -- Test verbose
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem; SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem;
Distributed Query into pg_merge_job_570007 Aggregate
Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2")))
-> Custom Scan (CitusScan)
Output: "?column?", "?column?_1", "?column?_2"
Executor: Real-Time Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -286,17 +278,15 @@ Distributed Query into pg_merge_job_570007
Output: sum(l_quantity), sum(l_quantity), count(l_quantity) Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
-> Seq Scan on public.lineitem_290001 lineitem -> 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 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
-- Test join -- Test join
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT * FROM lineitem SELECT * FROM lineitem
JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0 JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0
ORDER BY l_quantity LIMIT 10; ORDER BY l_quantity LIMIT 10;
Distributed Query into pg_merge_job_570008 Limit
-> Sort
Sort Key: l_quantity
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -311,15 +301,10 @@ Distributed Query into pg_merge_job_570008
Filter: (l_quantity < 5.0) Filter: (l_quantity < 5.0)
-> Hash -> Hash
-> Seq Scan on orders_290008 orders -> Seq Scan on orders_290008 orders
Master Query
-> Limit
-> Sort
Sort Key: intermediate_column_570008_4
-> Seq Scan on pg_merge_job_570008
-- Test insert -- Test insert
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
INSERT INTO lineitem VALUES(1,0); INSERT INTO lineitem VALUES(1,0);
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -332,7 +317,7 @@ EXPLAIN (COSTS FALSE)
UPDATE lineitem UPDATE lineitem
SET l_suppkey = 12 SET l_suppkey = 12
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -348,7 +333,7 @@ Distributed Query
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
DELETE FROM lineitem DELETE FROM lineitem
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -363,7 +348,7 @@ Distributed Query
-- Test single-shard SELECT -- Test single-shard SELECT
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
Distributed Query into pg_merge_job_570009 Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -383,20 +368,22 @@ t
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
CREATE TABLE explain_result AS CREATE TABLE explain_result AS
SELECT * FROM lineitem; SELECT * FROM lineitem;
Distributed Query into pg_merge_job_570012 Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
-> Task -> Task
Node: host=localhost port=57637 dbname=regression Node: host=localhost port=57637 dbname=regression
-> Seq Scan on lineitem_290001 lineitem -> Seq Scan on lineitem_290001 lineitem
Master Query
-> Seq Scan on pg_merge_job_570012
-- Test having -- Test having
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem
HAVING sum(l_quantity) > 100; HAVING sum(l_quantity) > 100;
Distributed Query into pg_merge_job_570013 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -406,18 +393,17 @@ Distributed Query into pg_merge_job_570013
Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity) Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity)
-> Seq Scan on public.lineitem_290001 lineitem -> 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 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
-- Test having without aggregate -- Test having without aggregate
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT l_quantity FROM lineitem SELECT l_quantity FROM lineitem
GROUP BY l_quantity GROUP BY l_quantity
HAVING l_quantity > (100 * random()); HAVING l_quantity > (100 * random());
Distributed Query into pg_merge_job_570014 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -428,18 +414,12 @@ Distributed Query into pg_merge_job_570014
Group Key: lineitem.l_quantity Group Key: lineitem.l_quantity
-> Seq Scan on public.lineitem_290001 lineitem -> 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 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
-- Test all tasks output -- Test all tasks output
SET citus.explain_all_tasks TO on; SET citus.explain_all_tasks TO on;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_570015 Aggregate
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 4 Task Count: 4
Tasks Shown: All Tasks Shown: All
@ -463,9 +443,6 @@ Distributed Query into pg_merge_job_570015
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290006 lineitem -> Seq Scan on lineitem_290006 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570015
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$); SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$);
t t
@ -477,7 +454,8 @@ SET citus.task_executor_type TO 'task-tracker';
SET citus.explain_all_tasks TO off; SET citus.explain_all_tasks TO off;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_570018 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: One of 4 Tasks Shown: One of 4
@ -486,9 +464,6 @@ Distributed Query into pg_merge_job_570018
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290005 lineitem -> Seq Scan on lineitem_290005 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570018
-- Test re-partition join -- Test re-partition join
SET citus.large_table_shard_count TO 1; SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
@ -497,7 +472,8 @@ EXPLAIN (COSTS FALSE)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
Distributed Query into pg_merge_job_570021 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 1 Task Count: 1
Tasks Shown: None, not supported for re-partition queries Tasks Shown: None, not supported for re-partition queries
@ -507,9 +483,6 @@ Distributed Query into pg_merge_job_570021
-> MapMergeJob -> MapMergeJob
Map Task Count: 8 Map Task Count: 8
Merge Task Count: 1 Merge Task Count: 1
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570021
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*) SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard FROM lineitem, orders, customer, supplier_single_shard
@ -518,6 +491,18 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
[ [
{ {
"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,
"Distributed Query": {
"Executor": "Task-Tracker", "Executor": "Task-Tracker",
"Job": { "Job": {
"Task Count": 1, "Task Count": 1,
@ -534,27 +519,12 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
] ]
} }
] ]
}, }
"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($$ SELECT true AS valid FROM explain_json($$
SELECT count(*) SELECT count(*)
@ -570,6 +540,18 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parallel-Aware>false</Parallel-Aware>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Parallel-Aware>false</Parallel-Aware>
<Distributed-Query> <Distributed-Query>
<Executor>Task-Tracker</Executor> <Executor>Task-Tracker</Executor>
<Job> <Job>
@ -588,26 +570,11 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</MapMergeJob> </MapMergeJob>
</Depended-Jobs> </Depended-Jobs>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parallel-Aware>false</Parallel-Aware>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Relation-Name>pg_merge_job_570030</Relation-Name>
<Alias>pg_merge_job_570030</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT count(*) SELECT count(*)
@ -635,7 +602,18 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
- Executor: "Task-Tracker" - 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
Distributed Query:
Executor: "Task-Tracker"
Job: Job:
Task Count: 1 Task Count: 1
Tasks Shown: "None, not supported for re-partition queries" Tasks Shown: "None, not supported for re-partition queries"
@ -645,18 +623,6 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Depended Jobs: Depended Jobs:
- Map Task Count: 8 - Map Task Count: 8
Merge Task Count: 1 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_570035"
Alias: "pg_merge_job_570035"
-- test parallel aggregates -- test parallel aggregates
SET parallel_setup_cost=0; SET parallel_setup_cost=0;
SET parallel_tuple_cost=0; SET parallel_tuple_cost=0;
@ -672,7 +638,8 @@ Finalize Aggregate
-> Parallel Seq Scan on lineitem_clone -> Parallel Seq Scan on lineitem_clone
-- ensure distributed plans don't break -- ensure distributed plans don't break
EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem;
Distributed Query into pg_merge_job_570036 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -680,14 +647,12 @@ Distributed Query into pg_merge_job_570036
Node: host=localhost port=57637 dbname=regression Node: host=localhost port=57637 dbname=regression
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290001 lineitem -> Seq Scan on lineitem_290001 lineitem
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570036
-- ensure EXPLAIN EXECUTE doesn't crash -- ensure EXPLAIN EXECUTE doesn't crash
PREPARE task_tracker_query AS PREPARE task_tracker_query AS
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query;
Distributed Query into pg_merge_job_570037 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: One of 4 Tasks Shown: One of 4
@ -696,13 +661,10 @@ Distributed Query into pg_merge_job_570037
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290005 lineitem -> Seq Scan on lineitem_290005 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570037
SET citus.task_executor_type TO 'real-time'; SET citus.task_executor_type TO 'real-time';
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
EXPLAIN EXECUTE router_executor_query; 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 Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -715,7 +677,8 @@ Distributed Query into pg_merge_job_570038
PREPARE real_time_executor_query AS PREPARE real_time_executor_query AS
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query; EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query;
Distributed Query into pg_merge_job_570039 Aggregate
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 4 Task Count: 4
Tasks Shown: One of 4 Tasks Shown: One of 4
@ -724,13 +687,17 @@ Distributed Query into pg_merge_job_570039
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290005 lineitem -> Seq Scan on lineitem_290005 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570039
-- EXPLAIN EXECUTE of parametrized prepared statements is broken, but -- EXPLAIN EXECUTE of parametrized prepared statements is broken, but
-- at least make sure to fail without crashing -- at least make sure to fail without crashing
PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1; PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1;
EXPLAIN EXECUTE router_executor_query_param(5); EXPLAIN EXECUTE router_executor_query_param(5);
ERROR: could not create distributed plan Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus. Executor: Router
HINT: Consider using PLPGSQL functions instead. 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)

View File

@ -39,7 +39,11 @@ $BODY$ LANGUAGE plpgsql;
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -48,18 +52,27 @@ Distributed Query into pg_merge_job_570000
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_290001 lineitem -> 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
-- Test JSON format -- Test JSON format
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
[ [
{ {
"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": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "CitusScan",
"Distributed Query": {
"Executor": "Real-Time", "Executor": "Real-Time",
"Job": { "Job": {
"Task Count": 8, "Task Count": 8,
@ -89,32 +102,14 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
] ]
} }
] ]
}, }
"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 -- Validate JSON format
SELECT true AS valid FROM explain_json($$ SELECT true AS valid FROM explain_json($$
@ -126,6 +121,26 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Sort-Key>
<Item>COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>l_quantity</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Parent-Relationship>Outer</Parent-Relationship>
<Group-Key>
<Item>l_quantity</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Distributed-Query> <Distributed-Query>
<Executor>Real-Time</Executor> <Executor>Real-Time</Executor>
<Job> <Job>
@ -158,36 +173,13 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</Task> </Task>
</Tasks> </Tasks>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Sort-Key>
<Item>COALESCE((sum((COALESCE((sum(intermediate_column_570003_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>intermediate_column_570003_0</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Parent-Relationship>Outer</Parent-Relationship>
<Group-Key>
<Item>intermediate_column_570003_0</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Relation-Name>pg_merge_job_570003</Relation-Name>
<Alias>pg_merge_job_570003</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
-- Validate XML format -- Validate XML format
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
@ -198,7 +190,23 @@ t
EXPLAIN (COSTS FALSE, FORMAT YAML) EXPLAIN (COSTS FALSE, FORMAT YAML)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
- Executor: "Real-Time" - 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: "Custom Scan"
Parent Relationship: "Outer"
Custom Plan Provider: "CitusScan"
Distributed Query:
Executor: "Real-Time"
Job: Job:
Task Count: 8 Task Count: 8
Tasks Shown: "One of 8" Tasks Shown: "One of 8"
@ -216,28 +224,15 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Relation Name: "lineitem_290001" Relation Name: "lineitem_290001"
Alias: "lineitem" 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"
Plans:
- Node Type: "Aggregate"
Strategy: "Hashed"
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"
-- Test Text format -- Test Text format
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem SELECT l_quantity, count(*) count_quantity FROM lineitem
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -246,16 +241,13 @@ Distributed Query into pg_merge_job_570006
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_290001 lineitem -> 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
-- Test verbose -- Test verbose
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem; SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem;
Distributed Query into pg_merge_job_570007 Aggregate
Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2")))
-> Custom Scan (CitusScan)
Output: "?column?", "?column?_1", "?column?_2"
Executor: Real-Time Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -265,17 +257,15 @@ Distributed Query into pg_merge_job_570007
Output: sum(l_quantity), sum(l_quantity), count(l_quantity) Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
-> Seq Scan on public.lineitem_290001 lineitem -> 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 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
-- Test join -- Test join
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT * FROM lineitem SELECT * FROM lineitem
JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0 JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0
ORDER BY l_quantity LIMIT 10; ORDER BY l_quantity LIMIT 10;
Distributed Query into pg_merge_job_570008 Limit
-> Sort
Sort Key: l_quantity
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -290,15 +280,10 @@ Distributed Query into pg_merge_job_570008
Filter: (l_quantity < 5.0) Filter: (l_quantity < 5.0)
-> Hash -> Hash
-> Seq Scan on orders_290008 orders -> Seq Scan on orders_290008 orders
Master Query
-> Limit
-> Sort
Sort Key: intermediate_column_570008_4
-> Seq Scan on pg_merge_job_570008
-- Test insert -- Test insert
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
INSERT INTO lineitem VALUES(1,0); INSERT INTO lineitem VALUES(1,0);
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -311,7 +296,7 @@ EXPLAIN (COSTS FALSE)
UPDATE lineitem UPDATE lineitem
SET l_suppkey = 12 SET l_suppkey = 12
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -327,7 +312,7 @@ Distributed Query
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
DELETE FROM lineitem DELETE FROM lineitem
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -342,7 +327,7 @@ Distributed Query
-- Test single-shard SELECT -- Test single-shard SELECT
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
Distributed Query into pg_merge_job_570009 Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -362,20 +347,22 @@ t
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
CREATE TABLE explain_result AS CREATE TABLE explain_result AS
SELECT * FROM lineitem; SELECT * FROM lineitem;
Distributed Query into pg_merge_job_570012 Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
-> Task -> Task
Node: host=localhost port=57637 dbname=regression Node: host=localhost port=57637 dbname=regression
-> Seq Scan on lineitem_290001 lineitem -> Seq Scan on lineitem_290001 lineitem
Master Query
-> Seq Scan on pg_merge_job_570012
-- Test having -- Test having
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem
HAVING sum(l_quantity) > 100; HAVING sum(l_quantity) > 100;
Distributed Query into pg_merge_job_570013 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -385,18 +372,17 @@ Distributed Query into pg_merge_job_570013
Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity) Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity)
-> Seq Scan on public.lineitem_290001 lineitem -> 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 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
-- Test having without aggregate -- Test having without aggregate
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT l_quantity FROM lineitem SELECT l_quantity FROM lineitem
GROUP BY l_quantity GROUP BY l_quantity
HAVING l_quantity > (100 * random()); HAVING l_quantity > (100 * random());
Distributed Query into pg_merge_job_570014 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 Executor: Real-Time
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -407,18 +393,12 @@ Distributed Query into pg_merge_job_570014
Group Key: lineitem.l_quantity Group Key: lineitem.l_quantity
-> Seq Scan on public.lineitem_290001 lineitem -> 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 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
-- Test all tasks output -- Test all tasks output
SET citus.explain_all_tasks TO on; SET citus.explain_all_tasks TO on;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_570015 Aggregate
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 4 Task Count: 4
Tasks Shown: All Tasks Shown: All
@ -442,9 +422,6 @@ Distributed Query into pg_merge_job_570015
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290006 lineitem -> Seq Scan on lineitem_290006 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570015
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$); SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$);
t t
@ -456,7 +433,8 @@ SET citus.task_executor_type TO 'task-tracker';
SET citus.explain_all_tasks TO off; SET citus.explain_all_tasks TO off;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_570018 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: One of 4 Tasks Shown: One of 4
@ -465,9 +443,6 @@ Distributed Query into pg_merge_job_570018
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290005 lineitem -> Seq Scan on lineitem_290005 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570018
-- Test re-partition join -- Test re-partition join
SET citus.large_table_shard_count TO 1; SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
@ -476,7 +451,8 @@ EXPLAIN (COSTS FALSE)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
Distributed Query into pg_merge_job_570021 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 1 Task Count: 1
Tasks Shown: None, not supported for re-partition queries Tasks Shown: None, not supported for re-partition queries
@ -486,9 +462,6 @@ Distributed Query into pg_merge_job_570021
-> MapMergeJob -> MapMergeJob
Map Task Count: 8 Map Task Count: 8
Merge Task Count: 1 Merge Task Count: 1
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570021
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*) SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard FROM lineitem, orders, customer, supplier_single_shard
@ -497,6 +470,15 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
[ [
{ {
"Plan": {
"Node Type": "Aggregate",
"Strategy": "Plain",
"Plans": [
{
"Node Type": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "CitusScan",
"Distributed Query": {
"Executor": "Task-Tracker", "Executor": "Task-Tracker",
"Job": { "Job": {
"Task Count": 1, "Task Count": 1,
@ -513,24 +495,12 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
] ]
} }
] ]
}, }
"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($$ SELECT true AS valid FROM explain_json($$
SELECT count(*) SELECT count(*)
@ -546,6 +516,15 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Distributed-Query> <Distributed-Query>
<Executor>Task-Tracker</Executor> <Executor>Task-Tracker</Executor>
<Job> <Job>
@ -564,23 +543,11 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</MapMergeJob> </MapMergeJob>
</Depended-Jobs> </Depended-Jobs>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Relation-Name>pg_merge_job_570030</Relation-Name>
<Alias>pg_merge_job_570030</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT count(*) SELECT count(*)
@ -608,7 +575,15 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
- Executor: "Task-Tracker" - Plan:
Node Type: "Aggregate"
Strategy: "Plain"
Plans:
- Node Type: "Custom Scan"
Parent Relationship: "Outer"
Custom Plan Provider: "CitusScan"
Distributed Query:
Executor: "Task-Tracker"
Job: Job:
Task Count: 1 Task Count: 1
Tasks Shown: "None, not supported for re-partition queries" Tasks Shown: "None, not supported for re-partition queries"
@ -618,15 +593,6 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Depended Jobs: Depended Jobs:
- Map Task Count: 8 - Map Task Count: 8
Merge Task Count: 1 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"
-- test parallel aggregates -- test parallel aggregates
SET parallel_setup_cost=0; SET parallel_setup_cost=0;
ERROR: unrecognized configuration parameter "parallel_setup_cost" ERROR: unrecognized configuration parameter "parallel_setup_cost"
@ -643,7 +609,8 @@ Aggregate
-> Seq Scan on lineitem_clone -> Seq Scan on lineitem_clone
-- ensure distributed plans don't break -- ensure distributed plans don't break
EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem;
Distributed Query into pg_merge_job_570036 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 8 Task Count: 8
Tasks Shown: One of 8 Tasks Shown: One of 8
@ -651,14 +618,12 @@ Distributed Query into pg_merge_job_570036
Node: host=localhost port=57637 dbname=regression Node: host=localhost port=57637 dbname=regression
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290001 lineitem -> Seq Scan on lineitem_290001 lineitem
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570036
-- ensure EXPLAIN EXECUTE doesn't crash -- ensure EXPLAIN EXECUTE doesn't crash
PREPARE task_tracker_query AS PREPARE task_tracker_query AS
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query;
Distributed Query into pg_merge_job_570037 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: One of 4 Tasks Shown: One of 4
@ -667,13 +632,10 @@ Distributed Query into pg_merge_job_570037
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290005 lineitem -> Seq Scan on lineitem_290005 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570037
SET citus.task_executor_type TO 'real-time'; SET citus.task_executor_type TO 'real-time';
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
EXPLAIN EXECUTE router_executor_query; 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 Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -686,7 +648,8 @@ Distributed Query into pg_merge_job_570038
PREPARE real_time_executor_query AS PREPARE real_time_executor_query AS
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query; EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query;
Distributed Query into pg_merge_job_570039 Aggregate
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 4 Task Count: 4
Tasks Shown: One of 4 Tasks Shown: One of 4
@ -695,13 +658,17 @@ Distributed Query into pg_merge_job_570039
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_290005 lineitem -> Seq Scan on lineitem_290005 lineitem
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_570039
-- EXPLAIN EXECUTE of parametrized prepared statements is broken, but -- EXPLAIN EXECUTE of parametrized prepared statements is broken, but
-- at least make sure to fail without crashing -- at least make sure to fail without crashing
PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1; PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1;
EXPLAIN EXECUTE router_executor_query_param(5); EXPLAIN EXECUTE router_executor_query_param(5);
ERROR: could not create distributed plan Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus. Executor: Router
HINT: Consider using PLPGSQL functions instead. 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)

View File

@ -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 configuration to print table join order and pruned shards
SET citus.explain_distributed_queries TO off; SET citus.explain_distributed_queries TO off;
SET citus.log_multi_join_order TO TRUE; 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; SET client_min_messages TO DEBUG2;
-- Create new table definitions for use in testing in distributed planning and -- Create new table definitions for use in testing in distributed planning and
-- execution functionality. Also create indexes to boost performance. -- execution functionality. Also create indexes to boost performance.
@ -140,9 +141,10 @@ 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 [8997,10560]
DEBUG: join prunable for intervals [13473,14947] and [10560,12036] DEBUG: join prunable for intervals [13473,14947] and [10560,12036]
QUERY PLAN QUERY PLAN
------------------------------------------------------------ --------------------------------------------------------------
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (2 rows)
-- Update configuration to treat lineitem and orders tables as large -- Update configuration to treat lineitem and orders tables as large
SET citus.large_table_shard_count TO 2; SET citus.large_table_shard_count TO 2;
@ -155,9 +157,11 @@ EXPLAIN SELECT count(*) FROM lineitem, orders
OR (l_orderkey = o_orderkey AND l_quantity < 10); OR (l_orderkey = o_orderkey AND l_quantity < 10);
LOG: join order: [ "lineitem" ][ local partition join "orders" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT l_quantity FROM lineitem, orders EXPLAIN SELECT l_quantity FROM lineitem, orders
WHERE (l_orderkey = o_orderkey OR l_quantity > 5); WHERE (l_orderkey = o_orderkey OR l_quantity > 5);
@ -174,27 +178,33 @@ EXPLAIN SELECT count(*) FROM orders, lineitem_hash
WHERE o_orderkey = l_orderkey; WHERE o_orderkey = l_orderkey;
LOG: join order: [ "orders" ][ single partition join "lineitem_hash" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Verify we handle local joins between two hash-partitioned tables. -- Verify we handle local joins between two hash-partitioned tables.
EXPLAIN SELECT count(*) FROM orders_hash, lineitem_hash EXPLAIN SELECT count(*) FROM orders_hash, lineitem_hash
WHERE o_orderkey = l_orderkey; WHERE o_orderkey = l_orderkey;
LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Validate that we can handle broadcast joins with hash-partitioned tables. -- Validate that we can handle broadcast joins with hash-partitioned tables.
EXPLAIN SELECT count(*) FROM customer_hash, nation EXPLAIN SELECT count(*) FROM customer_hash, nation
WHERE c_nationkey = n_nationkey; WHERE c_nationkey = n_nationkey;
LOG: join order: [ "customer_hash" ][ broadcast join "nation" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Update the large table shard count for all the following tests. -- Update the large table shard count for all the following tests.
SET citus.large_table_shard_count TO 1; SET citus.large_table_shard_count TO 1;
@ -204,9 +214,11 @@ EXPLAIN SELECT count(*) FROM orders, lineitem, customer
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey; WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Validate that we don't chose a single-partition join method with a -- Validate that we don't chose a single-partition join method with a
-- hash-partitioned base table -- hash-partitioned base table
@ -214,9 +226,11 @@ EXPLAIN SELECT count(*) FROM orders, customer_hash
WHERE c_custkey = o_custkey; WHERE c_custkey = o_custkey;
LOG: join order: [ "orders" ][ dual partition join "customer_hash" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Validate that we can re-partition a hash partitioned table to join with a -- Validate that we can re-partition a hash partitioned table to join with a
-- range partitioned one. -- range partitioned one.
@ -224,9 +238,11 @@ EXPLAIN SELECT count(*) FROM orders_hash, customer
WHERE c_custkey = o_custkey; WHERE c_custkey = o_custkey;
LOG: join order: [ "orders_hash" ][ single partition join "customer" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
COMMIT; COMMIT;
-- Reset client logging level to its previous value -- Reset client logging level to its previous value

View File

@ -6,6 +6,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 660000;
-- Enable configuration to print table join order -- Enable configuration to print table join order
SET citus.explain_distributed_queries TO off; SET citus.explain_distributed_queries TO off;
SET citus.log_multi_join_order TO TRUE; 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; SET client_min_messages TO LOG;
-- Change configuration to treat lineitem, orders, customer, and part tables as -- 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 -- large. The following queries are basically the same as the ones in tpch_small
@ -24,9 +25,11 @@ WHERE
and l_quantity < 24; and l_quantity < 24;
LOG: join order: [ "lineitem" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Query #3 from the TPC-H decision support benchmark -- Query #3 from the TPC-H decision support benchmark
EXPLAIN SELECT EXPLAIN SELECT
@ -53,9 +56,14 @@ ORDER BY
o_orderdate; o_orderdate;
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (6 rows)
-- Query #10 from the TPC-H decision support benchmark -- Query #10 from the TPC-H decision support benchmark
EXPLAIN SELECT EXPLAIN SELECT
@ -91,9 +99,14 @@ ORDER BY
revenue DESC; revenue DESC;
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ][ broadcast join "nation" ] LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ][ broadcast join "nation" ]
QUERY PLAN 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 explain statements for distributed queries are not enabled
(1 row) (6 rows)
-- Query #19 from the TPC-H decision support benchmark (modified) -- Query #19 from the TPC-H decision support benchmark (modified)
EXPLAIN SELECT EXPLAIN SELECT
@ -127,9 +140,11 @@ WHERE
); );
LOG: join order: [ "lineitem" ][ single partition join "part" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Query to test multiple re-partition jobs in a single query -- Query to test multiple re-partition jobs in a single query
EXPLAIN SELECT EXPLAIN SELECT
@ -144,9 +159,12 @@ GROUP BY
l_partkey; l_partkey;
LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single partition join "part" ][ single partition join "customer" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (4 rows)
-- Reset client logging level to its previous value -- Reset client logging level to its previous value
SET client_min_messages TO NOTICE; SET client_min_messages TO NOTICE;

View File

@ -19,9 +19,11 @@ WHERE
and l_quantity < 24; and l_quantity < 24;
LOG: join order: [ "lineitem" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Query #3 from the TPC-H decision support benchmark -- Query #3 from the TPC-H decision support benchmark
EXPLAIN SELECT EXPLAIN SELECT
@ -48,9 +50,14 @@ ORDER BY
o_orderdate; o_orderdate;
LOG: join order: [ "orders" ][ broadcast join "customer" ][ local partition join "lineitem" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (6 rows)
-- Query #10 from the TPC-H decision support benchmark -- Query #10 from the TPC-H decision support benchmark
EXPLAIN SELECT EXPLAIN SELECT
@ -86,9 +93,14 @@ ORDER BY
revenue DESC; revenue DESC;
LOG: join order: [ "orders" ][ broadcast join "customer" ][ broadcast join "nation" ][ local partition join "lineitem" ] LOG: join order: [ "orders" ][ broadcast join "customer" ][ broadcast join "nation" ][ local partition join "lineitem" ]
QUERY PLAN 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 explain statements for distributed queries are not enabled
(1 row) (6 rows)
-- Query #19 from the TPC-H decision support benchmark (modified) -- Query #19 from the TPC-H decision support benchmark (modified)
EXPLAIN SELECT EXPLAIN SELECT
@ -122,9 +134,11 @@ WHERE
); );
LOG: join order: [ "lineitem" ][ broadcast join "part" ] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Reset client logging level to its previous value -- Reset client logging level to its previous value
SET client_min_messages TO NOTICE; SET client_min_messages TO NOTICE;

View File

@ -101,9 +101,11 @@ EXPLAIN SELECT count(*)
DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}]
DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) EXPLAIN SELECT count(*)
FROM composite_partitioned_table table1, composite_partitioned_table table2 FROM composite_partitioned_table table1, composite_partitioned_table table2
@ -111,9 +113,11 @@ EXPLAIN SELECT count(*)
DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)] 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)] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Test that large table joins on partition varchar columns work -- Test that large table joins on partition varchar columns work
EXPLAIN SELECT count(*) EXPLAIN SELECT count(*)
@ -122,7 +126,9 @@ EXPLAIN SELECT count(*)
DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U2AMO4ZGX,BZZXSP27F21T6] DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U2AMO4ZGX,BZZXSP27F21T6]
DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6] 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)

View File

@ -61,7 +61,11 @@ $BODY$ LANGUAGE plpgsql;
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -70,18 +74,31 @@ Distributed Query into pg_merge_job_68720796736
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> 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
-- Test JSON format -- Test JSON format
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
[ [
{ {
"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,
"Group Key": ["l_quantity"],
"Plans": [
{
"Node Type": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "CitusScan",
"Parallel Aware": false,
"Distributed Query": {
"Executor": "Real-Time", "Executor": "Real-Time",
"Job": { "Job": {
"Task Count": 16, "Task Count": 16,
@ -114,36 +131,14 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
] ]
} }
] ]
}, }
"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 -- Validate JSON format
SELECT true AS valid FROM explain_json($$ SELECT true AS valid FROM explain_json($$
@ -156,6 +151,30 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Parallel-Aware>false</Parallel-Aware>
<Sort-Key>
<Item>COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>l_quantity</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Group-Key>
<Item>l_quantity</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Parallel-Aware>false</Parallel-Aware>
<Distributed-Query> <Distributed-Query>
<Executor>Real-Time</Executor> <Executor>Real-Time</Executor>
<Job> <Job>
@ -191,40 +210,13 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</Task> </Task>
</Tasks> </Tasks>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Parallel-Aware>false</Parallel-Aware>
<Sort-Key>
<Item>COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_60130862144_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>intermediate_column_60130862144_0</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Group-Key>
<Item>intermediate_column_60130862144_0</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Relation-Name>pg_merge_job_60130862144</Relation-Name>
<Alias>pg_merge_job_60130862144</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
-- Validate XML format -- Validate XML format
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
@ -235,7 +227,27 @@ t
EXPLAIN (COSTS FALSE, FORMAT YAML) EXPLAIN (COSTS FALSE, FORMAT YAML)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
- Executor: "Real-Time" - 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
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: Job:
Task Count: 16 Task Count: 16
Tasks Shown: "One of 16" Tasks Shown: "One of 16"
@ -256,32 +268,15 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Relation Name: "lineitem_mx_1220052" Relation Name: "lineitem_mx_1220052"
Alias: "lineitem_mx" Alias: "lineitem_mx"
Master Query:
- Plan:
Node Type: "Sort"
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"
Plans:
- Node Type: "Aggregate"
Strategy: "Hashed"
Partial Mode: "Simple"
Parent Relationship: "Outer"
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"
-- Test Text format -- Test Text format
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -290,17 +285,14 @@ Distributed Query into pg_merge_job_60130862147
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> 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
\c - - - :worker_2_port \c - - - :worker_2_port
-- Test verbose -- Test verbose
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx; SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx;
Distributed Query into pg_merge_job_68720796739 Aggregate
Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2")))
-> Custom Scan (CitusScan)
Output: "?column?", "?column?_1", "?column?_2"
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -310,17 +302,15 @@ Distributed Query into pg_merge_job_68720796739
Output: sum(l_quantity), sum(l_quantity), count(l_quantity) Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
-> Seq Scan on public.lineitem_mx_1220052 lineitem_mx -> 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 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
-- Test join -- Test join
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT * FROM lineitem_mx SELECT * FROM lineitem_mx
JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0 JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0
ORDER BY l_quantity LIMIT 10; ORDER BY l_quantity LIMIT 10;
Distributed Query into pg_merge_job_68720796740 Limit
-> Sort
Sort Key: l_quantity
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -335,15 +325,10 @@ Distributed Query into pg_merge_job_68720796740
Filter: (l_quantity < 5.0) Filter: (l_quantity < 5.0)
-> Hash -> Hash
-> Seq Scan on orders_mx_1220068 orders_mx -> 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
-- Test insert -- Test insert
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
INSERT INTO lineitem_mx VALUES(1,0); INSERT INTO lineitem_mx VALUES(1,0);
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -356,7 +341,7 @@ EXPLAIN (COSTS FALSE)
UPDATE lineitem_mx UPDATE lineitem_mx
SET l_suppkey = 12 SET l_suppkey = 12
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -370,7 +355,7 @@ Distributed Query
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
DELETE FROM lineitem_mx DELETE FROM lineitem_mx
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -383,7 +368,7 @@ Distributed Query
-- Test single-shard SELECT -- Test single-shard SELECT
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -403,20 +388,19 @@ t
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
CREATE TABLE explain_result AS CREATE TABLE explain_result AS
SELECT * FROM lineitem_mx; SELECT * FROM lineitem_mx;
Distributed Query into pg_merge_job_68720796741 Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
-> Task -> Task
Node: host=localhost port=57637 dbname=regression Node: host=localhost port=57637 dbname=regression
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> Seq Scan on lineitem_mx_1220052 lineitem_mx
Master Query
-> Seq Scan on pg_merge_job_68720796741
-- Test all tasks output -- Test all tasks output
SET citus.explain_all_tasks TO on; SET citus.explain_all_tasks TO on;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_68720796742 Aggregate
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: All Tasks Shown: All
@ -500,9 +484,6 @@ Distributed Query into pg_merge_job_68720796742
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_mx_1220067 lineitem_mx -> Seq Scan on lineitem_mx_1220067 lineitem_mx
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_68720796742
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$);
t t
@ -514,7 +495,8 @@ SET citus.task_executor_type TO 'task-tracker';
SET citus.explain_all_tasks TO off; SET citus.explain_all_tasks TO off;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_68720796745 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -523,9 +505,6 @@ Distributed Query into pg_merge_job_68720796745
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> Seq Scan on lineitem_mx_1220052 lineitem_mx
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_68720796745
-- Test re-partition join -- Test re-partition join
SET citus.large_table_shard_count TO 1; SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
@ -534,7 +513,8 @@ EXPLAIN (COSTS FALSE)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
Distributed Query into pg_merge_job_68720796750 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: None, not supported for re-partition queries Tasks Shown: None, not supported for re-partition queries
@ -550,9 +530,6 @@ Distributed Query into pg_merge_job_68720796750
-> MapMergeJob -> MapMergeJob
Map Task Count: 1 Map Task Count: 1
Merge Task Count: 4 Merge Task Count: 4
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_68720796750
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*) SELECT count(*)
FROM lineitem_mx, orders_mx, customer_mx, supplier_mx FROM lineitem_mx, orders_mx, customer_mx, supplier_mx
@ -561,6 +538,18 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
[ [
{ {
"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,
"Distributed Query": {
"Executor": "Task-Tracker", "Executor": "Task-Tracker",
"Job": { "Job": {
"Task Count": 4, "Task Count": 4,
@ -585,27 +574,12 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
"Merge Task Count": 4 "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($$ SELECT true AS valid FROM explain_json($$
SELECT count(*) SELECT count(*)
@ -621,6 +595,18 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parallel-Aware>false</Parallel-Aware>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Parallel-Aware>false</Parallel-Aware>
<Distributed-Query> <Distributed-Query>
<Executor>Task-Tracker</Executor> <Executor>Task-Tracker</Executor>
<Job> <Job>
@ -647,26 +633,11 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</MapMergeJob> </MapMergeJob>
</Depended-Jobs> </Depended-Jobs>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Partial-Mode>Simple</Partial-Mode>
<Parallel-Aware>false</Parallel-Aware>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Parallel-Aware>false</Parallel-Aware>
<Relation-Name>pg_merge_job_68720796765</Relation-Name>
<Alias>pg_merge_job_68720796765</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT count(*) SELECT count(*)
@ -681,7 +652,18 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
- Executor: "Task-Tracker" - 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
Distributed Query:
Executor: "Task-Tracker"
Job: Job:
Task Count: 4 Task Count: 4
Tasks Shown: "None, not supported for re-partition queries" Tasks Shown: "None, not supported for re-partition queries"
@ -695,15 +677,3 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Merge Task Count: 4 Merge Task Count: 4
- Map Task Count: 1 - Map Task Count: 1
Merge Task Count: 4 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_68720796775"
Alias: "pg_merge_job_68720796775"

View File

@ -61,7 +61,11 @@ $BODY$ LANGUAGE plpgsql;
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -70,18 +74,27 @@ Distributed Query into pg_merge_job_68720796736
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> 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
-- Test JSON format -- Test JSON format
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
[ [
{ {
"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": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "CitusScan",
"Distributed Query": {
"Executor": "Real-Time", "Executor": "Real-Time",
"Job": { "Job": {
"Task Count": 16, "Task Count": 16,
@ -111,32 +124,14 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
] ]
} }
] ]
}, }
"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 -- Validate JSON format
SELECT true AS valid FROM explain_json($$ SELECT true AS valid FROM explain_json($$
@ -149,6 +144,26 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Sort-Key>
<Item>COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>l_quantity</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Parent-Relationship>Outer</Parent-Relationship>
<Group-Key>
<Item>l_quantity</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Distributed-Query> <Distributed-Query>
<Executor>Real-Time</Executor> <Executor>Real-Time</Executor>
<Job> <Job>
@ -181,36 +196,13 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</Task> </Task>
</Tasks> </Tasks>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Sort</Node-Type>
<Sort-Key>
<Item>COALESCE((sum((COALESCE((sum(intermediate_column_60130862144_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)</Item>
<Item>intermediate_column_60130862144_0</Item>
</Sort-Key>
<Plans>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Hashed</Strategy>
<Parent-Relationship>Outer</Parent-Relationship>
<Group-Key>
<Item>intermediate_column_60130862144_0</Item>
</Group-Key>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Relation-Name>pg_merge_job_60130862144</Relation-Name>
<Alias>pg_merge_job_60130862144</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
-- Validate XML format -- Validate XML format
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
@ -221,7 +213,23 @@ t
EXPLAIN (COSTS FALSE, FORMAT YAML) EXPLAIN (COSTS FALSE, FORMAT YAML)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
- Executor: "Real-Time" - 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: "Custom Scan"
Parent Relationship: "Outer"
Custom Plan Provider: "CitusScan"
Distributed Query:
Executor: "Real-Time"
Job: Job:
Task Count: 16 Task Count: 16
Tasks Shown: "One of 16" Tasks Shown: "One of 16"
@ -239,28 +247,15 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Relation Name: "lineitem_mx_1220052" Relation Name: "lineitem_mx_1220052"
Alias: "lineitem_mx" 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"
Plans:
- Node Type: "Aggregate"
Strategy: "Hashed"
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"
-- Test Text format -- Test Text format
EXPLAIN (COSTS FALSE, FORMAT TEXT) EXPLAIN (COSTS FALSE, FORMAT TEXT)
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
GROUP BY l_quantity ORDER BY count_quantity, l_quantity; GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
Distributed Query into 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 Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -269,17 +264,14 @@ Distributed Query into pg_merge_job_60130862147
-> HashAggregate -> HashAggregate
Group Key: l_quantity Group Key: l_quantity
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> 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
\c - - - :worker_2_port \c - - - :worker_2_port
-- Test verbose -- Test verbose
EXPLAIN (COSTS FALSE, VERBOSE TRUE) EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx; SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx;
Distributed Query into pg_merge_job_68720796739 Aggregate
Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2")))
-> Custom Scan (CitusScan)
Output: "?column?", "?column?_1", "?column?_2"
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -289,17 +281,15 @@ Distributed Query into pg_merge_job_68720796739
Output: sum(l_quantity), sum(l_quantity), count(l_quantity) Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
-> Seq Scan on public.lineitem_mx_1220052 lineitem_mx -> 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 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
-- Test join -- Test join
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT * FROM lineitem_mx SELECT * FROM lineitem_mx
JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0 JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0
ORDER BY l_quantity LIMIT 10; ORDER BY l_quantity LIMIT 10;
Distributed Query into pg_merge_job_68720796740 Limit
-> Sort
Sort Key: l_quantity
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -314,15 +304,10 @@ Distributed Query into pg_merge_job_68720796740
Filter: (l_quantity < 5.0) Filter: (l_quantity < 5.0)
-> Hash -> Hash
-> Seq Scan on orders_mx_1220068 orders_mx -> 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
-- Test insert -- Test insert
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
INSERT INTO lineitem_mx VALUES(1,0); INSERT INTO lineitem_mx VALUES(1,0);
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -335,7 +320,7 @@ EXPLAIN (COSTS FALSE)
UPDATE lineitem_mx UPDATE lineitem_mx
SET l_suppkey = 12 SET l_suppkey = 12
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -349,7 +334,7 @@ Distributed Query
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
DELETE FROM lineitem_mx DELETE FROM lineitem_mx
WHERE l_orderkey = 1 AND l_partkey = 0; WHERE l_orderkey = 1 AND l_partkey = 0;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -362,7 +347,7 @@ Distributed Query
-- Test single-shard SELECT -- Test single-shard SELECT
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5; SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5;
Distributed Query Custom Scan (CitusScan)
Executor: Router Executor: Router
Task Count: 1 Task Count: 1
Tasks Shown: All Tasks Shown: All
@ -382,20 +367,19 @@ t
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
CREATE TABLE explain_result AS CREATE TABLE explain_result AS
SELECT * FROM lineitem_mx; SELECT * FROM lineitem_mx;
Distributed Query into pg_merge_job_68720796741 Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
-> Task -> Task
Node: host=localhost port=57637 dbname=regression Node: host=localhost port=57637 dbname=regression
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> Seq Scan on lineitem_mx_1220052 lineitem_mx
Master Query
-> Seq Scan on pg_merge_job_68720796741
-- Test all tasks output -- Test all tasks output
SET citus.explain_all_tasks TO on; SET citus.explain_all_tasks TO on;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_68720796742 Aggregate
-> Custom Scan (CitusScan)
Executor: Real-Time Executor: Real-Time
Task Count: 16 Task Count: 16
Tasks Shown: All Tasks Shown: All
@ -479,9 +463,6 @@ Distributed Query into pg_merge_job_68720796742
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_mx_1220067 lineitem_mx -> Seq Scan on lineitem_mx_1220067 lineitem_mx
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_68720796742
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$);
t t
@ -493,7 +474,8 @@ SET citus.task_executor_type TO 'task-tracker';
SET citus.explain_all_tasks TO off; SET citus.explain_all_tasks TO off;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
Distributed Query into pg_merge_job_68720796745 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 16 Task Count: 16
Tasks Shown: One of 16 Tasks Shown: One of 16
@ -502,9 +484,6 @@ Distributed Query into pg_merge_job_68720796745
-> Aggregate -> Aggregate
-> Seq Scan on lineitem_mx_1220052 lineitem_mx -> Seq Scan on lineitem_mx_1220052 lineitem_mx
Filter: (l_orderkey > 9030) Filter: (l_orderkey > 9030)
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_68720796745
-- Test re-partition join -- Test re-partition join
SET citus.large_table_shard_count TO 1; SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
@ -513,7 +492,8 @@ EXPLAIN (COSTS FALSE)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
Distributed Query into pg_merge_job_68720796750 Aggregate
-> Custom Scan (CitusScan)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: None, not supported for re-partition queries Tasks Shown: None, not supported for re-partition queries
@ -529,9 +509,6 @@ Distributed Query into pg_merge_job_68720796750
-> MapMergeJob -> MapMergeJob
Map Task Count: 1 Map Task Count: 1
Merge Task Count: 4 Merge Task Count: 4
Master Query
-> Aggregate
-> Seq Scan on pg_merge_job_68720796750
EXPLAIN (COSTS FALSE, FORMAT JSON) EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*) SELECT count(*)
FROM lineitem_mx, orders_mx, customer_mx, supplier_mx FROM lineitem_mx, orders_mx, customer_mx, supplier_mx
@ -540,6 +517,15 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
[ [
{ {
"Plan": {
"Node Type": "Aggregate",
"Strategy": "Plain",
"Plans": [
{
"Node Type": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "CitusScan",
"Distributed Query": {
"Executor": "Task-Tracker", "Executor": "Task-Tracker",
"Job": { "Job": {
"Task Count": 4, "Task Count": 4,
@ -564,24 +550,12 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
"Merge Task Count": 4 "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($$ SELECT true AS valid FROM explain_json($$
SELECT count(*) SELECT count(*)
@ -597,6 +571,15 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
<explain xmlns="http://www.postgresql.org/2009/explain"> <explain xmlns="http://www.postgresql.org/2009/explain">
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Plans>
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
<Distributed-Query> <Distributed-Query>
<Executor>Task-Tracker</Executor> <Executor>Task-Tracker</Executor>
<Job> <Job>
@ -623,23 +606,11 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</MapMergeJob> </MapMergeJob>
</Depended-Jobs> </Depended-Jobs>
</Job> </Job>
<Master-Query> </Distributed-Query>
<Query>
<Plan>
<Node-Type>Aggregate</Node-Type>
<Strategy>Plain</Strategy>
<Plans>
<Plan>
<Node-Type>Seq Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Relation-Name>pg_merge_job_68720796765</Relation-Name>
<Alias>pg_merge_job_68720796765</Alias>
</Plan> </Plan>
</Plans> </Plans>
</Plan> </Plan>
</Query> </Query>
</Master-Query>
</Distributed-Query>
</explain> </explain>
SELECT true AS valid FROM explain_xml($$ SELECT true AS valid FROM explain_xml($$
SELECT count(*) SELECT count(*)
@ -654,7 +625,15 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
WHERE l_orderkey = o_orderkey WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey AND o_custkey = c_custkey
AND l_suppkey = s_suppkey; AND l_suppkey = s_suppkey;
- Executor: "Task-Tracker" - Plan:
Node Type: "Aggregate"
Strategy: "Plain"
Plans:
- Node Type: "Custom Scan"
Parent Relationship: "Outer"
Custom Plan Provider: "CitusScan"
Distributed Query:
Executor: "Task-Tracker"
Job: Job:
Task Count: 4 Task Count: 4
Tasks Shown: "None, not supported for re-partition queries" Tasks Shown: "None, not supported for re-partition queries"
@ -668,12 +647,3 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Merge Task Count: 4 Merge Task Count: 4
- Map Task Count: 1 - Map Task Count: 1
Merge Task Count: 4 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"

View File

@ -174,8 +174,8 @@ EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
WHERE repartition_udt.pk > 1; WHERE repartition_udt.pk > 1;
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ] LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
QUERY PLAN QUERY PLAN
------------------------------------------------------------------------- -------------------------------------------------------------
Distributed Query into pg_merge_job_535003 Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: None, not supported for re-partition queries 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 -> MapMergeJob
Map Task Count: 5 Map Task Count: 5
Merge Task Count: 4 Merge Task Count: 4
Master Query (10 rows)
-> Seq Scan on pg_merge_job_535003 (cost=0.00..0.00 rows=0 width=0)
(12 rows)
SELECT * FROM repartition_udt JOIN repartition_udt_other SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.udtcol = repartition_udt_other.udtcol ON repartition_udt.udtcol = repartition_udt_other.udtcol

View File

@ -1489,7 +1489,6 @@ NOTICE: cannot use shard pruning with ANY/ALL (array expression)
HINT: Consider rewriting the expression with OR/AND clauses. HINT: Consider rewriting the expression with OR/AND clauses.
NOTICE: cannot use shard pruning with ANY/ALL (array expression) NOTICE: cannot use shard pruning with ANY/ALL (array expression)
HINT: Consider rewriting the expression with OR/AND clauses. 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 -- router planner/executor is disabled for task-tracker executor
-- following query is router plannable, but router planner is disabled -- following query is router plannable, but router planner is disabled

View File

@ -171,24 +171,30 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename,
EXPLAIN SELECT count(*) FROM varchar_partitioned_table WHERE varchar_column = 'BA2'; EXPLAIN SELECT count(*) FROM varchar_partitioned_table WHERE varchar_column = 'BA2';
DEBUG: predicate pruning for shardId 100 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) FROM array_partitioned_table EXPLAIN SELECT count(*) FROM array_partitioned_table
WHERE array_column > '{BA1000U2AMO4ZGX, BZZXSP27F21T6}'; WHERE array_column > '{BA1000U2AMO4ZGX, BZZXSP27F21T6}';
DEBUG: predicate pruning for shardId 102 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) FROM composite_partitioned_table EXPLAIN SELECT count(*) FROM composite_partitioned_table
WHERE composite_column < '(b,5,c)'::composite_type; WHERE composite_column < '(b,5,c)'::composite_type;
DEBUG: predicate pruning for shardId 105 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
SET client_min_messages TO NOTICE; SET client_min_messages TO NOTICE;

View File

@ -76,25 +76,145 @@ SELECT pg_ls_dir('base/pgsql_job_cache');
-- transaction. -- transaction.
BEGIN; BEGIN;
DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f;
f f
----------------- -----------------

View File

@ -182,8 +182,8 @@ EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
WHERE repartition_udt.pk > 1; WHERE repartition_udt.pk > 1;
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ] LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
QUERY PLAN QUERY PLAN
------------------------------------------------------------------------- -------------------------------------------------------------
Distributed Query into pg_merge_job_535003 Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
Executor: Task-Tracker Executor: Task-Tracker
Task Count: 4 Task Count: 4
Tasks Shown: None, not supported for re-partition queries 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 -> MapMergeJob
Map Task Count: 5 Map Task Count: 5
Merge Task Count: 4 Merge Task Count: 4
Master Query (10 rows)
-> Seq Scan on pg_merge_job_535003 (cost=0.00..0.00 rows=0 width=0)
(12 rows)
SELECT * FROM repartition_udt JOIN repartition_udt_other SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.udtcol = repartition_udt_other.udtcol ON repartition_udt.udtcol = repartition_udt_other.udtcol

View File

@ -2210,12 +2210,12 @@ CONTEXT: PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUE
(5 rows) (5 rows)
-- materialized views can be created for router plannable queries -- 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 articles_hash WHERE author_id = 1;
DEBUG: predicate pruning for shardId 840001 DEBUG: predicate pruning for shardId 840001
DEBUG: Creating router plan DEBUG: Creating router plan
DEBUG: Plan is router executable DEBUG: Plan is router executable
SELECT * FROM mv_articles_hash; SELECT * FROM mv_articles_hash_empty;
id | author_id | title | word_count id | author_id | title | word_count
----+-----------+--------------+------------ ----+-----------+--------------+------------
1 | 1 | arsenous | 9572 1 | 1 | arsenous | 9572
@ -2225,13 +2225,26 @@ SELECT * FROM mv_articles_hash;
41 | 1 | aznavour | 11814 41 | 1 | aznavour | 11814
(5 rows) (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); SELECT * FROM articles_hash WHERE author_id in (1,2);
NOTICE: cannot use shard pruning with ANY/ALL (array expression) NOTICE: cannot use shard pruning with ANY/ALL (array expression)
HINT: Consider rewriting the expression with OR/AND clauses. HINT: Consider rewriting the expression with OR/AND clauses.
NOTICE: cannot use shard pruning with ANY/ALL (array expression) NOTICE: cannot use shard pruning with ANY/ALL (array expression)
HINT: Consider rewriting the expression with OR/AND clauses. 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 -- router planner/executor is now enabled for task-tracker executor
SET citus.task_executor_type to 'task-tracker'; SET citus.task_executor_type to 'task-tracker';
@ -2346,7 +2359,8 @@ DROP USER router_user;
DROP TABLE failure_test; DROP TABLE failure_test;
DROP FUNCTION author_articles_max_id(); DROP FUNCTION author_articles_max_id();
DROP FUNCTION author_articles_id_word_count(); 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_hash;
DROP TABLE articles_single_shard_hash; DROP TABLE articles_single_shard_hash;
DROP TABLE authors_hash; DROP TABLE authors_hash;

View File

@ -61,9 +61,11 @@ DEBUG: assigned task 2 to node localhost:57638
DEBUG: assigned task 4 to node localhost:57637 DEBUG: assigned task 4 to node localhost:57637
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) FROM task_assignment_test_table; EXPLAIN SELECT count(*) FROM task_assignment_test_table;
DEBUG: StartTransactionCommand DEBUG: StartTransactionCommand
@ -73,9 +75,11 @@ DEBUG: assigned task 2 to node localhost:57638
DEBUG: assigned task 4 to node localhost:57637 DEBUG: assigned task 4 to node localhost:57637
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Next test the first-replica task assignment policy -- Next test the first-replica task assignment policy
SET citus.task_assignment_policy TO 'first-replica'; SET citus.task_assignment_policy TO 'first-replica';
@ -90,9 +94,11 @@ DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 2 to node localhost:57638 DEBUG: assigned task 2 to node localhost:57638
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) FROM task_assignment_test_table; EXPLAIN SELECT count(*) FROM task_assignment_test_table;
DEBUG: StartTransactionCommand DEBUG: StartTransactionCommand
@ -102,9 +108,11 @@ DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 2 to node localhost:57638 DEBUG: assigned task 2 to node localhost:57638
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
-- Round-robin task assignment relies on the current jobId. We therefore need to -- 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 -- ensure that jobIds start with an odd number here; this way, task assignment
@ -133,9 +141,11 @@ DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 2 to node localhost:57638 DEBUG: assigned task 2 to node localhost:57638
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) FROM task_assignment_test_table; EXPLAIN SELECT count(*) FROM task_assignment_test_table;
DEBUG: StartTransactionCommand DEBUG: StartTransactionCommand
@ -145,9 +155,11 @@ DEBUG: assigned task 4 to node localhost:57638
DEBUG: assigned task 2 to node localhost:57637 DEBUG: assigned task 2 to node localhost:57637
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
EXPLAIN SELECT count(*) FROM task_assignment_test_table; EXPLAIN SELECT count(*) FROM task_assignment_test_table;
DEBUG: StartTransactionCommand DEBUG: StartTransactionCommand
@ -157,9 +169,11 @@ DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 2 to node localhost:57638 DEBUG: assigned task 2 to node localhost:57638
DEBUG: CommitTransactionCommand 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)
explain statements for distributed queries are not enabled explain statements for distributed queries are not enabled
(1 row) (3 rows)
RESET citus.task_assignment_policy; RESET citus.task_assignment_policy;
DEBUG: StartTransactionCommand DEBUG: StartTransactionCommand

View File

@ -283,12 +283,12 @@ DROP TABLE ddl_commands;
-- Distributed SELECTs cannot appear after ALTER -- Distributed SELECTs cannot appear after ALTER
BEGIN; BEGIN;
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
SELECT l_orderkey FROM lineitem_alter LIMIT 0; SELECT count(*) FROM lineitem_alter;
COMMIT; COMMIT;
-- but are allowed before -- but are allowed before
BEGIN; BEGIN;
SELECT l_orderkey FROM lineitem_alter LIMIT 0; SELECT count(*) FROM lineitem_alter;
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
COMMIT; COMMIT;
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter'; SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';

View File

@ -596,15 +596,16 @@ BEGIN;
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
NOTICE: using one-phase commit for distributed DDL commands 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' 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 ERROR: cannot open new connections after the first modification command within a transaction
COMMIT; COMMIT;
-- but are allowed before -- but are allowed before
BEGIN; BEGIN;
SELECT l_orderkey FROM lineitem_alter LIMIT 0; SELECT count(*) FROM lineitem_alter;
l_orderkey count
------------ -------
(0 rows) 18000
(1 row)
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey); CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
COMMIT; COMMIT;

View File

@ -765,8 +765,9 @@ FROM
tenant_id, tenant_id,
user_id) AS subquery; user_id) AS subquery;
QUERY PLAN QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Distributed Query into pg_merge_job_270014 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 Executor: Real-Time
Task Count: 2 Task Count: 2
Tasks Shown: One of 2 Tasks Shown: One of 2
@ -785,10 +786,7 @@ FROM
Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) 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) -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556)
Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[]))
Master Query (20 rows)
-> 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)
-- Union and left join subquery pushdown -- Union and left join subquery pushdown
EXPLAIN SELECT EXPLAIN SELECT
@ -854,8 +852,10 @@ FROM
GROUP BY GROUP BY
hasdone; hasdone;
QUERY PLAN QUERY PLAN
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Distributed Query into pg_merge_job_270015 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 Executor: Real-Time
Task Count: 2 Task Count: 2
Tasks Shown: One of 2 Tasks Shown: One of 2
@ -894,11 +894,7 @@ GROUP BY
Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) 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) -> 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)) Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text))
Master Query (41 rows)
-> 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)
-- Union, left join and having subquery pushdown -- Union, left join and having subquery pushdown
EXPLAIN SELECT EXPLAIN SELECT
@ -1023,8 +1019,11 @@ ORDER BY
LIMIT LIMIT
10; 10;
QUERY PLAN QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Distributed Query into pg_merge_job_270017 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 Executor: Real-Time
Task Count: 2 Task Count: 2
Tasks Shown: One of 2 Tasks Shown: One of 2
@ -1048,11 +1047,6 @@ LIMIT
Sort Key: events.event_time DESC Sort Key: events.event_time DESC
-> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) -> 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))) Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id)))
Master Query (27 rows)
-> 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)
SET citus.enable_router_execution TO 'true'; SET citus.enable_router_execution TO 'true';

View File

@ -11,6 +11,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 650000;
SET citus.explain_distributed_queries TO off; SET citus.explain_distributed_queries TO off;
SET citus.log_multi_join_order TO TRUE; 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; SET client_min_messages TO DEBUG2;
-- Create new table definitions for use in testing in distributed planning and -- Create new table definitions for use in testing in distributed planning and

View File

@ -11,6 +11,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 660000;
SET citus.explain_distributed_queries TO off; SET citus.explain_distributed_queries TO off;
SET citus.log_multi_join_order TO TRUE; 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; SET client_min_messages TO LOG;
-- Change configuration to treat lineitem, orders, customer, and part tables as -- Change configuration to treat lineitem, orders, customer, and part tables as

View File

@ -46,25 +46,45 @@ SELECT pg_ls_dir('base/pgsql_job_cache');
-- transaction. -- transaction.
BEGIN; BEGIN;
DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; 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; SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f;
-- close first, 17th (first after re-alloc) and last cursor. -- close first, 17th (first after re-alloc) and last cursor.
CLOSE c_00; CLOSE c_00;

View File

@ -988,13 +988,13 @@ $$ LANGUAGE plpgsql;
SELECT * FROM author_articles_id_word_count(); SELECT * FROM author_articles_id_word_count();
-- materialized views can be created for router plannable queries -- 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 articles_hash WHERE author_id = 1;
SELECT * FROM mv_articles_hash_empty;
SELECT * FROM mv_articles_hash; CREATE MATERIALIZED VIEW mv_articles_hash_data AS
CREATE MATERIALIZED VIEW mv_articles_hash_error AS
SELECT * FROM articles_hash WHERE author_id in (1,2); 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 -- router planner/executor is now enabled for task-tracker executor
SET citus.task_executor_type to 'task-tracker'; SET citus.task_executor_type to 'task-tracker';
@ -1053,7 +1053,8 @@ DROP TABLE failure_test;
DROP FUNCTION author_articles_max_id(); DROP FUNCTION author_articles_max_id();
DROP FUNCTION author_articles_id_word_count(); 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_hash;
DROP TABLE articles_single_shard_hash; DROP TABLE articles_single_shard_hash;