mirror of https://github.com/citusdata/citus.git
Initial temp table removal implementation
parent
6f4886cd11
commit
52358fe891
|
@ -24,63 +24,107 @@
|
|||
#include "distributed/multi_utility.h"
|
||||
#include "distributed/worker_protocol.h"
|
||||
#include "executor/execdebug.h"
|
||||
#include "executor/executor.h"
|
||||
#include "commands/copy.h"
|
||||
#include "nodes/makefuncs.h"
|
||||
#include "storage/lmgr.h"
|
||||
#include "tcop/utility.h"
|
||||
#include "utils/snapmgr.h"
|
||||
|
||||
|
||||
static void CopyQueryResults(List *masterCopyStmtList);
|
||||
#include "utils/memutils.h"
|
||||
|
||||
|
||||
/*
|
||||
* multi_ExecutorStart is a hook called at at the beginning of any execution
|
||||
* of any query plan.
|
||||
* FIXME: It'd probably be better to have different set of methods for:
|
||||
* - router readonly queries
|
||||
* - router modify
|
||||
* - router insert ... select
|
||||
* - real-time/task-tracker (no point in seperating those)
|
||||
*
|
||||
* If a distributed relation is the target of the query, perform some validity
|
||||
* checks. If a legal statement, start the distributed execution. After that
|
||||
* the to-be-executed query is replaced with the portion executing solely on
|
||||
* the master.
|
||||
* I think it's better however to only have one type of CitusScanState, to
|
||||
* allow to easily share code between routines.
|
||||
*/
|
||||
void
|
||||
multi_ExecutorStart(QueryDesc *queryDesc, int eflags)
|
||||
static CustomExecMethods CitusCustomExecMethods = {
|
||||
"CitusScan",
|
||||
CitusBeginScan,
|
||||
CitusExecScan,
|
||||
CitusEndScan,
|
||||
CitusReScan,
|
||||
#if (PG_VERSION_NUM >= 90600)
|
||||
NULL, /* NO EstimateDSMCustomScan callback */
|
||||
NULL, /* NO InitializeDSMCustomScan callback */
|
||||
NULL, /* NO InitializeWorkerCustomScan callback */
|
||||
#endif
|
||||
NULL,
|
||||
NULL,
|
||||
CitusExplainScan
|
||||
};
|
||||
|
||||
|
||||
Node *
|
||||
CitusCreateScan(CustomScan *scan)
|
||||
{
|
||||
PlannedStmt *planStatement = queryDesc->plannedstmt;
|
||||
CitusScanState *scanState = palloc0(sizeof(CitusScanState));
|
||||
|
||||
if (HasCitusToplevelNode(planStatement))
|
||||
scanState->customScanState.ss.ps.type = T_CustomScanState;
|
||||
scanState->customScanState.methods = &CitusCustomExecMethods;
|
||||
scanState->multiPlan = GetMultiPlan(scan);
|
||||
scanState->executorType = JobExecutorType(scanState->multiPlan);
|
||||
|
||||
return (Node *) scanState;
|
||||
}
|
||||
|
||||
|
||||
void
|
||||
CitusBeginScan(CustomScanState *node,
|
||||
EState *estate,
|
||||
int eflags)
|
||||
{
|
||||
CitusScanState *scanState = (CitusScanState *) node;
|
||||
MultiPlan *multiPlan = scanState->multiPlan;
|
||||
|
||||
Assert(IsA(scanState, CustomScanState));
|
||||
|
||||
/* ensure plan is executable */
|
||||
VerifyMultiPlanValidity(multiPlan);
|
||||
|
||||
/* ExecCheckRTPerms(planStatement->rtable, true); */
|
||||
|
||||
if (scanState->executorType == MULTI_EXECUTOR_ROUTER)
|
||||
{
|
||||
MultiPlan *multiPlan = GetMultiPlan(planStatement);
|
||||
MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST;
|
||||
Job *workerJob = multiPlan->workerJob;
|
||||
RouterBeginScan(scanState);
|
||||
}
|
||||
}
|
||||
|
||||
/* ensure plan is executable */
|
||||
VerifyMultiPlanValidity(multiPlan);
|
||||
|
||||
ExecCheckRTPerms(planStatement->rtable, true);
|
||||
TupleTableSlot *
|
||||
CitusExecScan(CustomScanState *node)
|
||||
{
|
||||
CitusScanState *scanState = (CitusScanState *) node;
|
||||
MultiPlan *multiPlan = scanState->multiPlan;
|
||||
|
||||
executorType = JobExecutorType(multiPlan);
|
||||
if (executorType == MULTI_EXECUTOR_ROUTER)
|
||||
if (scanState->executorType == MULTI_EXECUTOR_ROUTER)
|
||||
{
|
||||
return RouterExecScan(scanState);
|
||||
}
|
||||
else
|
||||
{
|
||||
TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot;
|
||||
|
||||
if (!scanState->finishedUnderlyingScan)
|
||||
{
|
||||
List *taskList = workerJob->taskList;
|
||||
TupleDesc tupleDescriptor = ExecCleanTypeFromTL(
|
||||
planStatement->planTree->targetlist, false);
|
||||
List *dependendJobList PG_USED_FOR_ASSERTS_ONLY = workerJob->dependedJobList;
|
||||
|
||||
/* router executor cannot execute task with depencencies */
|
||||
Assert(dependendJobList == NIL);
|
||||
|
||||
/* we need to set tupleDesc in executorStart */
|
||||
queryDesc->tupDesc = tupleDescriptor;
|
||||
|
||||
/* drop into the router executor */
|
||||
RouterExecutorStart(queryDesc, eflags, taskList);
|
||||
}
|
||||
else
|
||||
{
|
||||
PlannedStmt *masterSelectPlan = MasterNodeSelectPlan(multiPlan);
|
||||
CreateStmt *masterCreateStmt = MasterNodeCreateStatement(multiPlan);
|
||||
List *masterCopyStmtList = MasterNodeCopyStatementList(multiPlan);
|
||||
RangeTblEntry *masterRangeTableEntry = NULL;
|
||||
Job *workerJob = multiPlan->workerJob;
|
||||
StringInfo jobDirectoryName = NULL;
|
||||
EState *executorState = scanState->customScanState.ss.ps.state;
|
||||
List *workerTaskList = workerJob->taskList;
|
||||
ListCell *workerTaskCell = NULL;
|
||||
TupleDesc tupleDescriptor = NULL;
|
||||
Relation fakeRel = NULL;
|
||||
MemoryContext executorTupleContext = GetPerTupleMemoryContext(executorState);
|
||||
ExprContext *executorExpressionContext =
|
||||
GetPerTupleExprContext(executorState);
|
||||
uint32 columnCount = 0;
|
||||
Datum *columnValues = NULL;
|
||||
bool *columnNulls = NULL;
|
||||
|
||||
/*
|
||||
* We create a directory on the master node to keep task execution results.
|
||||
|
@ -93,204 +137,133 @@ multi_ExecutorStart(QueryDesc *queryDesc, int eflags)
|
|||
ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId);
|
||||
|
||||
/* pick distributed executor to use */
|
||||
if (eflags & EXEC_FLAG_EXPLAIN_ONLY)
|
||||
if (executorState->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY)
|
||||
{
|
||||
/* skip distributed query execution for EXPLAIN commands */
|
||||
}
|
||||
else if (executorType == MULTI_EXECUTOR_REAL_TIME)
|
||||
else if (scanState->executorType == MULTI_EXECUTOR_REAL_TIME)
|
||||
{
|
||||
MultiRealTimeExecute(workerJob);
|
||||
}
|
||||
else if (executorType == MULTI_EXECUTOR_TASK_TRACKER)
|
||||
else if (scanState->executorType == MULTI_EXECUTOR_TASK_TRACKER)
|
||||
{
|
||||
MultiTaskTrackerExecute(workerJob);
|
||||
}
|
||||
|
||||
/* then create the result relation */
|
||||
ProcessUtility((Node *) masterCreateStmt,
|
||||
"(temp table creation)",
|
||||
PROCESS_UTILITY_QUERY,
|
||||
NULL,
|
||||
None_Receiver,
|
||||
NULL);
|
||||
tupleDescriptor = node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
|
||||
|
||||
/* make the temporary table visible */
|
||||
CommandCounterIncrement();
|
||||
/*
|
||||
* Load data, collected by Multi*Execute() above, into a
|
||||
* tuplestore. For that first create a tuplestore, and then copy
|
||||
* the files one-by-one.
|
||||
*
|
||||
* FIXME: Should probably be in a separate routine.
|
||||
*
|
||||
* Long term it'd be a lot better if Multi*Execute() directly
|
||||
* filled the tuplestores, but that's a fair bit of work.
|
||||
*/
|
||||
|
||||
if (!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
|
||||
/*
|
||||
* To be able to use copy.c, we need a Relation descriptor. As
|
||||
* there's no relation corresponding to the data loaded from
|
||||
* workers, fake one. We just need the bare minimal set of fields
|
||||
* accessed by BeginCopyFrom().
|
||||
*
|
||||
* FIXME: should be abstracted into a separate function.
|
||||
*/
|
||||
fakeRel = palloc0(sizeof(RelationData));
|
||||
fakeRel->rd_att = tupleDescriptor;
|
||||
fakeRel->rd_rel = palloc0(sizeof(FormData_pg_class));
|
||||
fakeRel->rd_rel->relkind = RELKIND_RELATION;
|
||||
|
||||
columnCount = tupleDescriptor->natts;
|
||||
columnValues = palloc0(columnCount * sizeof(Datum));
|
||||
columnNulls = palloc0(columnCount * sizeof(bool));
|
||||
|
||||
Assert(scanState->tuplestorestate == NULL);
|
||||
scanState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem);
|
||||
|
||||
foreach(workerTaskCell, workerTaskList)
|
||||
{
|
||||
CopyQueryResults(masterCopyStmtList);
|
||||
Task *workerTask = (Task *) lfirst(workerTaskCell);
|
||||
StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId);
|
||||
StringInfo taskFilename =
|
||||
TaskFilename(jobDirectoryName, workerTask->taskId);
|
||||
List *copyOptions = NIL;
|
||||
CopyState copyState = NULL;
|
||||
|
||||
if (BinaryMasterCopyFormat)
|
||||
{
|
||||
DefElem *copyOption = makeDefElem("format",
|
||||
(Node *) makeString("binary"));
|
||||
copyOptions = lappend(copyOptions, copyOption);
|
||||
}
|
||||
copyState = BeginCopyFrom(fakeRel, taskFilename->data, false, NULL,
|
||||
copyOptions);
|
||||
|
||||
while (true)
|
||||
{
|
||||
MemoryContext oldContext = NULL;
|
||||
bool nextRowFound = false;
|
||||
|
||||
ResetPerTupleExprContext(executorState);
|
||||
oldContext = MemoryContextSwitchTo(executorTupleContext);
|
||||
|
||||
nextRowFound = NextCopyFrom(copyState, executorExpressionContext,
|
||||
columnValues, columnNulls, NULL);
|
||||
if (!nextRowFound)
|
||||
{
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
break;
|
||||
}
|
||||
|
||||
tuplestore_putvalues(scanState->tuplestorestate,
|
||||
tupleDescriptor,
|
||||
columnValues, columnNulls);
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Update the QueryDesc's snapshot so it sees the table. That's not
|
||||
* particularly pretty, but we don't have much of a choice. One might
|
||||
* think we could unregister the snapshot, push a new active one,
|
||||
* update it, register it, and be happy. That only works if it's only
|
||||
* registered once though...
|
||||
*/
|
||||
queryDesc->snapshot->curcid = GetCurrentCommandId(false);
|
||||
|
||||
/*
|
||||
* Set the OID of the RTE used in the master select statement to point
|
||||
* to the now created (and filled) temporary table. The target
|
||||
* relation's oid is only known now.
|
||||
*/
|
||||
masterRangeTableEntry =
|
||||
(RangeTblEntry *) linitial(masterSelectPlan->rtable);
|
||||
masterRangeTableEntry->relid =
|
||||
RelnameGetRelid(masterRangeTableEntry->eref->aliasname);
|
||||
|
||||
/*
|
||||
* Replace to-be-run query with the master select query. As the
|
||||
* planned statement is now replaced we can't call GetMultiPlan() in
|
||||
* the later hooks, so we set a flag marking this as a distributed
|
||||
* statement running on the master. That e.g. allows us to drop the
|
||||
* temp table later.
|
||||
*
|
||||
* We copy the original statement's queryId, to allow
|
||||
* pg_stat_statements and similar extension to associate the
|
||||
* statement with the toplevel statement.
|
||||
*/
|
||||
masterSelectPlan->queryId = queryDesc->plannedstmt->queryId;
|
||||
queryDesc->plannedstmt = masterSelectPlan;
|
||||
|
||||
eflags |= EXEC_FLAG_CITUS_MASTER_SELECT;
|
||||
scanState->finishedUnderlyingScan = true;
|
||||
}
|
||||
}
|
||||
|
||||
/* if the execution is not done for router executor, drop into standard executor */
|
||||
if (queryDesc->estate == NULL ||
|
||||
!(queryDesc->estate->es_top_eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR))
|
||||
{
|
||||
standard_ExecutorStart(queryDesc, eflags);
|
||||
if (scanState->tuplestorestate != NULL)
|
||||
{
|
||||
Tuplestorestate *tupleStore = scanState->tuplestorestate;
|
||||
tuplestore_gettupleslot(tupleStore, true, false, resultSlot);
|
||||
|
||||
return resultSlot;
|
||||
}
|
||||
|
||||
return NULL;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CopyQueryResults executes the commands that copy query results into a
|
||||
* temporary table.
|
||||
*/
|
||||
static void
|
||||
CopyQueryResults(List *masterCopyStmtList)
|
||||
{
|
||||
ListCell *masterCopyStmtCell = NULL;
|
||||
|
||||
/* now copy data from all the remote nodes into temp table */
|
||||
foreach(masterCopyStmtCell, masterCopyStmtList)
|
||||
{
|
||||
Node *masterCopyStmt = (Node *) lfirst(masterCopyStmtCell);
|
||||
|
||||
Assert(IsA(masterCopyStmt, CopyStmt));
|
||||
|
||||
ProcessUtility(masterCopyStmt,
|
||||
"(copy job)",
|
||||
PROCESS_UTILITY_QUERY,
|
||||
NULL,
|
||||
None_Receiver,
|
||||
NULL);
|
||||
}
|
||||
|
||||
/* make the copied contents visible */
|
||||
CommandCounterIncrement();
|
||||
}
|
||||
|
||||
|
||||
/* Execute query plan. */
|
||||
void
|
||||
multi_ExecutorRun(QueryDesc *queryDesc, ScanDirection direction, tuplecount_t count)
|
||||
CitusEndScan(CustomScanState *node)
|
||||
{
|
||||
int eflags = queryDesc->estate->es_top_eflags;
|
||||
CitusScanState *scanState = (CitusScanState *) node;
|
||||
|
||||
if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR)
|
||||
if (scanState->tuplestorestate)
|
||||
{
|
||||
/* drop into the router executor */
|
||||
RouterExecutorRun(queryDesc, direction, count);
|
||||
}
|
||||
else
|
||||
{
|
||||
/* drop into the standard executor */
|
||||
standard_ExecutorRun(queryDesc, direction, count);
|
||||
tuplestore_end(scanState->tuplestorestate);
|
||||
scanState->tuplestorestate = NULL;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/* Perform actions, like e.g. firing triggers, after the query has run. */
|
||||
void
|
||||
multi_ExecutorFinish(QueryDesc *queryDesc)
|
||||
CitusReScan(CustomScanState *node)
|
||||
{
|
||||
int eflags = queryDesc->estate->es_top_eflags;
|
||||
CitusScanState *scanState = (CitusScanState *) node;
|
||||
|
||||
if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR)
|
||||
{
|
||||
/* drop into the router executor */
|
||||
RouterExecutorFinish(queryDesc);
|
||||
}
|
||||
else
|
||||
{
|
||||
/* drop into the standard executor */
|
||||
standard_ExecutorFinish(queryDesc);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* multi_ExecutorEnd is a hook called to deallocate resources used during
|
||||
* query execution.
|
||||
*
|
||||
* If the query executed was the portion of a distributed query running on the
|
||||
* master, remove the resources that were needed for distributed execution.
|
||||
*/
|
||||
void
|
||||
multi_ExecutorEnd(QueryDesc *queryDesc)
|
||||
{
|
||||
int eflags = queryDesc->estate->es_top_eflags;
|
||||
|
||||
if (eflags & EXEC_FLAG_CITUS_ROUTER_EXECUTOR)
|
||||
{
|
||||
/* drop into the router executor */
|
||||
RouterExecutorEnd(queryDesc);
|
||||
}
|
||||
else
|
||||
{
|
||||
/* drop into the standard executor */
|
||||
standard_ExecutorEnd(queryDesc);
|
||||
}
|
||||
scanState->tuplestorestate = NULL;
|
||||
scanState->finishedUnderlyingScan = true;
|
||||
|
||||
/*
|
||||
* Final step of a distributed query is executing the master node select
|
||||
* query. We clean up the temp tables after executing it, if we already created it.
|
||||
* XXX: this probably already works, but if not should be easily
|
||||
* supportable - probably hard to exercise right now though.
|
||||
*/
|
||||
if (eflags & EXEC_FLAG_CITUS_MASTER_SELECT)
|
||||
{
|
||||
PlannedStmt *planStatement = queryDesc->plannedstmt;
|
||||
int savedLogMinMessages = 0;
|
||||
int savedClientMinMessages = 0;
|
||||
|
||||
RangeTblEntry *rangeTableEntry = linitial(planStatement->rtable);
|
||||
Oid masterTableRelid = rangeTableEntry->relid;
|
||||
|
||||
ObjectAddress masterTableObject = { InvalidOid, InvalidOid, 0 };
|
||||
|
||||
masterTableObject.classId = RelationRelationId;
|
||||
masterTableObject.objectId = masterTableRelid;
|
||||
masterTableObject.objectSubId = 0;
|
||||
|
||||
/*
|
||||
* Temporarily change logging level to avoid DEBUG2 logging output by
|
||||
* performDeletion. This avoids breaking the regression tests which
|
||||
* use DEBUG2 logging.
|
||||
*/
|
||||
savedLogMinMessages = log_min_messages;
|
||||
savedClientMinMessages = client_min_messages;
|
||||
|
||||
log_min_messages = INFO;
|
||||
client_min_messages = INFO;
|
||||
|
||||
performDeletion(&masterTableObject, DROP_RESTRICT, PERFORM_DELETION_INTERNAL);
|
||||
|
||||
log_min_messages = savedLogMinMessages;
|
||||
client_min_messages = savedClientMinMessages;
|
||||
}
|
||||
elog(WARNING, "unsupported at this point");
|
||||
}
|
||||
|
|
|
@ -74,83 +74,33 @@ bool EnableDeadlockPrevention = true;
|
|||
|
||||
/* functions needed during run phase */
|
||||
static void ReacquireMetadataLocks(List *taskList);
|
||||
static void ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task,
|
||||
static void ExecuteSingleModifyTask(CitusScanState *scanState, Task *task,
|
||||
bool expectResults);
|
||||
static void ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task);
|
||||
static void ExecuteSingleSelectTask(CitusScanState *scanState, Task *task);
|
||||
static List * GetModifyConnections(List *taskPlacementList,
|
||||
bool markCritical,
|
||||
bool startedInTransaction);
|
||||
static void ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList,
|
||||
static void ExecuteMultipleTasks(CitusScanState *scanState, List *taskList,
|
||||
bool isModificationQuery, bool expectResults);
|
||||
static int64 ExecuteModifyTasks(List *taskList, bool expectResults,
|
||||
ParamListInfo paramListInfo,
|
||||
MaterialState *routerState,
|
||||
CitusScanState *scanState,
|
||||
TupleDesc tupleDescriptor);
|
||||
static List * TaskShardIntervalList(List *taskList);
|
||||
static void AcquireExecutorShardLock(Task *task, CmdType commandType);
|
||||
static void AcquireExecutorMultiShardLocks(List *taskList);
|
||||
static bool RequiresConsistentSnapshot(Task *task);
|
||||
static uint64 ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor,
|
||||
DestReceiver *destination,
|
||||
Tuplestorestate *tupleStore);
|
||||
static void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo,
|
||||
Oid **parameterTypes,
|
||||
const char ***parameterValues);
|
||||
static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query,
|
||||
ParamListInfo paramListInfo);
|
||||
static bool StoreQueryResult(MaterialState *routerState, MultiConnection *connection,
|
||||
static bool StoreQueryResult(CitusScanState *scanState, MultiConnection *connection,
|
||||
TupleDesc tupleDescriptor, bool failOnError, int64 *rows);
|
||||
static bool ConsumeQueryResult(MultiConnection *connection, bool failOnError,
|
||||
int64 *rows);
|
||||
|
||||
|
||||
/*
|
||||
* RouterExecutorStart sets up the executor state and queryDesc for router
|
||||
* execution.
|
||||
*/
|
||||
void
|
||||
RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList)
|
||||
{
|
||||
EState *executorState = NULL;
|
||||
CmdType commandType = queryDesc->operation;
|
||||
|
||||
/*
|
||||
* If we are executing a prepared statement, then we may not yet have obtained
|
||||
* the metadata locks in this transaction. To prevent a concurrent shard copy,
|
||||
* we re-obtain them here or error out if a shard copy has already started.
|
||||
*
|
||||
* If a shard copy finishes in between fetching a plan from cache and
|
||||
* re-acquiring the locks, then we might still run a stale plan, which could
|
||||
* cause shard placements to diverge. To minimize this window, we take the
|
||||
* locks as early as possible.
|
||||
*/
|
||||
ReacquireMetadataLocks(taskList);
|
||||
|
||||
/* disallow triggers during distributed modify commands */
|
||||
if (commandType != CMD_SELECT)
|
||||
{
|
||||
eflags |= EXEC_FLAG_SKIP_TRIGGERS;
|
||||
}
|
||||
|
||||
/* signal that it is a router execution */
|
||||
eflags |= EXEC_FLAG_CITUS_ROUTER_EXECUTOR;
|
||||
|
||||
/* build empty executor state to obtain per-query memory context */
|
||||
executorState = CreateExecutorState();
|
||||
executorState->es_top_eflags = eflags;
|
||||
executorState->es_instrument = queryDesc->instrument_options;
|
||||
|
||||
queryDesc->estate = executorState;
|
||||
|
||||
/*
|
||||
* As it's similar to what we're doing, use a MaterialState node to store
|
||||
* our state. This is used to store our tuplestore, so cursors etc. can
|
||||
* work.
|
||||
*/
|
||||
queryDesc->planstate = (PlanState *) makeNode(MaterialState);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ReacquireMetadataLocks re-acquires the metadata locks that are normally
|
||||
* acquired during planning.
|
||||
|
@ -457,73 +407,47 @@ RequiresConsistentSnapshot(Task *task)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* RouterExecutorRun actually executes a single task on a worker.
|
||||
*/
|
||||
void
|
||||
RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count)
|
||||
RouterBeginScan(CitusScanState *scanState)
|
||||
{
|
||||
PlannedStmt *planStatement = queryDesc->plannedstmt;
|
||||
MultiPlan *multiPlan = GetMultiPlan(planStatement);
|
||||
MultiPlan *multiPlan = scanState->multiPlan;
|
||||
Job *workerJob = multiPlan->workerJob;
|
||||
List *taskList = workerJob->taskList;
|
||||
EState *estate = queryDesc->estate;
|
||||
CmdType operation = queryDesc->operation;
|
||||
MemoryContext oldcontext = NULL;
|
||||
DestReceiver *destination = queryDesc->dest;
|
||||
MaterialState *routerState = (MaterialState *) queryDesc->planstate;
|
||||
bool sendTuples = operation == CMD_SELECT || queryDesc->plannedstmt->hasReturning;
|
||||
|
||||
Assert(estate != NULL);
|
||||
Assert(!(estate->es_top_eflags & EXEC_FLAG_EXPLAIN_ONLY));
|
||||
|
||||
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
|
||||
|
||||
if (queryDesc->totaltime != NULL)
|
||||
{
|
||||
InstrStartNode(queryDesc->totaltime);
|
||||
}
|
||||
|
||||
estate->es_processed = 0;
|
||||
|
||||
/* startup the tuple receiver */
|
||||
if (sendTuples)
|
||||
{
|
||||
(*destination->rStartup)(destination, operation, queryDesc->tupDesc);
|
||||
}
|
||||
|
||||
/* we only support returning nothing or scanning forward */
|
||||
if (ScanDirectionIsNoMovement(direction))
|
||||
{
|
||||
/* comments in PortalRunSelect() explain the reason for this case */
|
||||
goto out;
|
||||
}
|
||||
else if (!ScanDirectionIsForward(direction))
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("scan directions other than forward scans "
|
||||
"are unsupported")));
|
||||
}
|
||||
|
||||
/*
|
||||
* If query has not yet been executed, do so now. The main reason why the
|
||||
* query might already have been executed is cursors.
|
||||
* If we are executing a prepared statement, then we may not yet have obtained
|
||||
* the metadata locks in this transaction. To prevent a concurrent shard copy,
|
||||
* we re-obtain them here or error out if a shard copy has already started.
|
||||
*
|
||||
* If a shard copy finishes in between fetching a plan from cache and
|
||||
* re-acquiring the locks, then we might still run a stale plan, which could
|
||||
* cause shard placements to diverge. To minimize this window, we take the
|
||||
* locks as early as possible.
|
||||
*/
|
||||
if (!routerState->eof_underlying)
|
||||
{
|
||||
bool isModificationQuery = false;
|
||||
bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation;
|
||||
ReacquireMetadataLocks(taskList);
|
||||
}
|
||||
|
||||
|
||||
TupleTableSlot *
|
||||
RouterExecScan(CitusScanState *scanState)
|
||||
{
|
||||
MultiPlan *multiPlan = scanState->multiPlan;
|
||||
TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot;
|
||||
|
||||
if (!scanState->finishedUnderlyingScan)
|
||||
{
|
||||
Job *workerJob = multiPlan->workerJob;
|
||||
List *taskList = workerJob->taskList;
|
||||
bool requiresMasterEvaluation = workerJob->requiresMasterEvaluation;
|
||||
bool isModificationQuery = false;
|
||||
CmdType operation = multiPlan->operation;
|
||||
|
||||
/* should use IsModificationStmt or such */
|
||||
if (operation == CMD_INSERT || operation == CMD_UPDATE ||
|
||||
operation == CMD_DELETE)
|
||||
{
|
||||
isModificationQuery = true;
|
||||
}
|
||||
else if (operation != CMD_SELECT)
|
||||
{
|
||||
ereport(ERROR, (errmsg("unrecognized operation code: %d",
|
||||
(int) operation)));
|
||||
}
|
||||
|
||||
if (requiresMasterEvaluation)
|
||||
{
|
||||
|
@ -539,59 +463,42 @@ RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count)
|
|||
|
||||
if (isModificationQuery)
|
||||
{
|
||||
ExecuteSingleModifyTask(queryDesc, task, sendTuples);
|
||||
bool sendTuples = multiPlan->hasReturning;
|
||||
ExecuteSingleModifyTask(scanState, task, sendTuples);
|
||||
}
|
||||
else
|
||||
{
|
||||
ExecuteSingleSelectTask(queryDesc, task);
|
||||
ExecuteSingleSelectTask(scanState, task);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
ExecuteMultipleTasks(queryDesc, taskList, isModificationQuery,
|
||||
bool sendTuples = multiPlan->hasReturning;
|
||||
ExecuteMultipleTasks(scanState, taskList, isModificationQuery,
|
||||
sendTuples);
|
||||
}
|
||||
|
||||
/* mark underlying query as having executed */
|
||||
routerState->eof_underlying = true;
|
||||
scanState->finishedUnderlyingScan = true;
|
||||
}
|
||||
|
||||
/* if the underlying query produced output, return it */
|
||||
if (routerState->tuplestorestate != NULL)
|
||||
|
||||
/*
|
||||
* FIXME: centralize this into function to be shared between router and
|
||||
* other executors?
|
||||
*/
|
||||
if (scanState->tuplestorestate != NULL)
|
||||
{
|
||||
TupleDesc resultTupleDescriptor = queryDesc->tupDesc;
|
||||
int64 returnedRows = 0;
|
||||
Tuplestorestate *tupleStore = scanState->tuplestorestate;
|
||||
|
||||
/* return rows from the tuplestore */
|
||||
returnedRows = ReturnRowsFromTuplestore(count, resultTupleDescriptor,
|
||||
destination,
|
||||
routerState->tuplestorestate);
|
||||
/* XXX: could trivially support backward scans here */
|
||||
tuplestore_gettupleslot(tupleStore, true, false, resultSlot);
|
||||
|
||||
/*
|
||||
* Count tuples processed, if this is a SELECT. (For modifications
|
||||
* it'll already have been increased, as we want the number of
|
||||
* modified tuples, not the number of RETURNed tuples.)
|
||||
*/
|
||||
if (operation == CMD_SELECT)
|
||||
{
|
||||
estate->es_processed += returnedRows;
|
||||
}
|
||||
return resultSlot;
|
||||
}
|
||||
|
||||
out:
|
||||
|
||||
/* shutdown tuple receiver, if we started it */
|
||||
if (sendTuples)
|
||||
{
|
||||
(*destination->rShutdown)(destination);
|
||||
}
|
||||
|
||||
if (queryDesc->totaltime != NULL)
|
||||
{
|
||||
InstrStopNode(queryDesc->totaltime, estate->es_processed);
|
||||
}
|
||||
|
||||
MemoryContextSwitchTo(oldcontext);
|
||||
return NULL;
|
||||
}
|
||||
|
||||
|
||||
|
@ -603,11 +510,12 @@ out:
|
|||
* other placements or errors out if the query fails on all placements.
|
||||
*/
|
||||
static void
|
||||
ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task)
|
||||
ExecuteSingleSelectTask(CitusScanState *scanState, Task *task)
|
||||
{
|
||||
TupleDesc tupleDescriptor = queryDesc->tupDesc;
|
||||
MaterialState *routerState = (MaterialState *) queryDesc->planstate;
|
||||
ParamListInfo paramListInfo = queryDesc->params;
|
||||
TupleDesc tupleDescriptor =
|
||||
scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
|
||||
ParamListInfo paramListInfo =
|
||||
scanState->customScanState.ss.ps.state->es_param_list_info;
|
||||
List *taskPlacementList = task->taskPlacementList;
|
||||
ListCell *taskPlacementCell = NULL;
|
||||
char *queryString = task->queryString;
|
||||
|
@ -639,7 +547,7 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task)
|
|||
continue;
|
||||
}
|
||||
|
||||
queryOK = StoreQueryResult(routerState, connection, tupleDescriptor,
|
||||
queryOK = StoreQueryResult(scanState, connection, tupleDescriptor,
|
||||
dontFailOnError, ¤tAffectedTupleCount);
|
||||
if (queryOK)
|
||||
{
|
||||
|
@ -661,14 +569,14 @@ ExecuteSingleSelectTask(QueryDesc *queryDesc, Task *task)
|
|||
* framework), or errors out (failed on all placements).
|
||||
*/
|
||||
static void
|
||||
ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task,
|
||||
ExecuteSingleModifyTask(CitusScanState *scanState, Task *task,
|
||||
bool expectResults)
|
||||
{
|
||||
CmdType operation = queryDesc->operation;
|
||||
TupleDesc tupleDescriptor = queryDesc->tupDesc;
|
||||
EState *executorState = queryDesc->estate;
|
||||
MaterialState *routerState = (MaterialState *) queryDesc->planstate;
|
||||
ParamListInfo paramListInfo = queryDesc->params;
|
||||
CmdType operation = scanState->multiPlan->operation;
|
||||
TupleDesc tupleDescriptor =
|
||||
scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
|
||||
EState *executorState = scanState->customScanState.ss.ps.state;
|
||||
ParamListInfo paramListInfo = executorState->es_param_list_info;
|
||||
bool resultsOK = false;
|
||||
List *taskPlacementList = task->taskPlacementList;
|
||||
List *connectionList = NIL;
|
||||
|
@ -761,7 +669,7 @@ ExecuteSingleModifyTask(QueryDesc *queryDesc, Task *task,
|
|||
*/
|
||||
if (!gotResults && expectResults)
|
||||
{
|
||||
queryOK = StoreQueryResult(routerState, connection, tupleDescriptor,
|
||||
queryOK = StoreQueryResult(scanState, connection, tupleDescriptor,
|
||||
failOnError, ¤tAffectedTupleCount);
|
||||
}
|
||||
else
|
||||
|
@ -893,20 +801,21 @@ GetModifyConnections(List *taskPlacementList, bool markCritical, bool noNewTrans
|
|||
* commits.
|
||||
*/
|
||||
static void
|
||||
ExecuteMultipleTasks(QueryDesc *queryDesc, List *taskList,
|
||||
ExecuteMultipleTasks(CitusScanState *scanState, List *taskList,
|
||||
bool isModificationQuery, bool expectResults)
|
||||
{
|
||||
TupleDesc tupleDescriptor = queryDesc->tupDesc;
|
||||
EState *executorState = queryDesc->estate;
|
||||
MaterialState *routerState = (MaterialState *) queryDesc->planstate;
|
||||
ParamListInfo paramListInfo = queryDesc->params;
|
||||
TupleDesc tupleDescriptor =
|
||||
scanState->customScanState.ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor;
|
||||
EState *executorState = scanState->customScanState.ss.ps.state;
|
||||
ParamListInfo paramListInfo = executorState->es_param_list_info;
|
||||
int64 affectedTupleCount = -1;
|
||||
|
||||
/* can only support modifications right now */
|
||||
Assert(isModificationQuery);
|
||||
|
||||
/* XXX: Seems very redundant to pass both scanState and tupleDescriptor */
|
||||
affectedTupleCount = ExecuteModifyTasks(taskList, expectResults, paramListInfo,
|
||||
routerState, tupleDescriptor);
|
||||
scanState, tupleDescriptor);
|
||||
|
||||
executorState->es_processed = affectedTupleCount;
|
||||
}
|
||||
|
@ -936,7 +845,7 @@ ExecuteModifyTasksWithoutResults(List *taskList)
|
|||
*/
|
||||
static int64
|
||||
ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListInfo,
|
||||
MaterialState *routerState, TupleDesc tupleDescriptor)
|
||||
CitusScanState *scanState, TupleDesc tupleDescriptor)
|
||||
{
|
||||
int64 totalAffectedTupleCount = 0;
|
||||
ListCell *taskCell = NULL;
|
||||
|
@ -1066,9 +975,9 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn
|
|||
*/
|
||||
if (placementIndex == 0 && expectResults)
|
||||
{
|
||||
Assert(routerState != NULL && tupleDescriptor != NULL);
|
||||
Assert(scanState != NULL && tupleDescriptor != NULL);
|
||||
|
||||
queryOK = StoreQueryResult(routerState, connection, tupleDescriptor,
|
||||
queryOK = StoreQueryResult(scanState, connection, tupleDescriptor,
|
||||
failOnError, ¤tAffectedTupleCount);
|
||||
}
|
||||
else
|
||||
|
@ -1149,50 +1058,6 @@ TaskShardIntervalList(List *taskList)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* ReturnRowsFromTuplestore moves rows from a given tuplestore into a
|
||||
* receiver. It performs the necessary limiting to support cursors.
|
||||
*/
|
||||
static uint64
|
||||
ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor,
|
||||
DestReceiver *destination, Tuplestorestate *tupleStore)
|
||||
{
|
||||
TupleTableSlot *tupleTableSlot = NULL;
|
||||
uint64 currentTupleCount = 0;
|
||||
|
||||
tupleTableSlot = MakeSingleTupleTableSlot(tupleDescriptor);
|
||||
|
||||
/* iterate over tuples in tuple store, and send them to destination */
|
||||
for (;;)
|
||||
{
|
||||
bool nextTuple = tuplestore_gettupleslot(tupleStore, true, false, tupleTableSlot);
|
||||
if (!nextTuple)
|
||||
{
|
||||
break;
|
||||
}
|
||||
|
||||
(*destination->receiveSlot)(tupleTableSlot, destination);
|
||||
|
||||
ExecClearTuple(tupleTableSlot);
|
||||
|
||||
currentTupleCount++;
|
||||
|
||||
/*
|
||||
* If numberTuples is zero fetch all tuples, otherwise stop after
|
||||
* count tuples.
|
||||
*/
|
||||
if (tupleCount > 0 && tupleCount == currentTupleCount)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
ExecDropSingleTupleTableSlot(tupleTableSlot);
|
||||
|
||||
return currentTupleCount;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* SendQueryInSingleRowMode sends the given query on the connection in an
|
||||
* asynchronous way. The function also sets the single-row mode on the
|
||||
|
@ -1318,12 +1183,13 @@ ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterT
|
|||
* the connection.
|
||||
*/
|
||||
static bool
|
||||
StoreQueryResult(MaterialState *routerState, MultiConnection *connection,
|
||||
StoreQueryResult(CitusScanState *scanState, MultiConnection *connection,
|
||||
TupleDesc tupleDescriptor, bool failOnError, int64 *rows)
|
||||
{
|
||||
AttInMetadata *attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor);
|
||||
Tuplestorestate *tupleStore = NULL;
|
||||
uint32 expectedColumnCount = tupleDescriptor->natts;
|
||||
List *targetList = scanState->customScanState.ss.ps.plan->targetlist;
|
||||
uint32 expectedColumnCount = ExecCleanTargetListLength(targetList);
|
||||
char **columnArray = (char **) palloc0(expectedColumnCount * sizeof(char *));
|
||||
bool commandFailed = false;
|
||||
MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext,
|
||||
|
@ -1333,17 +1199,17 @@ StoreQueryResult(MaterialState *routerState, MultiConnection *connection,
|
|||
ALLOCSET_DEFAULT_MAXSIZE);
|
||||
*rows = 0;
|
||||
|
||||
if (routerState->tuplestorestate == NULL)
|
||||
if (scanState->tuplestorestate == NULL)
|
||||
{
|
||||
routerState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem);
|
||||
scanState->tuplestorestate = tuplestore_begin_heap(false, false, work_mem);
|
||||
}
|
||||
else if (!failOnError)
|
||||
{
|
||||
/* might have failed query execution on another placement before */
|
||||
tuplestore_clear(routerState->tuplestorestate);
|
||||
tuplestore_clear(scanState->tuplestorestate);
|
||||
}
|
||||
|
||||
tupleStore = routerState->tuplestorestate;
|
||||
tupleStore = scanState->tuplestorestate;
|
||||
|
||||
for (;;)
|
||||
{
|
||||
|
|
|
@ -369,51 +369,6 @@ multi_ProcessUtility(Node *parsetree,
|
|||
" necessary users and roles.")));
|
||||
}
|
||||
|
||||
/* due to an explain-hook limitation we have to special-case EXPLAIN EXECUTE */
|
||||
if (IsA(parsetree, ExplainStmt) && IsA(((ExplainStmt *) parsetree)->query, Query))
|
||||
{
|
||||
ExplainStmt *explainStmt = (ExplainStmt *) parsetree;
|
||||
Query *query = (Query *) explainStmt->query;
|
||||
|
||||
if (query->commandType == CMD_UTILITY &&
|
||||
IsA(query->utilityStmt, ExecuteStmt))
|
||||
{
|
||||
ExecuteStmt *execstmt = (ExecuteStmt *) query->utilityStmt;
|
||||
PreparedStatement *entry = FetchPreparedStatement(execstmt->name, true);
|
||||
CachedPlanSource *plansource = entry->plansource;
|
||||
Node *parseTreeCopy;
|
||||
Query *originalQuery;
|
||||
|
||||
/* copied from ExplainExecuteQuery, will never trigger if you used PREPARE */
|
||||
if (!plansource->fixed_result)
|
||||
{
|
||||
ereport(ERROR, (errmsg("EXPLAIN EXECUTE does not support variable-result"
|
||||
" cached plans")));
|
||||
}
|
||||
|
||||
parseTreeCopy = copyObject(plansource->raw_parse_tree);
|
||||
|
||||
originalQuery = parse_analyze(parseTreeCopy,
|
||||
plansource->query_string,
|
||||
plansource->param_types,
|
||||
plansource->num_params);
|
||||
|
||||
if (ExtractFirstDistributedTableId(originalQuery) != InvalidOid)
|
||||
{
|
||||
/*
|
||||
* since pg no longer sees EXECUTE it will use the explain hook we've
|
||||
* installed
|
||||
*/
|
||||
explainStmt->query = (Node *) originalQuery;
|
||||
standard_ProcessUtility(parsetree, plansource->query_string, context,
|
||||
params, dest, completionTag);
|
||||
return;
|
||||
}
|
||||
|
||||
/* if this is a normal query fall through to the usual executor */
|
||||
}
|
||||
}
|
||||
|
||||
if (commandMustRunAsOwner)
|
||||
{
|
||||
GetUserIdAndSecContext(&savedUserId, &savedSecurityContext);
|
||||
|
|
|
@ -65,9 +65,6 @@ typedef struct RemoteExplainPlan
|
|||
|
||||
|
||||
/* Explain functions for distributed queries */
|
||||
static void ExplainMasterPlan(PlannedStmt *masterPlan, IntoClause *into,
|
||||
ExplainState *es, const char *queryString,
|
||||
ParamListInfo params, const instr_time *planDuration);
|
||||
static void ExplainJob(Job *job, ExplainState *es);
|
||||
static void ExplainMapMergeJob(MapMergeJob *mapMergeJob, ExplainState *es);
|
||||
static void ExplainTaskList(List *taskList, ExplainState *es);
|
||||
|
@ -77,9 +74,6 @@ static void ExplainTask(Task *task, int placementIndex, List *explainOutputList,
|
|||
static void ExplainTaskPlacement(ShardPlacement *taskPlacement, List *explainOutputList,
|
||||
ExplainState *es);
|
||||
static StringInfo BuildRemoteExplainQuery(char *queryString, ExplainState *es);
|
||||
static void MultiExplainOnePlan(PlannedStmt *plan, IntoClause *into,
|
||||
ExplainState *es, const char *queryString,
|
||||
ParamListInfo params, const instr_time *planDuration);
|
||||
|
||||
/* Static Explain functions copied from explain.c */
|
||||
static void ExplainOpenGroup(const char *objtype, const char *labelname,
|
||||
|
@ -90,290 +84,62 @@ static void ExplainXMLTag(const char *tagname, int flags, ExplainState *es);
|
|||
static void ExplainJSONLineEnding(ExplainState *es);
|
||||
static void ExplainYAMLLineStarting(ExplainState *es);
|
||||
|
||||
|
||||
/*
|
||||
* MultiExplainOneQuery takes the given query, and checks if the query is local
|
||||
* or distributed. If the query is local, the function runs the standard explain
|
||||
* logic. If the query is distributed, the function looks up configuration and
|
||||
* prints out the distributed logical and physical plans as appropriate.
|
||||
*/
|
||||
void
|
||||
MultiExplainOneQuery(Query *query, IntoClause *into, ExplainState *es,
|
||||
const char *queryString, ParamListInfo params)
|
||||
CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es)
|
||||
{
|
||||
instr_time planStart;
|
||||
instr_time planDuration;
|
||||
int cursorOptions = 0;
|
||||
PlannedStmt *plan = NULL;
|
||||
|
||||
#if PG_VERSION_NUM >= 90600
|
||||
|
||||
/*
|
||||
* Allow parallel plans in 9.6+ unless selecting into a table.
|
||||
* Without this, we're breaking explain for non-Citus plans.
|
||||
*/
|
||||
if (!into)
|
||||
{
|
||||
cursorOptions |= CURSOR_OPT_PARALLEL_OK;
|
||||
}
|
||||
#endif
|
||||
|
||||
/* plan query, just like ExplainOneQuery does */
|
||||
INSTR_TIME_SET_CURRENT(planStart);
|
||||
|
||||
/* plan the query */
|
||||
plan = pg_plan_query(query, cursorOptions, params);
|
||||
|
||||
INSTR_TIME_SET_CURRENT(planDuration);
|
||||
INSTR_TIME_SUBTRACT(planDuration, planStart);
|
||||
|
||||
/* if not a distributed query, use plain explain infrastructure */
|
||||
if (!HasCitusToplevelNode(plan))
|
||||
{
|
||||
/* run it (if needed) and produce output */
|
||||
ExplainOnePlan(plan, into, es, queryString, params, &planDuration);
|
||||
}
|
||||
else
|
||||
{
|
||||
MultiExplainOnePlan(plan, into, es, queryString, params, &planDuration);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* MultiExplainOnePlan explains the plan for an individual distributed query.
|
||||
*/
|
||||
static void
|
||||
MultiExplainOnePlan(PlannedStmt *plan, IntoClause *into,
|
||||
ExplainState *es, const char *queryString,
|
||||
ParamListInfo params, const instr_time *planDuration)
|
||||
{
|
||||
MultiPlan *multiPlan = NULL;
|
||||
CmdType commandType = CMD_UNKNOWN;
|
||||
Job *workerJob = NULL;
|
||||
bool routerExecutablePlan = false;
|
||||
|
||||
commandType = plan->commandType;
|
||||
if (commandType == CMD_INSERT || commandType == CMD_UPDATE ||
|
||||
commandType == CMD_DELETE)
|
||||
{
|
||||
if (es->analyze)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("Using ANALYZE for INSERT/UPDATE/DELETE on "
|
||||
"distributed tables is not supported.")));
|
||||
}
|
||||
}
|
||||
|
||||
multiPlan = GetMultiPlan(plan);
|
||||
|
||||
/* ensure plan is executable */
|
||||
VerifyMultiPlanValidity(multiPlan);
|
||||
CitusScanState *scanState = (CitusScanState *) node;
|
||||
MultiPlan *multiPlan = scanState->multiPlan;
|
||||
const char *executorName = NULL;
|
||||
|
||||
if (!ExplainDistributedQueries)
|
||||
{
|
||||
appendStringInfoSpaces(es->str, es->indent * 2);
|
||||
appendStringInfo(es->str, "explain statements for distributed queries ");
|
||||
appendStringInfo(es->str, "are not enabled\n");
|
||||
return;
|
||||
}
|
||||
|
||||
ExplainOpenGroup("Distributed Query", NULL, true, es);
|
||||
|
||||
if (es->format == EXPLAIN_FORMAT_TEXT)
|
||||
{
|
||||
appendStringInfoSpaces(es->str, es->indent * 2);
|
||||
appendStringInfo(es->str, "Distributed Query");
|
||||
|
||||
if (multiPlan->masterTableName != NULL)
|
||||
{
|
||||
appendStringInfo(es->str, " into %s", multiPlan->masterTableName);
|
||||
}
|
||||
|
||||
appendStringInfo(es->str, "\n");
|
||||
|
||||
es->indent += 1;
|
||||
}
|
||||
|
||||
routerExecutablePlan = multiPlan->routerExecutable;
|
||||
|
||||
if (routerExecutablePlan)
|
||||
{
|
||||
ExplainPropertyText("Executor", "Router", es);
|
||||
}
|
||||
else
|
||||
{
|
||||
switch (TaskExecutorType)
|
||||
{
|
||||
case MULTI_EXECUTOR_REAL_TIME:
|
||||
{
|
||||
ExplainPropertyText("Executor", "Real-Time", es);
|
||||
}
|
||||
break;
|
||||
|
||||
case MULTI_EXECUTOR_TASK_TRACKER:
|
||||
{
|
||||
ExplainPropertyText("Executor", "Task-Tracker", es);
|
||||
}
|
||||
break;
|
||||
|
||||
default:
|
||||
{
|
||||
ExplainPropertyText("Executor", "Other", es);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
workerJob = multiPlan->workerJob;
|
||||
ExplainJob(workerJob, es);
|
||||
|
||||
if (es->format == EXPLAIN_FORMAT_TEXT)
|
||||
{
|
||||
es->indent -= 1;
|
||||
}
|
||||
|
||||
if (!routerExecutablePlan)
|
||||
{
|
||||
if (es->format == EXPLAIN_FORMAT_TEXT)
|
||||
{
|
||||
appendStringInfoSpaces(es->str, es->indent * 2);
|
||||
appendStringInfo(es->str, "Master Query\n");
|
||||
es->indent += 1;
|
||||
}
|
||||
|
||||
ExplainOpenGroup("Master Query", "Master Query", false, es);
|
||||
|
||||
ExplainMasterPlan(plan, into, es, queryString, params, planDuration);
|
||||
|
||||
ExplainCloseGroup("Master Query", "Master Query", false, es);
|
||||
|
||||
if (es->format == EXPLAIN_FORMAT_TEXT)
|
||||
{
|
||||
es->indent -= 1;
|
||||
}
|
||||
}
|
||||
|
||||
ExplainCloseGroup("Distributed Query", NULL, true, es);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExplainMasterPlan generates EXPLAIN output for the master query that merges results.
|
||||
* When using EXPLAIN ANALYZE, this function shows the execution time of the master query
|
||||
* in isolation. Calling ExplainOnePlan directly would show the overall execution time of
|
||||
* the distributed query, which makes it hard to determine how much time the master query
|
||||
* took.
|
||||
*
|
||||
* Parts of this function are copied directly from ExplainOnePlan.
|
||||
*/
|
||||
static void
|
||||
ExplainMasterPlan(PlannedStmt *masterPlan, IntoClause *into,
|
||||
ExplainState *es, const char *queryString,
|
||||
ParamListInfo params, const instr_time *planDuration)
|
||||
{
|
||||
DestReceiver *dest = NULL;
|
||||
int eflags = 0;
|
||||
QueryDesc *queryDesc = NULL;
|
||||
int instrument_option = 0;
|
||||
|
||||
if (es->analyze && es->timing)
|
||||
{
|
||||
instrument_option |= INSTRUMENT_TIMER;
|
||||
}
|
||||
else if (es->analyze)
|
||||
{
|
||||
instrument_option |= INSTRUMENT_ROWS;
|
||||
}
|
||||
|
||||
if (es->buffers)
|
||||
{
|
||||
instrument_option |= INSTRUMENT_BUFFERS;
|
||||
}
|
||||
/*
|
||||
* XXX: can we get by without the open/close group somehow - then we'd not
|
||||
* copy any code from explain.c? Seems unlikely.
|
||||
*/
|
||||
ExplainOpenGroup("Distributed Query", "Distributed Query", true, es);
|
||||
|
||||
/*
|
||||
* Use a snapshot with an updated command ID to ensure this query sees
|
||||
* results of any previously executed queries.
|
||||
* XXX: might be worthwhile to put this somewhere central, e.g. for
|
||||
* debugging output.
|
||||
*/
|
||||
PushCopiedSnapshot(GetActiveSnapshot());
|
||||
UpdateActiveSnapshotCommandId();
|
||||
|
||||
/*
|
||||
* Normally we discard the query's output, but if explaining CREATE TABLE
|
||||
* AS, we'd better use the appropriate tuple receiver.
|
||||
*/
|
||||
if (into)
|
||||
switch (scanState->executorType)
|
||||
{
|
||||
dest = CreateIntoRelDestReceiver(into);
|
||||
}
|
||||
else
|
||||
{
|
||||
dest = None_Receiver;
|
||||
}
|
||||
|
||||
/* Create a QueryDesc for the query */
|
||||
queryDesc = CreateQueryDesc(masterPlan, queryString,
|
||||
GetActiveSnapshot(), InvalidSnapshot,
|
||||
dest, params, instrument_option);
|
||||
|
||||
/* Select execution options */
|
||||
if (es->analyze)
|
||||
{
|
||||
eflags = 0; /* default run-to-completion flags */
|
||||
}
|
||||
else
|
||||
{
|
||||
eflags = EXEC_FLAG_EXPLAIN_ONLY;
|
||||
}
|
||||
if (into)
|
||||
{
|
||||
eflags |= GetIntoRelEFlags(into);
|
||||
}
|
||||
|
||||
/*
|
||||
* ExecutorStart creates the merge table. If using ANALYZE, it also executes the
|
||||
* worker job and populates the merge table.
|
||||
*/
|
||||
ExecutorStart(queryDesc, eflags);
|
||||
|
||||
if (es->analyze)
|
||||
{
|
||||
ScanDirection dir;
|
||||
|
||||
/* if using analyze, then finish query execution */
|
||||
|
||||
/* EXPLAIN ANALYZE CREATE TABLE AS WITH NO DATA is weird */
|
||||
if (into && into->skipData)
|
||||
case MULTI_EXECUTOR_ROUTER:
|
||||
{
|
||||
dir = NoMovementScanDirection;
|
||||
executorName = "Router";
|
||||
}
|
||||
else
|
||||
break;
|
||||
|
||||
case MULTI_EXECUTOR_REAL_TIME:
|
||||
{
|
||||
dir = ForwardScanDirection;
|
||||
executorName = "Real-Time";
|
||||
}
|
||||
break;
|
||||
|
||||
/* run the plan */
|
||||
ExecutorRun(queryDesc, dir, 0L);
|
||||
case MULTI_EXECUTOR_TASK_TRACKER:
|
||||
{
|
||||
executorName = "Task-Tracker";
|
||||
}
|
||||
break;
|
||||
|
||||
/* run cleanup too */
|
||||
ExecutorFinish(queryDesc);
|
||||
default:
|
||||
{
|
||||
executorName = "Other";
|
||||
}
|
||||
break;
|
||||
}
|
||||
ExplainPropertyText("Executor", executorName, es);
|
||||
|
||||
/*
|
||||
* ExplainOnePlan executes the master query again, which ensures that the execution
|
||||
* time only shows the execution time of the master query itself, instead of the
|
||||
* overall execution time.
|
||||
*/
|
||||
ExplainOnePlan(queryDesc->plannedstmt, into, es, queryString, params, planDuration);
|
||||
ExplainJob(multiPlan->workerJob, es);
|
||||
|
||||
/*
|
||||
* ExecutorEnd for the distributed query is deferred until after the master query
|
||||
* is executed again, otherwise the merge table would be dropped.
|
||||
*/
|
||||
ExecutorEnd(queryDesc);
|
||||
|
||||
FreeQueryDesc(queryDesc);
|
||||
|
||||
PopActiveSnapshot();
|
||||
ExplainCloseGroup("Distributed Query", "Distributed Query", true, es);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@
|
|||
* a target target list for the master node. This master target list keeps the
|
||||
* temporary table's columns on the master node.
|
||||
*/
|
||||
static List *
|
||||
List *
|
||||
MasterTargetList(List *workerTargetList)
|
||||
{
|
||||
List *masterTargetList = NIL;
|
||||
|
@ -67,49 +67,6 @@ MasterTargetList(List *workerTargetList)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* BuildCreateStatement builds the executable create statement for creating a
|
||||
* temporary table on the master; and then returns this create statement. This
|
||||
* function obtains the needed column type information from the target list.
|
||||
*/
|
||||
static CreateStmt *
|
||||
BuildCreateStatement(char *masterTableName, List *masterTargetList,
|
||||
List *masterColumnNameList)
|
||||
{
|
||||
CreateStmt *createStatement = NULL;
|
||||
RangeVar *relation = NULL;
|
||||
char *relationName = NULL;
|
||||
List *columnTypeList = NIL;
|
||||
List *columnDefinitionList = NIL;
|
||||
ListCell *masterTargetCell = NULL;
|
||||
|
||||
/* build rangevar object for temporary table */
|
||||
relationName = masterTableName;
|
||||
relation = makeRangeVar(NULL, relationName, -1);
|
||||
relation->relpersistence = RELPERSISTENCE_TEMP;
|
||||
|
||||
/* build the list of column types as cstrings */
|
||||
foreach(masterTargetCell, masterTargetList)
|
||||
{
|
||||
TargetEntry *targetEntry = (TargetEntry *) lfirst(masterTargetCell);
|
||||
Var *column = (Var *) targetEntry->expr;
|
||||
Oid columnTypeId = exprType((Node *) column);
|
||||
int32 columnTypeMod = exprTypmod((Node *) column);
|
||||
|
||||
char *columnTypeName = format_type_with_typemod(columnTypeId, columnTypeMod);
|
||||
columnTypeList = lappend(columnTypeList, columnTypeName);
|
||||
}
|
||||
|
||||
/* build the column definition list */
|
||||
columnDefinitionList = ColumnDefinitionList(masterColumnNameList, columnTypeList);
|
||||
|
||||
/* build the create statement */
|
||||
createStatement = CreateStatement(relation, columnDefinitionList);
|
||||
|
||||
return createStatement;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* BuildAggregatePlan creates and returns an aggregate plan. This aggregate plan
|
||||
* builds aggreation and grouping operators (if any) that are to be executed on
|
||||
|
@ -213,14 +170,22 @@ BuildAggregatePlan(Query *masterQuery, Plan *subPlan)
|
|||
*/
|
||||
static PlannedStmt *
|
||||
BuildSelectStatement(Query *masterQuery, char *masterTableName,
|
||||
List *masterTargetList)
|
||||
List *masterTargetList, CustomScan *dataScan)
|
||||
{
|
||||
PlannedStmt *selectStatement = NULL;
|
||||
RangeTblEntry *rangeTableEntry = NULL;
|
||||
RangeTblEntry *queryRangeTableEntry = NULL;
|
||||
SeqScan *sequentialScan = NULL;
|
||||
Agg *aggregationPlan = NULL;
|
||||
Plan *topLevelPlan = NULL;
|
||||
ListCell *lc = NULL;
|
||||
List *columnNames = NULL;
|
||||
|
||||
/* (0) compute column names */
|
||||
foreach(lc, masterTargetList)
|
||||
{
|
||||
TargetEntry *te = lfirst(lc);
|
||||
columnNames = lappend(columnNames, makeString(te->resname));
|
||||
}
|
||||
|
||||
/* (1) make PlannedStmt and set basic information */
|
||||
selectStatement = makeNode(PlannedStmt);
|
||||
|
@ -233,9 +198,8 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName,
|
|||
queryRangeTableEntry = (RangeTblEntry *) linitial(masterQuery->rtable);
|
||||
|
||||
rangeTableEntry = copyObject(queryRangeTableEntry);
|
||||
rangeTableEntry->rtekind = RTE_RELATION;
|
||||
rangeTableEntry->eref = makeAlias(masterTableName, NIL);
|
||||
rangeTableEntry->relid = 0; /* to be filled in exec_Start */
|
||||
rangeTableEntry->rtekind = RTE_VALUES; /* can't look up relation */
|
||||
rangeTableEntry->eref = makeAlias("remote scan", columnNames);
|
||||
rangeTableEntry->inh = false;
|
||||
rangeTableEntry->inFromCl = true;
|
||||
|
||||
|
@ -243,22 +207,21 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName,
|
|||
selectStatement->rtable = list_make1(rangeTableEntry);
|
||||
|
||||
/* (2) build and initialize sequential scan node */
|
||||
sequentialScan = makeNode(SeqScan);
|
||||
sequentialScan->scanrelid = 1; /* always one */
|
||||
/* Gone */
|
||||
|
||||
/* (3) add an aggregation plan if needed */
|
||||
if (masterQuery->hasAggs || masterQuery->groupClause)
|
||||
{
|
||||
sequentialScan->plan.targetlist = masterTargetList;
|
||||
dataScan->scan.plan.targetlist = masterTargetList;
|
||||
|
||||
aggregationPlan = BuildAggregatePlan(masterQuery, (Plan *) sequentialScan);
|
||||
aggregationPlan = BuildAggregatePlan(masterQuery, &dataScan->scan.plan);
|
||||
topLevelPlan = (Plan *) aggregationPlan;
|
||||
}
|
||||
else
|
||||
{
|
||||
/* otherwise set the final projections on the scan plan directly */
|
||||
sequentialScan->plan.targetlist = masterQuery->targetList;
|
||||
topLevelPlan = (Plan *) sequentialScan;
|
||||
dataScan->scan.plan.targetlist = masterQuery->targetList;
|
||||
topLevelPlan = &dataScan->scan.plan;
|
||||
}
|
||||
|
||||
/* (4) add a sorting plan if needed */
|
||||
|
@ -303,54 +266,6 @@ BuildSelectStatement(Query *masterQuery, char *masterTableName,
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* ValueToStringList walks over the given list of string value types, converts
|
||||
* value types to cstrings, and adds these cstrings into a new list.
|
||||
*/
|
||||
static List *
|
||||
ValueToStringList(List *valueList)
|
||||
{
|
||||
List *stringList = NIL;
|
||||
ListCell *valueCell = NULL;
|
||||
|
||||
foreach(valueCell, valueList)
|
||||
{
|
||||
Value *value = (Value *) lfirst(valueCell);
|
||||
char *stringValue = strVal(value);
|
||||
|
||||
stringList = lappend(stringList, stringValue);
|
||||
}
|
||||
|
||||
return stringList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* MasterNodeCreateStatement takes in a multi plan, and constructs a statement
|
||||
* to create a temporary table on the master node for final result
|
||||
* aggregation.
|
||||
*/
|
||||
CreateStmt *
|
||||
MasterNodeCreateStatement(MultiPlan *multiPlan)
|
||||
{
|
||||
Query *masterQuery = multiPlan->masterQuery;
|
||||
Job *workerJob = multiPlan->workerJob;
|
||||
List *workerTargetList = workerJob->jobQuery->targetList;
|
||||
List *rangeTableList = masterQuery->rtable;
|
||||
char *tableName = multiPlan->masterTableName;
|
||||
CreateStmt *createStatement = NULL;
|
||||
|
||||
RangeTblEntry *rangeTableEntry = (RangeTblEntry *) linitial(rangeTableList);
|
||||
List *columnNameValueList = rangeTableEntry->eref->colnames;
|
||||
List *columnNameList = ValueToStringList(columnNameValueList);
|
||||
List *targetList = MasterTargetList(workerTargetList);
|
||||
|
||||
createStatement = BuildCreateStatement(tableName, targetList, columnNameList);
|
||||
|
||||
return createStatement;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* MasterNodeSelectPlan takes in a distributed plan, finds the master node query
|
||||
* structure in that plan, and builds the final select plan to execute on the
|
||||
|
@ -358,7 +273,7 @@ MasterNodeCreateStatement(MultiPlan *multiPlan)
|
|||
* retrieved from worker nodes and are merged into a temporary table.
|
||||
*/
|
||||
PlannedStmt *
|
||||
MasterNodeSelectPlan(MultiPlan *multiPlan)
|
||||
MasterNodeSelectPlan(MultiPlan *multiPlan, CustomScan *dataScan)
|
||||
{
|
||||
Query *masterQuery = multiPlan->masterQuery;
|
||||
char *tableName = multiPlan->masterTableName;
|
||||
|
@ -368,49 +283,8 @@ MasterNodeSelectPlan(MultiPlan *multiPlan)
|
|||
List *workerTargetList = workerJob->jobQuery->targetList;
|
||||
List *masterTargetList = MasterTargetList(workerTargetList);
|
||||
|
||||
masterSelectPlan = BuildSelectStatement(masterQuery, tableName, masterTargetList);
|
||||
masterSelectPlan =
|
||||
BuildSelectStatement(masterQuery, tableName, masterTargetList, dataScan);
|
||||
|
||||
return masterSelectPlan;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* MasterNodeCopyStatementList takes in a multi plan, and constructs
|
||||
* statements that copy over worker task results to a temporary table on the
|
||||
* master node.
|
||||
*/
|
||||
List *
|
||||
MasterNodeCopyStatementList(MultiPlan *multiPlan)
|
||||
{
|
||||
Job *workerJob = multiPlan->workerJob;
|
||||
List *workerTaskList = workerJob->taskList;
|
||||
char *tableName = multiPlan->masterTableName;
|
||||
List *copyStatementList = NIL;
|
||||
|
||||
ListCell *workerTaskCell = NULL;
|
||||
foreach(workerTaskCell, workerTaskList)
|
||||
{
|
||||
Task *workerTask = (Task *) lfirst(workerTaskCell);
|
||||
StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId);
|
||||
StringInfo taskFilename = TaskFilename(jobDirectoryName, workerTask->taskId);
|
||||
|
||||
RangeVar *relation = makeRangeVar(NULL, tableName, -1);
|
||||
CopyStmt *copyStatement = makeNode(CopyStmt);
|
||||
copyStatement->relation = relation;
|
||||
copyStatement->is_from = true;
|
||||
copyStatement->filename = taskFilename->data;
|
||||
if (BinaryMasterCopyFormat)
|
||||
{
|
||||
DefElem *copyOption = makeDefElem("format", (Node *) makeString("binary"));
|
||||
copyStatement->options = list_make1(copyOption);
|
||||
}
|
||||
else
|
||||
{
|
||||
copyStatement->options = NIL;
|
||||
}
|
||||
|
||||
copyStatementList = lappend(copyStatementList, copyStatement);
|
||||
}
|
||||
|
||||
return copyStatementList;
|
||||
}
|
||||
|
|
|
@ -221,6 +221,7 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree)
|
|||
multiPlan->masterQuery = masterQuery;
|
||||
multiPlan->masterTableName = jobSchemaName->data;
|
||||
multiPlan->routerExecutable = MultiPlanRouterExecutable(multiPlan);
|
||||
multiPlan->operation = CMD_SELECT;
|
||||
|
||||
return multiPlan;
|
||||
}
|
||||
|
|
|
@ -16,10 +16,12 @@
|
|||
#include "distributed/citus_nodefuncs.h"
|
||||
#include "distributed/citus_nodes.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_planner.h"
|
||||
#include "distributed/multi_logical_optimizer.h"
|
||||
#include "distributed/multi_logical_planner.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/multi_master_planner.h"
|
||||
#include "distributed/multi_router_planner.h"
|
||||
|
||||
#include "executor/executor.h"
|
||||
|
@ -37,7 +39,6 @@ static List *relationRestrictionContextList = NIL;
|
|||
|
||||
/* local function forward declarations */
|
||||
static void CheckNodeIsDumpable(Node *node);
|
||||
static char * GetMultiPlanString(PlannedStmt *result);
|
||||
static PlannedStmt * MultiQueryContainerNode(PlannedStmt *result,
|
||||
struct MultiPlan *multiPlan);
|
||||
static struct PlannedStmt * CreateDistributedPlan(PlannedStmt *localPlan,
|
||||
|
@ -293,19 +294,25 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query
|
|||
}
|
||||
|
||||
|
||||
static CustomScanMethods CitusCustomScanMethods = {
|
||||
"CitusScan",
|
||||
CitusCreateScan
|
||||
};
|
||||
|
||||
|
||||
/*
|
||||
* GetMultiPlan returns the associated MultiPlan for a PlannedStmt if the
|
||||
* statement requires distributed execution, NULL otherwise.
|
||||
* GetMultiPlan returns the associated MultiPlan for a CustomScan.
|
||||
*/
|
||||
MultiPlan *
|
||||
GetMultiPlan(PlannedStmt *result)
|
||||
GetMultiPlan(CustomScan *customScan)
|
||||
{
|
||||
char *serializedMultiPlan = NULL;
|
||||
MultiPlan *multiPlan = NULL;
|
||||
|
||||
serializedMultiPlan = GetMultiPlanString(result);
|
||||
multiPlan = (MultiPlan *) CitusStringToNode(serializedMultiPlan);
|
||||
Assert(CitusIsA(multiPlan, MultiPlan));
|
||||
Assert(IsA(customScan, CustomScan));
|
||||
Assert(customScan->methods == &CitusCustomScanMethods);
|
||||
Assert(list_length(customScan->custom_private) == 1);
|
||||
|
||||
multiPlan = DeSerializeMultiPlan(linitial(customScan->custom_private));
|
||||
|
||||
return multiPlan;
|
||||
}
|
||||
|
@ -315,24 +322,49 @@ GetMultiPlan(PlannedStmt *result)
|
|||
bool
|
||||
HasCitusToplevelNode(PlannedStmt *result)
|
||||
{
|
||||
/*
|
||||
* Can't be a distributed query if the extension hasn't been loaded
|
||||
* yet. Directly return false, part of the required infrastructure for
|
||||
* further checks might not be present.
|
||||
*/
|
||||
if (!CitusHasBeenLoaded())
|
||||
{
|
||||
return false;
|
||||
}
|
||||
elog(ERROR, "gone");
|
||||
}
|
||||
|
||||
if (GetMultiPlanString(result) == NULL)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
Node *
|
||||
SerializableMultiPlan(MultiPlan *multiPlan)
|
||||
{
|
||||
/*
|
||||
* FIXME: This should be improved for 9.6+, we we can copy trees
|
||||
* efficiently. I.e. we should introduce copy support for relevant node
|
||||
* types, and just return the MultiPlan as-is for 9.6.
|
||||
*/
|
||||
char *serializedPlan = NULL;
|
||||
Const *multiPlanData = NULL;
|
||||
|
||||
serializedPlan = CitusNodeToString(multiPlan);
|
||||
|
||||
multiPlanData = makeNode(Const);
|
||||
multiPlanData->consttype = CSTRINGOID;
|
||||
multiPlanData->constlen = strlen(serializedPlan);
|
||||
multiPlanData->constvalue = CStringGetDatum(serializedPlan);
|
||||
multiPlanData->constbyval = false;
|
||||
multiPlanData->location = -1;
|
||||
|
||||
return (Node *) multiPlanData;
|
||||
}
|
||||
|
||||
|
||||
MultiPlan *
|
||||
DeSerializeMultiPlan(Node *node)
|
||||
{
|
||||
Const *multiPlanData = NULL;
|
||||
char *serializedMultiPlan = NULL;
|
||||
MultiPlan *multiPlan = NULL;
|
||||
|
||||
Assert(IsA(node, Const));
|
||||
multiPlanData = (Const *) node;
|
||||
serializedMultiPlan = DatumGetCString(multiPlanData->constvalue);
|
||||
|
||||
multiPlan = (MultiPlan *) CitusStringToNode(serializedMultiPlan);
|
||||
Assert(CitusIsA(multiPlan, MultiPlan));
|
||||
|
||||
return multiPlan;
|
||||
}
|
||||
|
||||
|
||||
|
@ -346,124 +378,98 @@ HasCitusToplevelNode(PlannedStmt *result)
|
|||
* which should not be referred to outside this file, as it's likely to become
|
||||
* version dependant. Use GetMultiPlan() and HasCitusToplevelNode() to access.
|
||||
*
|
||||
* FIXME
|
||||
*
|
||||
* Internally the data is stored as arguments to a 'citus_extradata_container'
|
||||
* function, which has to be removed from the really executed plan tree before
|
||||
* query execution.
|
||||
*/
|
||||
PlannedStmt *
|
||||
MultiQueryContainerNode(PlannedStmt *result, MultiPlan *multiPlan)
|
||||
MultiQueryContainerNode(PlannedStmt *originalPlan, MultiPlan *multiPlan)
|
||||
{
|
||||
FunctionScan *fauxFunctionScan = NULL;
|
||||
RangeTblFunction *fauxFunction = NULL;
|
||||
FuncExpr *fauxFuncExpr = NULL;
|
||||
Const *multiPlanData = NULL;
|
||||
char *serializedPlan = NULL;
|
||||
PlannedStmt *resultPlan = NULL;
|
||||
CustomScan *customScan = makeNode(CustomScan);
|
||||
Node *multiPlanData = SerializableMultiPlan(multiPlan);
|
||||
|
||||
/* pass multiPlan serialized as a constant function argument */
|
||||
serializedPlan = CitusNodeToString(multiPlan);
|
||||
multiPlanData = makeNode(Const);
|
||||
multiPlanData->consttype = CSTRINGOID;
|
||||
multiPlanData->constlen = strlen(serializedPlan);
|
||||
multiPlanData->constvalue = CStringGetDatum(serializedPlan);
|
||||
multiPlanData->constbyval = false;
|
||||
multiPlanData->location = -1;
|
||||
customScan->methods = &CitusCustomScanMethods;
|
||||
customScan->custom_private = list_make1(multiPlanData);
|
||||
|
||||
fauxFuncExpr = makeNode(FuncExpr);
|
||||
fauxFuncExpr->funcid = CitusExtraDataContainerFuncId();
|
||||
fauxFuncExpr->funcretset = true;
|
||||
fauxFuncExpr->location = -1;
|
||||
|
||||
fauxFuncExpr->args = list_make1(multiPlanData);
|
||||
fauxFunction = makeNode(RangeTblFunction);
|
||||
fauxFunction->funcexpr = (Node *) fauxFuncExpr;
|
||||
|
||||
fauxFunctionScan = makeNode(FunctionScan);
|
||||
fauxFunctionScan->functions = lappend(fauxFunctionScan->functions, fauxFunction);
|
||||
|
||||
/* copy original targetlist, accessed for RETURNING queries */
|
||||
fauxFunctionScan->scan.plan.targetlist = copyObject(result->planTree->targetlist);
|
||||
/* FIXME: This probably ain't correct */
|
||||
if (ExecSupportsBackwardScan(originalPlan->planTree))
|
||||
{
|
||||
customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN;
|
||||
}
|
||||
|
||||
/*
|
||||
* Add set returning function to target list if the original (postgres
|
||||
* created) plan doesn't support backward scans; doing so prevents
|
||||
* backward scans being supported by the new plantree as well. This is
|
||||
* ugly as hell, but until we can rely on custom scans (which can signal
|
||||
* this via CUSTOMPATH_SUPPORT_BACKWARD_SCAN), there's not really a pretty
|
||||
* method to achieve this.
|
||||
*
|
||||
* FIXME: This should really be done on the master select plan.
|
||||
* FIXME: these two branches/pieces of code should probably be moved into
|
||||
* router / logical planner code respectively.
|
||||
*/
|
||||
if (!ExecSupportsBackwardScan(result->planTree))
|
||||
if (multiPlan->masterQuery)
|
||||
{
|
||||
FuncExpr *funcExpr = makeNode(FuncExpr);
|
||||
TargetEntry *targetEntry = NULL;
|
||||
bool resjunkAttribute = true;
|
||||
resultPlan = MasterNodeSelectPlan(multiPlan, customScan);
|
||||
resultPlan->queryId = originalPlan->queryId;
|
||||
resultPlan->utilityStmt = originalPlan->utilityStmt;
|
||||
}
|
||||
else
|
||||
{
|
||||
ListCell *lc = NULL;
|
||||
List *targetList = NIL;
|
||||
bool foundJunk = false;
|
||||
RangeTblEntry *rangeTableEntry = NULL;
|
||||
List *columnNames = NIL;
|
||||
int newRTI = list_length(originalPlan->rtable) + 1;
|
||||
|
||||
funcExpr->funcretset = true;
|
||||
/*
|
||||
* XXX: This basically just builds a targetlist to "read" from the
|
||||
* custom scan output.
|
||||
*/
|
||||
foreach(lc, originalPlan->planTree->targetlist)
|
||||
{
|
||||
TargetEntry *te = lfirst(lc);
|
||||
Var *newVar = NULL;
|
||||
TargetEntry *newTargetEntry = NULL;
|
||||
|
||||
targetEntry = makeTargetEntry((Expr *) funcExpr, InvalidAttrNumber, NULL,
|
||||
resjunkAttribute);
|
||||
Assert(IsA(te, TargetEntry));
|
||||
|
||||
fauxFunctionScan->scan.plan.targetlist =
|
||||
lappend(fauxFunctionScan->scan.plan.targetlist,
|
||||
targetEntry);
|
||||
/*
|
||||
* XXX: I can't think of a case where we'd need resjunk stuff at
|
||||
* the toplevel of a router query - all things needing it have
|
||||
* been pushed down.
|
||||
*/
|
||||
if (te->resjunk)
|
||||
{
|
||||
foundJunk = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (foundJunk)
|
||||
{
|
||||
ereport(ERROR, (errmsg("unexpected !junk entry after resjunk entry")));
|
||||
}
|
||||
|
||||
/* build TE pointing to custom scan */
|
||||
newVar = makeVarFromTargetEntry(newRTI, te);
|
||||
newTargetEntry = flatCopyTargetEntry(te);
|
||||
newTargetEntry->expr = (Expr *) newVar;
|
||||
targetList = lappend(targetList, newTargetEntry);
|
||||
|
||||
columnNames = lappend(columnNames, makeString(te->resname));
|
||||
}
|
||||
|
||||
/* XXX: can't think of a better RTE type than VALUES */
|
||||
rangeTableEntry = makeNode(RangeTblEntry);
|
||||
rangeTableEntry->rtekind = RTE_VALUES; /* can't look up relation */
|
||||
rangeTableEntry->eref = makeAlias("remote_scan", columnNames);
|
||||
rangeTableEntry->inh = false;
|
||||
rangeTableEntry->inFromCl = true;
|
||||
|
||||
resultPlan = originalPlan;
|
||||
resultPlan->planTree = (Plan *) customScan;
|
||||
resultPlan->rtable = lappend(resultPlan->rtable, rangeTableEntry);
|
||||
customScan->scan.plan.targetlist = targetList;
|
||||
}
|
||||
|
||||
result->planTree = (Plan *) fauxFunctionScan;
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetMultiPlanString returns either NULL, if the plan is not a distributed
|
||||
* one, or the string representing the distributed plan.
|
||||
*/
|
||||
static char *
|
||||
GetMultiPlanString(PlannedStmt *result)
|
||||
{
|
||||
FunctionScan *fauxFunctionScan = NULL;
|
||||
RangeTblFunction *fauxFunction = NULL;
|
||||
FuncExpr *fauxFuncExpr = NULL;
|
||||
Const *multiPlanData = NULL;
|
||||
|
||||
if (!IsA(result->planTree, FunctionScan))
|
||||
{
|
||||
return NULL;
|
||||
}
|
||||
|
||||
fauxFunctionScan = (FunctionScan *) result->planTree;
|
||||
|
||||
if (list_length(fauxFunctionScan->functions) != 1)
|
||||
{
|
||||
return NULL;
|
||||
}
|
||||
|
||||
fauxFunction = linitial(fauxFunctionScan->functions);
|
||||
|
||||
if (!IsA(fauxFunction->funcexpr, FuncExpr))
|
||||
{
|
||||
return NULL;
|
||||
}
|
||||
|
||||
fauxFuncExpr = (FuncExpr *) fauxFunction->funcexpr;
|
||||
|
||||
if (fauxFuncExpr->funcid != CitusExtraDataContainerFuncId())
|
||||
{
|
||||
return NULL;
|
||||
}
|
||||
|
||||
if (list_length(fauxFuncExpr->args) != 1)
|
||||
{
|
||||
ereport(ERROR, (errmsg("unexpected number of function arguments to "
|
||||
"citus_extradata_container")));
|
||||
}
|
||||
|
||||
multiPlanData = (Const *) linitial(fauxFuncExpr->args);
|
||||
Assert(IsA(multiPlanData, Const));
|
||||
Assert(multiPlanData->consttype == CSTRINGOID);
|
||||
|
||||
return DatumGetCString(multiPlanData->constvalue);
|
||||
return resultPlan;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -195,6 +195,8 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query,
|
|||
List *placementList = NIL;
|
||||
MultiPlan *multiPlan = CitusMakeNode(MultiPlan);
|
||||
|
||||
multiPlan->operation = query->commandType;
|
||||
|
||||
if (commandType == CMD_INSERT || commandType == CMD_UPDATE ||
|
||||
commandType == CMD_DELETE)
|
||||
{
|
||||
|
@ -236,6 +238,7 @@ CreateSingleTaskRouterPlan(Query *originalQuery, Query *query,
|
|||
multiPlan->masterQuery = NULL;
|
||||
multiPlan->masterTableName = NULL;
|
||||
multiPlan->routerExecutable = true;
|
||||
multiPlan->hasReturning = list_length(originalQuery->returningList) > 0;
|
||||
|
||||
return multiPlan;
|
||||
}
|
||||
|
@ -264,6 +267,8 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
|
|||
int shardCount = targetCacheEntry->shardIntervalArrayLength;
|
||||
bool allReferenceTables = restrictionContext->allReferenceTables;
|
||||
|
||||
multiPlan->operation = originalQuery->commandType;
|
||||
|
||||
/*
|
||||
* Error semantics for INSERT ... SELECT queries are different than regular
|
||||
* modify queries. Thus, handle separately.
|
||||
|
@ -319,6 +324,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
|
|||
multiPlan->masterTableName = NULL;
|
||||
multiPlan->masterQuery = NULL;
|
||||
multiPlan->routerExecutable = true;
|
||||
multiPlan->hasReturning = list_length(originalQuery->returningList) > 0;
|
||||
|
||||
return multiPlan;
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@
|
|||
#include "distributed/master_metadata_utility.h"
|
||||
#include "distributed/master_protocol.h"
|
||||
#include "distributed/multi_copy.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_explain.h"
|
||||
#include "distributed/multi_join_order.h"
|
||||
#include "distributed/multi_logical_optimizer.h"
|
||||
|
@ -117,10 +116,6 @@ _PG_init(void)
|
|||
*/
|
||||
if (planner_hook != NULL ||
|
||||
ExplainOneQuery_hook != NULL ||
|
||||
ExecutorStart_hook != NULL ||
|
||||
ExecutorRun_hook != NULL ||
|
||||
ExecutorFinish_hook != NULL ||
|
||||
ExecutorEnd_hook != NULL ||
|
||||
ProcessUtility_hook != NULL)
|
||||
{
|
||||
ereport(ERROR, (errmsg("Citus has to be loaded first"),
|
||||
|
@ -147,15 +142,6 @@ _PG_init(void)
|
|||
/* intercept planner */
|
||||
planner_hook = multi_planner;
|
||||
|
||||
/* intercept explain */
|
||||
ExplainOneQuery_hook = MultiExplainOneQuery;
|
||||
|
||||
/* intercept executor */
|
||||
ExecutorStart_hook = multi_ExecutorStart;
|
||||
ExecutorRun_hook = multi_ExecutorRun;
|
||||
ExecutorFinish_hook = multi_ExecutorFinish;
|
||||
ExecutorEnd_hook = multi_ExecutorEnd;
|
||||
|
||||
/* register utility hook */
|
||||
ProcessUtility_hook = multi_ProcessUtility;
|
||||
|
||||
|
|
|
@ -276,6 +276,9 @@ OutMultiPlan(OUTFUNC_ARGS)
|
|||
|
||||
WRITE_NODE_TYPE("MULTIPLAN");
|
||||
|
||||
WRITE_INT_FIELD(operation);
|
||||
WRITE_BOOL_FIELD(hasReturning);
|
||||
|
||||
WRITE_NODE_FIELD(workerJob);
|
||||
WRITE_NODE_FIELD(masterQuery);
|
||||
WRITE_STRING_FIELD(masterTableName);
|
||||
|
|
|
@ -183,6 +183,9 @@ ReadMultiPlan(READFUNC_ARGS)
|
|||
{
|
||||
READ_LOCALS(MultiPlan);
|
||||
|
||||
READ_INT_FIELD(operation);
|
||||
READ_BOOL_FIELD(hasReturning);
|
||||
|
||||
READ_NODE_FIELD(workerJob);
|
||||
READ_NODE_FIELD(masterQuery);
|
||||
READ_STRING_FIELD(masterTableName);
|
||||
|
|
|
@ -12,6 +12,10 @@
|
|||
|
||||
#include "executor/execdesc.h"
|
||||
#include "nodes/parsenodes.h"
|
||||
#include "nodes/execnodes.h"
|
||||
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/multi_server_executor.h"
|
||||
|
||||
/* signal currently executed statement is a master select statement or router execution */
|
||||
#define EXEC_FLAG_CITUS_MASTER_SELECT 0x100
|
||||
|
@ -23,10 +27,26 @@
|
|||
#define tuplecount_t long
|
||||
#endif
|
||||
|
||||
extern void multi_ExecutorStart(QueryDesc *queryDesc, int eflags);
|
||||
extern void multi_ExecutorRun(QueryDesc *queryDesc,
|
||||
ScanDirection direction, tuplecount_t count);
|
||||
extern void multi_ExecutorFinish(QueryDesc *queryDesc);
|
||||
extern void multi_ExecutorEnd(QueryDesc *queryDesc);
|
||||
|
||||
typedef struct CitusScanState
|
||||
{
|
||||
CustomScanState customScanState;
|
||||
MultiPlan *multiPlan;
|
||||
MultiExecutorType executorType;
|
||||
|
||||
/* state for router */
|
||||
bool finishedUnderlyingScan;
|
||||
Tuplestorestate *tuplestorestate;
|
||||
} CitusScanState;
|
||||
|
||||
Node * CitusCreateScan(CustomScan *scan);
|
||||
extern void CitusBeginScan(CustomScanState *node,
|
||||
EState *estate,
|
||||
int eflags);
|
||||
extern TupleTableSlot * CitusExecScan(CustomScanState *node);
|
||||
extern void CitusEndScan(CustomScanState *node);
|
||||
extern void CitusReScan(CustomScanState *node);
|
||||
extern void CitusExplainScan(CustomScanState *node, List *ancestors,
|
||||
struct ExplainState *es);
|
||||
|
||||
#endif /* MULTI_EXECUTOR_H */
|
||||
|
|
|
@ -16,7 +16,4 @@
|
|||
extern bool ExplainDistributedQueries;
|
||||
extern bool ExplainAllTasks;
|
||||
|
||||
extern void MultiExplainOneQuery(Query *query, IntoClause *into, ExplainState *es,
|
||||
const char *queryString, ParamListInfo params);
|
||||
|
||||
#endif /* MULTI_EXPLAIN_H */
|
||||
|
|
|
@ -21,8 +21,9 @@
|
|||
|
||||
/* Function declarations for building local plans on the master node */
|
||||
struct MultiPlan;
|
||||
extern CreateStmt * MasterNodeCreateStatement(struct MultiPlan *multiPlan);
|
||||
extern List * MasterNodeCopyStatementList(struct MultiPlan *multiPlan);
|
||||
extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan);
|
||||
struct CustomScan;
|
||||
extern PlannedStmt * MasterNodeSelectPlan(struct MultiPlan *multiPlan,
|
||||
struct CustomScan *dataScan);
|
||||
extern List * MasterTargetList(List *workerTargetList);
|
||||
|
||||
#endif /* MULTI_MASTER_PLANNER_H */
|
||||
|
|
|
@ -213,6 +213,10 @@ typedef struct JoinSequenceNode
|
|||
typedef struct MultiPlan
|
||||
{
|
||||
CitusNode type;
|
||||
|
||||
CmdType operation;
|
||||
bool hasReturning;
|
||||
|
||||
Job *workerJob;
|
||||
Query *masterQuery;
|
||||
char *masterTableName;
|
||||
|
|
|
@ -53,7 +53,9 @@ extern PlannedStmt * multi_planner(Query *parse, int cursorOptions,
|
|||
|
||||
extern bool HasCitusToplevelNode(PlannedStmt *planStatement);
|
||||
struct MultiPlan;
|
||||
extern struct MultiPlan * GetMultiPlan(PlannedStmt *planStatement);
|
||||
extern struct MultiPlan * GetMultiPlan(CustomScan *node);
|
||||
extern Node * SerializableMultiPlan(struct MultiPlan *multiPlan);
|
||||
extern struct MultiPlan * DeSerializeMultiPlan(Node *node);
|
||||
extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
|
||||
Index index, RangeTblEntry *rte);
|
||||
extern bool IsModifyCommand(Query *query);
|
||||
|
|
|
@ -12,8 +12,10 @@
|
|||
#include "c.h"
|
||||
|
||||
#include "access/sdir.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "executor/execdesc.h"
|
||||
#include "executor/tuptable.h"
|
||||
#include "nodes/pg_list.h"
|
||||
|
||||
|
||||
|
@ -33,6 +35,9 @@ typedef struct XactShardConnSet
|
|||
extern bool AllModificationsCommutative;
|
||||
extern bool EnableDeadlockPrevention;
|
||||
|
||||
extern void RouterBeginScan(CitusScanState *scanState);
|
||||
|
||||
extern TupleTableSlot * RouterExecScan(CitusScanState *scanState);
|
||||
|
||||
extern void RouterExecutorStart(QueryDesc *queryDesc, int eflags, List *taskList);
|
||||
extern void RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count);
|
||||
|
|
|
@ -39,88 +39,83 @@ $BODY$ LANGUAGE plpgsql;
|
|||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_570000
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570000_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570000_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_570000_0
|
||||
-> Seq Scan on pg_merge_job_570000
|
||||
Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
-- Test JSON format
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
[
|
||||
{
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 8,
|
||||
"Tasks Shown": "One of 8",
|
||||
"Tasks": [
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Parallel Aware": false,
|
||||
"Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Parallel Aware": false,
|
||||
"Distributed Query": {
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 8,
|
||||
"Tasks Shown": "One of 8",
|
||||
"Tasks": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "lineitem_290001",
|
||||
"Alias": "lineitem"
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "lineitem_290001",
|
||||
"Alias": "lineitem"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Parallel Aware": false,
|
||||
"Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570001_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_570001_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["intermediate_column_570001_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "pg_merge_job_570001",
|
||||
"Alias": "pg_merge_job_570001"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
-- Validate JSON format
|
||||
|
@ -133,75 +128,72 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>8</Task-Count>
|
||||
<Tasks-Shown>One of 8</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Relation-Name>lineitem_290001</Relation-Name>
|
||||
<Alias>lineitem</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<Query>
|
||||
<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>Sort</Node-Type>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<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>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<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>
|
||||
</Plans>
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>8</Task-Count>
|
||||
<Tasks-Shown>One of 8</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Relation-Name>lineitem_290001</Relation-Name>
|
||||
<Alias>lineitem</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
-- Validate XML format
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
|
@ -212,114 +204,107 @@ t
|
|||
EXPLAIN (COSTS FALSE, FORMAT YAML)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
- Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 8
|
||||
Tasks Shown: "One of 8"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
Parallel Aware: false
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "lineitem_290001"
|
||||
Alias: "lineitem"
|
||||
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
Parallel Aware: false
|
||||
Sort Key:
|
||||
- "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Sort Key:
|
||||
- "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570005_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "intermediate_column_570005_0"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Parallel Aware: false
|
||||
Group Key:
|
||||
- "intermediate_column_570005_0"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "pg_merge_job_570005"
|
||||
Alias: "pg_merge_job_570005"
|
||||
Distributed Query:
|
||||
Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 8
|
||||
Tasks Shown: "One of 8"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
Parallel Aware: false
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "lineitem_290001"
|
||||
Alias: "lineitem"
|
||||
|
||||
-- Test Text format
|
||||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_570006
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_570006_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570006_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_570006_0
|
||||
-> Seq Scan on pg_merge_job_570006
|
||||
Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
-- Test verbose
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem;
|
||||
Distributed Query into pg_merge_job_570007
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> Aggregate
|
||||
Output: (sum(intermediate_column_570007_0) / (sum(intermediate_column_570007_1) / pg_catalog.sum(intermediate_column_570007_2)))
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_570007
|
||||
Output: intermediate_column_570007_0, intermediate_column_570007_1, intermediate_column_570007_2
|
||||
Aggregate
|
||||
Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2")))
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: "?column?", "?column?_1", "?column?_2"
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test join
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT * FROM lineitem
|
||||
JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0
|
||||
ORDER BY l_quantity LIMIT 10;
|
||||
Distributed Query into pg_merge_job_570008
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem.l_orderkey = orders.o_orderkey)
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_290008 orders
|
||||
Master Query
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: intermediate_column_570008_4
|
||||
-> Seq Scan on pg_merge_job_570008
|
||||
Limit
|
||||
-> Sort
|
||||
Sort Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem.l_orderkey = orders.o_orderkey)
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_290008 orders
|
||||
-- Test insert
|
||||
EXPLAIN (COSTS FALSE)
|
||||
INSERT INTO lineitem VALUES(1,0);
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -332,7 +317,7 @@ EXPLAIN (COSTS FALSE)
|
|||
UPDATE lineitem
|
||||
SET l_suppkey = 12
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -348,7 +333,7 @@ Distributed Query
|
|||
EXPLAIN (COSTS FALSE)
|
||||
DELETE FROM lineitem
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -363,7 +348,7 @@ Distributed Query
|
|||
-- Test single-shard SELECT
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
|
||||
Distributed Query into pg_merge_job_570009
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -383,89 +368,81 @@ t
|
|||
EXPLAIN (COSTS FALSE)
|
||||
CREATE TABLE explain_result AS
|
||||
SELECT * FROM lineitem;
|
||||
Distributed Query into pg_merge_job_570012
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Seq Scan on pg_merge_job_570012
|
||||
-- Test having
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem
|
||||
HAVING sum(l_quantity) > 100;
|
||||
Distributed Query into pg_merge_job_570013
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> Aggregate
|
||||
Output: (sum(intermediate_column_570013_0) / (sum(intermediate_column_570013_1) / pg_catalog.sum(intermediate_column_570013_2)))
|
||||
Filter: (sum(pg_merge_job_570013.intermediate_column_570013_3) > '100'::numeric)
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_570013
|
||||
Output: intermediate_column_570013_0, intermediate_column_570013_1, intermediate_column_570013_2, intermediate_column_570013_3
|
||||
Aggregate
|
||||
Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2")))
|
||||
Filter: (sum("remote scan".worker_column_4) > '100'::numeric)
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: "?column?", "?column?_1", "?column?_2", worker_column_4
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test having without aggregate
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT l_quantity FROM lineitem
|
||||
GROUP BY l_quantity
|
||||
HAVING l_quantity > (100 * random());
|
||||
Distributed Query into pg_merge_job_570014
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Output: l_quantity, l_quantity
|
||||
Group Key: lineitem.l_quantity
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> HashAggregate
|
||||
Output: intermediate_column_570014_0
|
||||
Group Key: pg_merge_job_570014.intermediate_column_570014_0
|
||||
Filter: ((pg_merge_job_570014.intermediate_column_570014_1)::double precision > ('100'::double precision * random()))
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_570014
|
||||
Output: intermediate_column_570014_0, intermediate_column_570014_1
|
||||
HashAggregate
|
||||
Output: l_quantity
|
||||
Group Key: "remote scan".l_quantity
|
||||
Filter: (("remote scan".worker_column_2)::double precision > ('100'::double precision * random()))
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: l_quantity, worker_column_2
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Output: l_quantity, l_quantity
|
||||
Group Key: lineitem.l_quantity
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test all tasks output
|
||||
SET citus.explain_all_tasks TO on;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_570015
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290004 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290007 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290006 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570015
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290004 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290007 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290006 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$);
|
||||
t
|
||||
|
@ -477,18 +454,16 @@ SET citus.task_executor_type TO 'task-tracker';
|
|||
SET citus.explain_all_tasks TO off;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_570018
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570018
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-- Test re-partition join
|
||||
SET citus.large_table_shard_count TO 1;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
|
@ -497,19 +472,17 @@ EXPLAIN (COSTS FALSE)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
Distributed Query into pg_merge_job_570021
|
||||
Executor: Task-Tracker
|
||||
Task Count: 1
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 1
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 8
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570021
|
||||
-> MapMergeJob
|
||||
Map Task Count: 8
|
||||
Merge Task Count: 1
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT count(*)
|
||||
FROM lineitem, orders, customer, supplier_single_shard
|
||||
|
@ -518,42 +491,39 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
|
|||
AND l_suppkey = s_suppkey;
|
||||
[
|
||||
{
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Plans": [
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 1,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 8,
|
||||
"Merge Task Count": 1
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Parallel Aware": false,
|
||||
"Distributed Query": {
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 1,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 8,
|
||||
"Merge Task Count": 1
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "pg_merge_job_570024",
|
||||
"Alias": "pg_merge_job_570024"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
SELECT true AS valid FROM explain_json($$
|
||||
|
@ -570,44 +540,41 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>1</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>8</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<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>Aggregate</Node-Type>
|
||||
<Strategy>Plain</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<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>
|
||||
</Plans>
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>1</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>8</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT count(*)
|
||||
|
@ -635,28 +602,27 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
- Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 1
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Depended Jobs:
|
||||
- Map Task Count: 8
|
||||
Merge Task Count: 1
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Partial Mode: "Simple"
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Partial Mode: "Simple"
|
||||
Parallel Aware: false
|
||||
Plans:
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Parallel Aware: false
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "pg_merge_job_570035"
|
||||
Alias: "pg_merge_job_570035"
|
||||
Distributed Query:
|
||||
Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 1
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Depended Jobs:
|
||||
- Map Task Count: 8
|
||||
Merge Task Count: 1
|
||||
-- test parallel aggregates
|
||||
SET parallel_setup_cost=0;
|
||||
SET parallel_tuple_cost=0;
|
||||
|
@ -672,37 +638,33 @@ Finalize Aggregate
|
|||
-> Parallel Seq Scan on lineitem_clone
|
||||
-- ensure distributed plans don't break
|
||||
EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem;
|
||||
Distributed Query into pg_merge_job_570036
|
||||
Executor: Task-Tracker
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570036
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
-- ensure EXPLAIN EXECUTE doesn't crash
|
||||
PREPARE task_tracker_query AS
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query;
|
||||
Distributed Query into pg_merge_job_570037
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570037
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
SET citus.task_executor_type TO 'real-time';
|
||||
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
|
||||
EXPLAIN EXECUTE router_executor_query;
|
||||
Distributed Query into pg_merge_job_570038
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -715,22 +677,27 @@ Distributed Query into pg_merge_job_570038
|
|||
PREPARE real_time_executor_query AS
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query;
|
||||
Distributed Query into pg_merge_job_570039
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570039
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-- EXPLAIN EXECUTE of parametrized prepared statements is broken, but
|
||||
-- at least make sure to fail without crashing
|
||||
PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1;
|
||||
EXPLAIN EXECUTE router_executor_query_param(5);
|
||||
ERROR: could not create distributed plan
|
||||
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus.
|
||||
HINT: Consider using PLPGSQL functions instead.
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Bitmap Heap Scan on lineitem_290000 lineitem (cost=4.30..13.44 rows=3 width=18)
|
||||
Recheck Cond: (l_orderkey = 5)
|
||||
-> Bitmap Index Scan on lineitem_pkey_290000 (cost=0.00..4.30 rows=3 width=0)
|
||||
Index Cond: (l_orderkey = 5)
|
||||
|
|
|
@ -39,81 +39,76 @@ $BODY$ LANGUAGE plpgsql;
|
|||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_570000
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_570000_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570000_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_570000_0
|
||||
-> Seq Scan on pg_merge_job_570000
|
||||
Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
-- Test JSON format
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
[
|
||||
{
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 8,
|
||||
"Tasks Shown": "One of 8",
|
||||
"Tasks": [
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Sort Key": ["COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Parent Relationship": "Outer",
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Distributed Query": {
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 8,
|
||||
"Tasks Shown": "One of 8",
|
||||
"Tasks": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "lineitem_290001",
|
||||
"Alias": "lineitem"
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "lineitem_290001",
|
||||
"Alias": "lineitem"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Sort Key": ["COALESCE((sum((COALESCE((sum(intermediate_column_570001_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_570001_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Parent Relationship": "Outer",
|
||||
"Group Key": ["intermediate_column_570001_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "pg_merge_job_570001",
|
||||
"Alias": "pg_merge_job_570001"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
-- Validate JSON format
|
||||
|
@ -126,68 +121,65 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>8</Task-Count>
|
||||
<Tasks-Shown>One of 8</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Relation-Name>lineitem_290001</Relation-Name>
|
||||
<Alias>lineitem</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<Query>
|
||||
<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>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>
|
||||
<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>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<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>
|
||||
</Plans>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>8</Task-Count>
|
||||
<Tasks-Shown>One of 8</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Relation-Name>lineitem_290001</Relation-Name>
|
||||
<Alias>lineitem</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
-- Validate XML format
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
|
@ -198,107 +190,100 @@ t
|
|||
EXPLAIN (COSTS FALSE, FORMAT YAML)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
- Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 8
|
||||
Tasks Shown: "One of 8"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "lineitem_290001"
|
||||
Alias: "lineitem"
|
||||
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
Sort Key:
|
||||
- "COALESCE((sum((COALESCE((sum(intermediate_column_570005_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "intermediate_column_570005_0"
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
Sort Key:
|
||||
- "COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Parent Relationship: "Outer"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Group Key:
|
||||
- "intermediate_column_570005_0"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "pg_merge_job_570005"
|
||||
Alias: "pg_merge_job_570005"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Distributed Query:
|
||||
Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 8
|
||||
Tasks Shown: "One of 8"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "lineitem_290001"
|
||||
Alias: "lineitem"
|
||||
|
||||
-- Test Text format
|
||||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_570006
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_570006_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_570006_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_570006_0
|
||||
-> Seq Scan on pg_merge_job_570006
|
||||
Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
-- Test verbose
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem;
|
||||
Distributed Query into pg_merge_job_570007
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> Aggregate
|
||||
Output: (sum(intermediate_column_570007_0) / (sum(intermediate_column_570007_1) / sum(intermediate_column_570007_2)))
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_570007
|
||||
Output: intermediate_column_570007_0, intermediate_column_570007_1, intermediate_column_570007_2
|
||||
Aggregate
|
||||
Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2")))
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: "?column?", "?column?_1", "?column?_2"
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test join
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT * FROM lineitem
|
||||
JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5.0
|
||||
ORDER BY l_quantity LIMIT 10;
|
||||
Distributed Query into pg_merge_job_570008
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem.l_orderkey = orders.o_orderkey)
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_290008 orders
|
||||
Master Query
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: intermediate_column_570008_4
|
||||
-> Seq Scan on pg_merge_job_570008
|
||||
Limit
|
||||
-> Sort
|
||||
Sort Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem.l_orderkey = orders.o_orderkey)
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_290008 orders
|
||||
-- Test insert
|
||||
EXPLAIN (COSTS FALSE)
|
||||
INSERT INTO lineitem VALUES(1,0);
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -311,7 +296,7 @@ EXPLAIN (COSTS FALSE)
|
|||
UPDATE lineitem
|
||||
SET l_suppkey = 12
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -327,7 +312,7 @@ Distributed Query
|
|||
EXPLAIN (COSTS FALSE)
|
||||
DELETE FROM lineitem
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -342,7 +327,7 @@ Distributed Query
|
|||
-- Test single-shard SELECT
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
|
||||
Distributed Query into pg_merge_job_570009
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -362,89 +347,81 @@ t
|
|||
EXPLAIN (COSTS FALSE)
|
||||
CREATE TABLE explain_result AS
|
||||
SELECT * FROM lineitem;
|
||||
Distributed Query into pg_merge_job_570012
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Seq Scan on pg_merge_job_570012
|
||||
-- Test having
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem
|
||||
HAVING sum(l_quantity) > 100;
|
||||
Distributed Query into pg_merge_job_570013
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> Aggregate
|
||||
Output: (sum(intermediate_column_570013_0) / (sum(intermediate_column_570013_1) / sum(intermediate_column_570013_2)))
|
||||
Filter: (sum(pg_merge_job_570013.intermediate_column_570013_3) > '100'::numeric)
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_570013
|
||||
Output: intermediate_column_570013_0, intermediate_column_570013_1, intermediate_column_570013_2, intermediate_column_570013_3
|
||||
Aggregate
|
||||
Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2")))
|
||||
Filter: (sum("remote scan".worker_column_4) > '100'::numeric)
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: "?column?", "?column?_1", "?column?_2", worker_column_4
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity), sum(l_quantity)
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test having without aggregate
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT l_quantity FROM lineitem
|
||||
GROUP BY l_quantity
|
||||
HAVING l_quantity > (100 * random());
|
||||
Distributed Query into pg_merge_job_570014
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Output: l_quantity, l_quantity
|
||||
Group Key: lineitem.l_quantity
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> HashAggregate
|
||||
Output: intermediate_column_570014_0
|
||||
Group Key: pg_merge_job_570014.intermediate_column_570014_0
|
||||
Filter: ((pg_merge_job_570014.intermediate_column_570014_1)::double precision > ('100'::double precision * random()))
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_570014
|
||||
Output: intermediate_column_570014_0, intermediate_column_570014_1
|
||||
HashAggregate
|
||||
Output: l_quantity
|
||||
Group Key: "remote scan".l_quantity
|
||||
Filter: (("remote scan".worker_column_2)::double precision > ('100'::double precision * random()))
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: l_quantity, worker_column_2
|
||||
Executor: Real-Time
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Output: l_quantity, l_quantity
|
||||
Group Key: lineitem.l_quantity
|
||||
-> Seq Scan on public.lineitem_290001 lineitem
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test all tasks output
|
||||
SET citus.explain_all_tasks TO on;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_570015
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290004 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290007 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290006 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570015
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290004 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290007 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290006 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030$$);
|
||||
t
|
||||
|
@ -456,18 +433,16 @@ SET citus.task_executor_type TO 'task-tracker';
|
|||
SET citus.explain_all_tasks TO off;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_570018
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570018
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-- Test re-partition join
|
||||
SET citus.large_table_shard_count TO 1;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
|
@ -476,19 +451,17 @@ EXPLAIN (COSTS FALSE)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
Distributed Query into pg_merge_job_570021
|
||||
Executor: Task-Tracker
|
||||
Task Count: 1
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 1
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 8
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570021
|
||||
-> MapMergeJob
|
||||
Map Task Count: 8
|
||||
Merge Task Count: 1
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT count(*)
|
||||
FROM lineitem, orders, customer, supplier_single_shard
|
||||
|
@ -497,39 +470,36 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
|
|||
AND l_suppkey = s_suppkey;
|
||||
[
|
||||
{
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Plans": [
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 1,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 8,
|
||||
"Merge Task Count": 1
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Distributed Query": {
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 1,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 8,
|
||||
"Merge Task Count": 1
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "pg_merge_job_570024",
|
||||
"Alias": "pg_merge_job_570024"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
SELECT true AS valid FROM explain_json($$
|
||||
|
@ -546,41 +516,38 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>1</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>8</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<Query>
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Plain</Strategy>
|
||||
<Plans>
|
||||
<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>
|
||||
</Plans>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>1</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>8</Map-Task-Count>
|
||||
<Merge-Task-Count>1</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT count(*)
|
||||
|
@ -608,25 +575,24 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
- Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 1
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Depended Jobs:
|
||||
- Map Task Count: 8
|
||||
Merge Task Count: 1
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "pg_merge_job_570035"
|
||||
Alias: "pg_merge_job_570035"
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Plans:
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Distributed Query:
|
||||
Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 1
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 1
|
||||
Depended Jobs:
|
||||
- Map Task Count: 8
|
||||
Merge Task Count: 1
|
||||
-- test parallel aggregates
|
||||
SET parallel_setup_cost=0;
|
||||
ERROR: unrecognized configuration parameter "parallel_setup_cost"
|
||||
|
@ -643,37 +609,33 @@ Aggregate
|
|||
-> Seq Scan on lineitem_clone
|
||||
-- ensure distributed plans don't break
|
||||
EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem;
|
||||
Distributed Query into pg_merge_job_570036
|
||||
Executor: Task-Tracker
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570036
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 8
|
||||
Tasks Shown: One of 8
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290001 lineitem
|
||||
-- ensure EXPLAIN EXECUTE doesn't crash
|
||||
PREPARE task_tracker_query AS
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query;
|
||||
Distributed Query into pg_merge_job_570037
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570037
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
SET citus.task_executor_type TO 'real-time';
|
||||
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
|
||||
EXPLAIN EXECUTE router_executor_query;
|
||||
Distributed Query into pg_merge_job_570038
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -686,22 +648,27 @@ Distributed Query into pg_merge_job_570038
|
|||
PREPARE real_time_executor_query AS
|
||||
SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||
EXPLAIN (COSTS FALSE) EXECUTE real_time_executor_query;
|
||||
Distributed Query into pg_merge_job_570039
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_570039
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_290005 lineitem
|
||||
Filter: (l_orderkey > 9030)
|
||||
-- EXPLAIN EXECUTE of parametrized prepared statements is broken, but
|
||||
-- at least make sure to fail without crashing
|
||||
PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1;
|
||||
EXPLAIN EXECUTE router_executor_query_param(5);
|
||||
ERROR: could not create distributed plan
|
||||
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus.
|
||||
HINT: Consider using PLPGSQL functions instead.
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Bitmap Heap Scan on lineitem_290000 lineitem (cost=4.30..13.44 rows=3 width=18)
|
||||
Recheck Cond: (l_orderkey = 5)
|
||||
-> Bitmap Index Scan on lineitem_pkey_290000 (cost=0.00..4.30 rows=3 width=0)
|
||||
Index Cond: (l_orderkey = 5)
|
||||
|
|
|
@ -6,6 +6,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 650000;
|
|||
-- Set configuration to print table join order and pruned shards
|
||||
SET citus.explain_distributed_queries TO off;
|
||||
SET citus.log_multi_join_order TO TRUE;
|
||||
SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwhise
|
||||
SET client_min_messages TO DEBUG2;
|
||||
-- Create new table definitions for use in testing in distributed planning and
|
||||
-- execution functionality. Also create indexes to boost performance.
|
||||
|
@ -139,10 +140,11 @@ DEBUG: join prunable for intervals [13473,14947] and [2951,4455]
|
|||
DEBUG: join prunable for intervals [13473,14947] and [4480,5986]
|
||||
DEBUG: join prunable for intervals [13473,14947] and [8997,10560]
|
||||
DEBUG: join prunable for intervals [13473,14947] and [10560,12036]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(2 rows)
|
||||
|
||||
-- Update configuration to treat lineitem and orders tables as large
|
||||
SET citus.large_table_shard_count TO 2;
|
||||
|
@ -154,10 +156,12 @@ EXPLAIN SELECT count(*) FROM lineitem, orders
|
|||
WHERE (l_orderkey = o_orderkey AND l_quantity > 5)
|
||||
OR (l_orderkey = o_orderkey AND l_quantity < 10);
|
||||
LOG: join order: [ "lineitem" ][ local partition join "orders" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT l_quantity FROM lineitem, orders
|
||||
WHERE (l_orderkey = o_orderkey OR l_quantity > 5);
|
||||
|
@ -173,28 +177,34 @@ BEGIN;
|
|||
EXPLAIN SELECT count(*) FROM orders, lineitem_hash
|
||||
WHERE o_orderkey = l_orderkey;
|
||||
LOG: join order: [ "orders" ][ single partition join "lineitem_hash" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Verify we handle local joins between two hash-partitioned tables.
|
||||
EXPLAIN SELECT count(*) FROM orders_hash, lineitem_hash
|
||||
WHERE o_orderkey = l_orderkey;
|
||||
LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Validate that we can handle broadcast joins with hash-partitioned tables.
|
||||
EXPLAIN SELECT count(*) FROM customer_hash, nation
|
||||
WHERE c_nationkey = n_nationkey;
|
||||
LOG: join order: [ "customer_hash" ][ broadcast join "nation" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Update the large table shard count for all the following tests.
|
||||
SET citus.large_table_shard_count TO 1;
|
||||
|
@ -203,30 +213,36 @@ SET citus.large_table_shard_count TO 1;
|
|||
EXPLAIN SELECT count(*) FROM orders, lineitem, customer
|
||||
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
|
||||
LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Validate that we don't chose a single-partition join method with a
|
||||
-- hash-partitioned base table
|
||||
EXPLAIN SELECT count(*) FROM orders, customer_hash
|
||||
WHERE c_custkey = o_custkey;
|
||||
LOG: join order: [ "orders" ][ dual partition join "customer_hash" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Validate that we can re-partition a hash partitioned table to join with a
|
||||
-- range partitioned one.
|
||||
EXPLAIN SELECT count(*) FROM orders_hash, customer
|
||||
WHERE c_custkey = o_custkey;
|
||||
LOG: join order: [ "orders_hash" ][ single partition join "customer" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
COMMIT;
|
||||
-- Reset client logging level to its previous value
|
||||
|
|
|
@ -6,6 +6,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 660000;
|
|||
-- Enable configuration to print table join order
|
||||
SET citus.explain_distributed_queries TO off;
|
||||
SET citus.log_multi_join_order TO TRUE;
|
||||
SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwhise
|
||||
SET client_min_messages TO LOG;
|
||||
-- Change configuration to treat lineitem, orders, customer, and part tables as
|
||||
-- large. The following queries are basically the same as the ones in tpch_small
|
||||
|
@ -23,10 +24,12 @@ WHERE
|
|||
and l_discount between 0.06 - 0.01 and 0.06 + 0.01
|
||||
and l_quantity < 24;
|
||||
LOG: join order: [ "lineitem" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Query #3 from the TPC-H decision support benchmark
|
||||
EXPLAIN SELECT
|
||||
|
@ -52,10 +55,15 @@ ORDER BY
|
|||
revenue DESC,
|
||||
o_orderdate;
|
||||
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------------
|
||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
||||
Sort Key: sum((sum(revenue))) DESC, o_orderdate
|
||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: l_orderkey, o_orderdate, o_shippriority
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(6 rows)
|
||||
|
||||
-- Query #10 from the TPC-H decision support benchmark
|
||||
EXPLAIN SELECT
|
||||
|
@ -90,10 +98,15 @@ GROUP BY
|
|||
ORDER BY
|
||||
revenue DESC;
|
||||
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ][ broadcast join "nation" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------
|
||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
||||
Sort Key: sum((sum(revenue))) DESC
|
||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(6 rows)
|
||||
|
||||
-- Query #19 from the TPC-H decision support benchmark (modified)
|
||||
EXPLAIN SELECT
|
||||
|
@ -126,10 +139,12 @@ WHERE
|
|||
AND l_shipinstruct = 'DELIVER IN PERSON'
|
||||
);
|
||||
LOG: join order: [ "lineitem" ][ single partition join "part" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Query to test multiple re-partition jobs in a single query
|
||||
EXPLAIN SELECT
|
||||
|
@ -143,10 +158,13 @@ WHERE
|
|||
GROUP BY
|
||||
l_partkey;
|
||||
LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single partition join "part" ][ single partition join "customer" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: l_partkey
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(4 rows)
|
||||
|
||||
-- Reset client logging level to its previous value
|
||||
SET client_min_messages TO NOTICE;
|
||||
|
|
|
@ -18,10 +18,12 @@ WHERE
|
|||
and l_discount between 0.06 - 0.01 and 0.06 + 0.01
|
||||
and l_quantity < 24;
|
||||
LOG: join order: [ "lineitem" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Query #3 from the TPC-H decision support benchmark
|
||||
EXPLAIN SELECT
|
||||
|
@ -47,10 +49,15 @@ ORDER BY
|
|||
revenue DESC,
|
||||
o_orderdate;
|
||||
LOG: join order: [ "orders" ][ broadcast join "customer" ][ local partition join "lineitem" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------------
|
||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
||||
Sort Key: sum((sum(revenue))) DESC, o_orderdate
|
||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: l_orderkey, o_orderdate, o_shippriority
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(6 rows)
|
||||
|
||||
-- Query #10 from the TPC-H decision support benchmark
|
||||
EXPLAIN SELECT
|
||||
|
@ -85,10 +92,15 @@ GROUP BY
|
|||
ORDER BY
|
||||
revenue DESC;
|
||||
LOG: join order: [ "orders" ][ broadcast join "customer" ][ broadcast join "nation" ][ local partition join "lineitem" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------
|
||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
||||
Sort Key: sum((sum(revenue))) DESC
|
||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(6 rows)
|
||||
|
||||
-- Query #19 from the TPC-H decision support benchmark (modified)
|
||||
EXPLAIN SELECT
|
||||
|
@ -121,10 +133,12 @@ WHERE
|
|||
AND l_shipinstruct = 'DELIVER IN PERSON'
|
||||
);
|
||||
LOG: join order: [ "lineitem" ][ broadcast join "part" ]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Reset client logging level to its previous value
|
||||
SET client_min_messages TO NOTICE;
|
||||
|
|
|
@ -100,20 +100,24 @@ EXPLAIN SELECT count(*)
|
|||
WHERE table1.array_column = table2.array_column;
|
||||
DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}]
|
||||
DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*)
|
||||
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
||||
WHERE table1.composite_column = table2.composite_column;
|
||||
DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)]
|
||||
DEBUG: join prunable for intervals [(c,5,d),(d,6,e)] and [(a,3,b),(b,4,c)]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Test that large table joins on partition varchar columns work
|
||||
EXPLAIN SELECT count(*)
|
||||
|
@ -121,8 +125,10 @@ EXPLAIN SELECT count(*)
|
|||
WHERE table1.varchar_column = table2.varchar_column;
|
||||
DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U2AMO4ZGX,BZZXSP27F21T6]
|
||||
DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6]
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
|
|
|
@ -61,88 +61,83 @@ $BODY$ LANGUAGE plpgsql;
|
|||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_68720796736
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_68720796736_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_68720796736_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_68720796736_0
|
||||
-> Seq Scan on pg_merge_job_68720796736
|
||||
Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
-- Test JSON format
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
[
|
||||
{
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 16,
|
||||
"Tasks Shown": "One of 16",
|
||||
"Tasks": [
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Parallel Aware": false,
|
||||
"Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Parallel Aware": false,
|
||||
"Distributed Query": {
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 16,
|
||||
"Tasks Shown": "One of 16",
|
||||
"Tasks": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "lineitem_mx_1220052",
|
||||
"Alias": "lineitem_mx"
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "lineitem_mx_1220052",
|
||||
"Alias": "lineitem_mx"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Parallel Aware": false,
|
||||
"Sort Key": ["COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_68720796737_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_68720796737_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Partial Mode": "Simple",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Group Key": ["intermediate_column_68720796737_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "pg_merge_job_68720796737",
|
||||
"Alias": "pg_merge_job_68720796737"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
-- Validate JSON format
|
||||
|
@ -156,75 +151,72 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>16</Task-Count>
|
||||
<Tasks-Shown>One of 16</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Relation-Name>lineitem_mx_1220052</Relation-Name>
|
||||
<Alias>lineitem_mx</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<Query>
|
||||
<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>Sort</Node-Type>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<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>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<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>
|
||||
</Plans>
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>16</Task-Count>
|
||||
<Tasks-Shown>One of 16</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Relation-Name>lineitem_mx_1220052</Relation-Name>
|
||||
<Alias>lineitem_mx</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
-- Validate XML format
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
|
@ -235,115 +227,108 @@ t
|
|||
EXPLAIN (COSTS FALSE, FORMAT YAML)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
- Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 16
|
||||
Tasks Shown: "One of 16"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
Parallel Aware: false
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "lineitem_mx_1220052"
|
||||
Alias: "lineitem_mx"
|
||||
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
Parallel Aware: false
|
||||
Sort Key:
|
||||
- "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Sort Key:
|
||||
- "COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_60130862146_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "intermediate_column_60130862146_0"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Parallel Aware: false
|
||||
Group Key:
|
||||
- "intermediate_column_60130862146_0"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "pg_merge_job_60130862146"
|
||||
Alias: "pg_merge_job_60130862146"
|
||||
Distributed Query:
|
||||
Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 16
|
||||
Tasks Shown: "One of 16"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Partial Mode: "Simple"
|
||||
Parallel Aware: false
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "lineitem_mx_1220052"
|
||||
Alias: "lineitem_mx"
|
||||
|
||||
-- Test Text format
|
||||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_60130862147
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(intermediate_column_60130862147_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_60130862147_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_60130862147_0
|
||||
-> Seq Scan on pg_merge_job_60130862147
|
||||
Sort
|
||||
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
\c - - - :worker_2_port
|
||||
-- Test verbose
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx;
|
||||
Distributed Query into pg_merge_job_68720796739
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_mx_1220052 lineitem_mx
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> Aggregate
|
||||
Output: (sum(intermediate_column_68720796739_0) / (sum(intermediate_column_68720796739_1) / pg_catalog.sum(intermediate_column_68720796739_2)))
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_68720796739
|
||||
Output: intermediate_column_68720796739_0, intermediate_column_68720796739_1, intermediate_column_68720796739_2
|
||||
Aggregate
|
||||
Output: (sum("?column?") / (sum("?column?_1") / pg_catalog.sum("?column?_2")))
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: "?column?", "?column?_1", "?column?_2"
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_mx_1220052 lineitem_mx
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test join
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT * FROM lineitem_mx
|
||||
JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0
|
||||
ORDER BY l_quantity LIMIT 10;
|
||||
Distributed Query into pg_merge_job_68720796740
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem_mx.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey)
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_mx_1220068 orders_mx
|
||||
Master Query
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: intermediate_column_68720796740_4
|
||||
-> Seq Scan on pg_merge_job_68720796740
|
||||
Limit
|
||||
-> Sort
|
||||
Sort Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem_mx.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey)
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_mx_1220068 orders_mx
|
||||
-- Test insert
|
||||
EXPLAIN (COSTS FALSE)
|
||||
INSERT INTO lineitem_mx VALUES(1,0);
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -356,7 +341,7 @@ EXPLAIN (COSTS FALSE)
|
|||
UPDATE lineitem_mx
|
||||
SET l_suppkey = 12
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -370,7 +355,7 @@ Distributed Query
|
|||
EXPLAIN (COSTS FALSE)
|
||||
DELETE FROM lineitem_mx
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -383,7 +368,7 @@ Distributed Query
|
|||
-- Test single-shard SELECT
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -403,106 +388,102 @@ t
|
|||
EXPLAIN (COSTS FALSE)
|
||||
CREATE TABLE explain_result AS
|
||||
SELECT * FROM lineitem_mx;
|
||||
Distributed Query into pg_merge_job_68720796741
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Master Query
|
||||
-> Seq Scan on pg_merge_job_68720796741
|
||||
-- Test all tasks output
|
||||
SET citus.explain_all_tasks TO on;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_68720796742
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220053 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220054 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220055 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220056 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220057 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220058 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220059 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220060 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220061 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220062 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220063 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220064 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220065 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220066 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220067 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_68720796742
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220053 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220054 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220055 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220056 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220057 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220058 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220059 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220060 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220061 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220062 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220063 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220064 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220065 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220066 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220067 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$);
|
||||
t
|
||||
|
@ -514,18 +495,16 @@ SET citus.task_executor_type TO 'task-tracker';
|
|||
SET citus.explain_all_tasks TO off;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_68720796745
|
||||
Executor: Task-Tracker
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_68720796745
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-- Test re-partition join
|
||||
SET citus.large_table_shard_count TO 1;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
|
@ -534,25 +513,23 @@ EXPLAIN (COSTS FALSE)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
Distributed Query into pg_merge_job_68720796750
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 16
|
||||
Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 16
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_68720796750
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT count(*)
|
||||
FROM lineitem_mx, orders_mx, customer_mx, supplier_mx
|
||||
|
@ -561,50 +538,47 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
|
|||
AND l_suppkey = s_suppkey;
|
||||
[
|
||||
{
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 4,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Plans": [
|
||||
{
|
||||
"Map Task Count": 4,
|
||||
"Merge Task Count": 4,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 16,
|
||||
"Merge Task Count": 4
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Parallel Aware": false,
|
||||
"Distributed Query": {
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 4,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 4,
|
||||
"Merge Task Count": 4,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 16,
|
||||
"Merge Task Count": 4
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Partial Mode": "Simple",
|
||||
"Parallel Aware": false,
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Relation Name": "pg_merge_job_68720796755",
|
||||
"Alias": "pg_merge_job_68720796755"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
SELECT true AS valid FROM explain_json($$
|
||||
|
@ -621,52 +595,49 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>4</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>4</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>16</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<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>Aggregate</Node-Type>
|
||||
<Strategy>Plain</Strategy>
|
||||
<Partial-Mode>Simple</Partial-Mode>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<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>
|
||||
</Plans>
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>4</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>4</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>16</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT count(*)
|
||||
|
@ -681,29 +652,28 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
- Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 4
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
Depended Jobs:
|
||||
- Map Task Count: 16
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Partial Mode: "Simple"
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Partial Mode: "Simple"
|
||||
Parallel Aware: false
|
||||
Plans:
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Parallel Aware: false
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Parallel Aware: false
|
||||
Relation Name: "pg_merge_job_68720796775"
|
||||
Alias: "pg_merge_job_68720796775"
|
||||
Distributed Query:
|
||||
Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 4
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
Depended Jobs:
|
||||
- Map Task Count: 16
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
|
|
|
@ -61,81 +61,76 @@ $BODY$ LANGUAGE plpgsql;
|
|||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_68720796736
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_68720796736_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_68720796736_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_68720796736_0
|
||||
-> Seq Scan on pg_merge_job_68720796736
|
||||
Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
-- Test JSON format
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
[
|
||||
{
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 16,
|
||||
"Tasks Shown": "One of 16",
|
||||
"Tasks": [
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Sort Key": ["COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Parent Relationship": "Outer",
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Distributed Query": {
|
||||
"Executor": "Real-Time",
|
||||
"Job": {
|
||||
"Task Count": 16,
|
||||
"Tasks Shown": "One of 16",
|
||||
"Tasks": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "lineitem_mx_1220052",
|
||||
"Alias": "lineitem_mx"
|
||||
"Node": "host=localhost port=57637 dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Group Key": ["l_quantity"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "lineitem_mx_1220052",
|
||||
"Alias": "lineitem_mx"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Sort",
|
||||
"Sort Key": ["COALESCE((sum((COALESCE((sum(intermediate_column_68720796737_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)", "intermediate_column_68720796737_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Hashed",
|
||||
"Parent Relationship": "Outer",
|
||||
"Group Key": ["intermediate_column_68720796737_0"],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "pg_merge_job_68720796737",
|
||||
"Alias": "pg_merge_job_68720796737"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
-- Validate JSON format
|
||||
|
@ -149,68 +144,65 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>16</Task-Count>
|
||||
<Tasks-Shown>One of 16</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Relation-Name>lineitem_mx_1220052</Relation-Name>
|
||||
<Alias>lineitem_mx</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<Query>
|
||||
<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>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>
|
||||
<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>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<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>
|
||||
</Plans>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<Distributed-Query>
|
||||
<Executor>Real-Time</Executor>
|
||||
<Job>
|
||||
<Task-Count>16</Task-Count>
|
||||
<Tasks-Shown>One of 16</Tasks-Shown>
|
||||
<Tasks>
|
||||
<Task>
|
||||
<Node>host=localhost port=57637 dbname=regression</Node>
|
||||
<Remote-Plan>
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Hashed</Strategy>
|
||||
<Group-Key>
|
||||
<Item>l_quantity</Item>
|
||||
</Group-Key>
|
||||
<Plans>
|
||||
<Plan>
|
||||
<Node-Type>Seq Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Relation-Name>lineitem_mx_1220052</Relation-Name>
|
||||
<Alias>lineitem_mx</Alias>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
</Remote-Plan>
|
||||
</Task>
|
||||
</Tasks>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
-- Validate XML format
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
|
@ -221,108 +213,101 @@ t
|
|||
EXPLAIN (COSTS FALSE, FORMAT YAML)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
- Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 16
|
||||
Tasks Shown: "One of 16"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "lineitem_mx_1220052"
|
||||
Alias: "lineitem_mx"
|
||||
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
Sort Key:
|
||||
- "COALESCE((sum((COALESCE((sum(intermediate_column_60130862146_1))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "intermediate_column_60130862146_0"
|
||||
- Plan:
|
||||
Node Type: "Sort"
|
||||
Sort Key:
|
||||
- "COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint)"
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Parent Relationship: "Outer"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Group Key:
|
||||
- "intermediate_column_60130862146_0"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "pg_merge_job_60130862146"
|
||||
Alias: "pg_merge_job_60130862146"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Distributed Query:
|
||||
Executor: "Real-Time"
|
||||
Job:
|
||||
Task Count: 16
|
||||
Tasks Shown: "One of 16"
|
||||
Tasks:
|
||||
- Node: "host=localhost port=57637 dbname=regression"
|
||||
Remote Plan:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Hashed"
|
||||
Group Key:
|
||||
- "l_quantity"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "lineitem_mx_1220052"
|
||||
Alias: "lineitem_mx"
|
||||
|
||||
-- Test Text format
|
||||
EXPLAIN (COSTS FALSE, FORMAT TEXT)
|
||||
SELECT l_quantity, count(*) count_quantity FROM lineitem_mx
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
Distributed Query into pg_merge_job_60130862147
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Master Query
|
||||
-> Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(intermediate_column_60130862147_1))::bigint, '0'::bigint))))::bigint, '0'::bigint), intermediate_column_60130862147_0
|
||||
-> HashAggregate
|
||||
Group Key: intermediate_column_60130862147_0
|
||||
-> Seq Scan on pg_merge_job_60130862147
|
||||
Sort
|
||||
Sort Key: COALESCE((sum((COALESCE((sum(count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), l_quantity
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate
|
||||
Group Key: l_quantity
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
\c - - - :worker_2_port
|
||||
-- Test verbose
|
||||
EXPLAIN (COSTS FALSE, VERBOSE TRUE)
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx;
|
||||
Distributed Query into pg_merge_job_68720796739
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_mx_1220052 lineitem_mx
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
Master Query
|
||||
-> Aggregate
|
||||
Output: (sum(intermediate_column_68720796739_0) / (sum(intermediate_column_68720796739_1) / sum(intermediate_column_68720796739_2)))
|
||||
-> Seq Scan on pg_temp_2.pg_merge_job_68720796739
|
||||
Output: intermediate_column_68720796739_0, intermediate_column_68720796739_1, intermediate_column_68720796739_2
|
||||
Aggregate
|
||||
Output: (sum("?column?") / (sum("?column?_1") / sum("?column?_2")))
|
||||
-> Custom Scan (CitusScan)
|
||||
Output: "?column?", "?column?_1", "?column?_2"
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
Output: sum(l_quantity), sum(l_quantity), count(l_quantity)
|
||||
-> Seq Scan on public.lineitem_mx_1220052 lineitem_mx
|
||||
Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||
-- Test join
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT * FROM lineitem_mx
|
||||
JOIN orders_mx ON l_orderkey = o_orderkey AND l_quantity < 5.0
|
||||
ORDER BY l_quantity LIMIT 10;
|
||||
Distributed Query into pg_merge_job_68720796740
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem_mx.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey)
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_mx_1220068 orders_mx
|
||||
Master Query
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: intermediate_column_68720796740_4
|
||||
-> Seq Scan on pg_merge_job_68720796740
|
||||
Limit
|
||||
-> Sort
|
||||
Sort Key: l_quantity
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: lineitem_mx.l_quantity
|
||||
-> Hash Join
|
||||
Hash Cond: (lineitem_mx.l_orderkey = orders_mx.o_orderkey)
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_quantity < 5.0)
|
||||
-> Hash
|
||||
-> Seq Scan on orders_mx_1220068 orders_mx
|
||||
-- Test insert
|
||||
EXPLAIN (COSTS FALSE)
|
||||
INSERT INTO lineitem_mx VALUES(1,0);
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -335,7 +320,7 @@ EXPLAIN (COSTS FALSE)
|
|||
UPDATE lineitem_mx
|
||||
SET l_suppkey = 12
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -349,7 +334,7 @@ Distributed Query
|
|||
EXPLAIN (COSTS FALSE)
|
||||
DELETE FROM lineitem_mx
|
||||
WHERE l_orderkey = 1 AND l_partkey = 0;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -362,7 +347,7 @@ Distributed Query
|
|||
-- Test single-shard SELECT
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5;
|
||||
Distributed Query
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Router
|
||||
Task Count: 1
|
||||
Tasks Shown: All
|
||||
|
@ -382,106 +367,102 @@ t
|
|||
EXPLAIN (COSTS FALSE)
|
||||
CREATE TABLE explain_result AS
|
||||
SELECT * FROM lineitem_mx;
|
||||
Distributed Query into pg_merge_job_68720796741
|
||||
Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Master Query
|
||||
-> Seq Scan on pg_merge_job_68720796741
|
||||
-- Test all tasks output
|
||||
SET citus.explain_all_tasks TO on;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_68720796742
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220053 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220054 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220055 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220056 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220057 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220058 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220059 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220060 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220061 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220062 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220063 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220064 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220065 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220066 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220067 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_68720796742
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Real-Time
|
||||
Task Count: 16
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220053 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220054 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220055 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220056 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220057 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220058 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220059 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220060 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220061 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220062 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220063 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220064 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220065 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220066 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-> Task
|
||||
Node: host=localhost port=57638 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220067 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$);
|
||||
t
|
||||
|
@ -493,18 +474,16 @@ SET citus.task_executor_type TO 'task-tracker';
|
|||
SET citus.explain_all_tasks TO off;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
|
||||
Distributed Query into pg_merge_job_68720796745
|
||||
Executor: Task-Tracker
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_68720796745
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 16
|
||||
Tasks Shown: One of 16
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate
|
||||
-> Seq Scan on lineitem_mx_1220052 lineitem_mx
|
||||
Filter: (l_orderkey > 9030)
|
||||
-- Test re-partition join
|
||||
SET citus.large_table_shard_count TO 1;
|
||||
EXPLAIN (COSTS FALSE)
|
||||
|
@ -513,25 +492,23 @@ EXPLAIN (COSTS FALSE)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
Distributed Query into pg_merge_job_68720796750
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
Aggregate
|
||||
-> Custom Scan (CitusScan)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
-> MapMergeJob
|
||||
Map Task Count: 16
|
||||
Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 16
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
-> MapMergeJob
|
||||
Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
Master Query
|
||||
-> Aggregate
|
||||
-> Seq Scan on pg_merge_job_68720796750
|
||||
EXPLAIN (COSTS FALSE, FORMAT JSON)
|
||||
SELECT count(*)
|
||||
FROM lineitem_mx, orders_mx, customer_mx, supplier_mx
|
||||
|
@ -540,47 +517,44 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
|
|||
AND l_suppkey = s_suppkey;
|
||||
[
|
||||
{
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 4,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Plans": [
|
||||
{
|
||||
"Map Task Count": 4,
|
||||
"Merge Task Count": 4,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 16,
|
||||
"Merge Task Count": 4
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
"Node Type": "Custom Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Custom Plan Provider": "CitusScan",
|
||||
"Distributed Query": {
|
||||
"Executor": "Task-Tracker",
|
||||
"Job": {
|
||||
"Task Count": 4,
|
||||
"Tasks Shown": "None, not supported for re-partition queries",
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 4,
|
||||
"Merge Task Count": 4,
|
||||
"Depended Jobs": [
|
||||
{
|
||||
"Map Task Count": 16,
|
||||
"Merge Task Count": 4
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Map Task Count": 1,
|
||||
"Merge Task Count": 4
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"Master Query": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Aggregate",
|
||||
"Strategy": "Plain",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Seq Scan",
|
||||
"Parent Relationship": "Outer",
|
||||
"Relation Name": "pg_merge_job_68720796755",
|
||||
"Alias": "pg_merge_job_68720796755"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
SELECT true AS valid FROM explain_json($$
|
||||
|
@ -597,49 +571,46 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
|
|||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
<explain xmlns="http://www.postgresql.org/2009/explain">
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>4</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>4</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>16</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
<Master-Query>
|
||||
<Query>
|
||||
<Query>
|
||||
<Plan>
|
||||
<Node-Type>Aggregate</Node-Type>
|
||||
<Strategy>Plain</Strategy>
|
||||
<Plans>
|
||||
<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>
|
||||
</Plans>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>CitusScan</Custom-Plan-Provider>
|
||||
<Distributed-Query>
|
||||
<Executor>Task-Tracker</Executor>
|
||||
<Job>
|
||||
<Task-Count>4</Task-Count>
|
||||
<Tasks-Shown>None, not supported for re-partition queries</Tasks-Shown>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>4</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
<Depended-Jobs>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>16</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</MapMergeJob>
|
||||
<MapMergeJob>
|
||||
<Map-Task-Count>1</Map-Task-Count>
|
||||
<Merge-Task-Count>4</Merge-Task-Count>
|
||||
</MapMergeJob>
|
||||
</Depended-Jobs>
|
||||
</Job>
|
||||
</Distributed-Query>
|
||||
</Plan>
|
||||
</Query>
|
||||
</Master-Query>
|
||||
</Distributed-Query>
|
||||
</Plans>
|
||||
</Plan>
|
||||
</Query>
|
||||
</explain>
|
||||
SELECT true AS valid FROM explain_xml($$
|
||||
SELECT count(*)
|
||||
|
@ -654,26 +625,25 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
|
|||
WHERE l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND l_suppkey = s_suppkey;
|
||||
- Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 4
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
Depended Jobs:
|
||||
- Map Task Count: 16
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
Master Query:
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Plans:
|
||||
- Node Type: "Seq Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Relation Name: "pg_merge_job_68720796775"
|
||||
Alias: "pg_merge_job_68720796775"
|
||||
- Plan:
|
||||
Node Type: "Aggregate"
|
||||
Strategy: "Plain"
|
||||
Plans:
|
||||
- Node Type: "Custom Scan"
|
||||
Parent Relationship: "Outer"
|
||||
Custom Plan Provider: "CitusScan"
|
||||
Distributed Query:
|
||||
Executor: "Task-Tracker"
|
||||
Job:
|
||||
Task Count: 4
|
||||
Tasks Shown: "None, not supported for re-partition queries"
|
||||
Depended Jobs:
|
||||
- Map Task Count: 4
|
||||
Merge Task Count: 4
|
||||
Depended Jobs:
|
||||
- Map Task Count: 16
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
- Map Task Count: 1
|
||||
Merge Task Count: 4
|
||||
|
|
|
@ -173,9 +173,9 @@ EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
|
|||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||
WHERE repartition_udt.pk > 1;
|
||||
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------
|
||||
Distributed Query into pg_merge_job_535003
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
|
@ -185,9 +185,7 @@ LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_ot
|
|||
-> MapMergeJob
|
||||
Map Task Count: 5
|
||||
Merge Task Count: 4
|
||||
Master Query
|
||||
-> Seq Scan on pg_merge_job_535003 (cost=0.00..0.00 rows=0 width=0)
|
||||
(12 rows)
|
||||
(10 rows)
|
||||
|
||||
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||
|
|
|
@ -1489,7 +1489,6 @@ NOTICE: cannot use shard pruning with ANY/ALL (array expression)
|
|||
HINT: Consider rewriting the expression with OR/AND clauses.
|
||||
NOTICE: cannot use shard pruning with ANY/ALL (array expression)
|
||||
HINT: Consider rewriting the expression with OR/AND clauses.
|
||||
ERROR: cannot create temporary table within security-restricted operation
|
||||
|
||||
-- router planner/executor is disabled for task-tracker executor
|
||||
-- following query is router plannable, but router planner is disabled
|
||||
|
|
|
@ -170,25 +170,31 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename,
|
|||
-- one shard.
|
||||
EXPLAIN SELECT count(*) FROM varchar_partitioned_table WHERE varchar_column = 'BA2';
|
||||
DEBUG: predicate pruning for shardId 100
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*) FROM array_partitioned_table
|
||||
WHERE array_column > '{BA1000U2AMO4ZGX, BZZXSP27F21T6}';
|
||||
DEBUG: predicate pruning for shardId 102
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*) FROM composite_partitioned_table
|
||||
WHERE composite_column < '(b,5,c)'::composite_type;
|
||||
DEBUG: predicate pruning for shardId 105
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
SET client_min_messages TO NOTICE;
|
||||
|
|
|
@ -76,25 +76,145 @@ SELECT pg_ls_dir('base/pgsql_job_cache');
|
|||
-- transaction.
|
||||
BEGIN;
|
||||
DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_00;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_01 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_01;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_02 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_02;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_03 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_03;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_04 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_04;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_05 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_05;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_06 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_06;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_07 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_07;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_08 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_08;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_09 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_09;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_10 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_10;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_11 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_11;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_12 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_12;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_13 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_13;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_14 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_14;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_15 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_15;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_16 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_16;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_17 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_17;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_18 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_18;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
DECLARE c_19 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_19;
|
||||
revenue
|
||||
---------------
|
||||
22770844.7654
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f;
|
||||
f
|
||||
-----------------
|
||||
|
|
|
@ -181,9 +181,9 @@ EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
|
|||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||
WHERE repartition_udt.pk > 1;
|
||||
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------
|
||||
Distributed Query into pg_merge_job_535003
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------
|
||||
Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Task-Tracker
|
||||
Task Count: 4
|
||||
Tasks Shown: None, not supported for re-partition queries
|
||||
|
@ -193,9 +193,7 @@ LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_ot
|
|||
-> MapMergeJob
|
||||
Map Task Count: 5
|
||||
Merge Task Count: 4
|
||||
Master Query
|
||||
-> Seq Scan on pg_merge_job_535003 (cost=0.00..0.00 rows=0 width=0)
|
||||
(12 rows)
|
||||
(10 rows)
|
||||
|
||||
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||
|
|
|
@ -2210,12 +2210,12 @@ CONTEXT: PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUE
|
|||
(5 rows)
|
||||
|
||||
-- materialized views can be created for router plannable queries
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash AS
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash_empty AS
|
||||
SELECT * FROM articles_hash WHERE author_id = 1;
|
||||
DEBUG: predicate pruning for shardId 840001
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
SELECT * FROM mv_articles_hash;
|
||||
SELECT * FROM mv_articles_hash_empty;
|
||||
id | author_id | title | word_count
|
||||
----+-----------+--------------+------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -2225,14 +2225,27 @@ SELECT * FROM mv_articles_hash;
|
|||
41 | 1 | aznavour | 11814
|
||||
(5 rows)
|
||||
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash_error AS
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash_data AS
|
||||
SELECT * FROM articles_hash WHERE author_id in (1,2);
|
||||
NOTICE: cannot use shard pruning with ANY/ALL (array expression)
|
||||
HINT: Consider rewriting the expression with OR/AND clauses.
|
||||
NOTICE: cannot use shard pruning with ANY/ALL (array expression)
|
||||
HINT: Consider rewriting the expression with OR/AND clauses.
|
||||
ERROR: cannot create temporary table within security-restricted operation
|
||||
|
||||
SELECT * FROM mv_articles_hash_data;
|
||||
id | author_id | title | word_count
|
||||
----+-----------+--------------+------------
|
||||
1 | 1 | arsenous | 9572
|
||||
11 | 1 | alamo | 1347
|
||||
21 | 1 | arcading | 5890
|
||||
31 | 1 | athwartships | 7271
|
||||
41 | 1 | aznavour | 11814
|
||||
2 | 2 | abducing | 13642
|
||||
12 | 2 | archiblast | 18185
|
||||
22 | 2 | antipope | 2728
|
||||
32 | 2 | amazon | 11342
|
||||
42 | 2 | ausable | 15885
|
||||
(10 rows)
|
||||
|
||||
-- router planner/executor is now enabled for task-tracker executor
|
||||
SET citus.task_executor_type to 'task-tracker';
|
||||
SELECT id
|
||||
|
@ -2346,7 +2359,8 @@ DROP USER router_user;
|
|||
DROP TABLE failure_test;
|
||||
DROP FUNCTION author_articles_max_id();
|
||||
DROP FUNCTION author_articles_id_word_count();
|
||||
DROP MATERIALIZED VIEW mv_articles_hash;
|
||||
DROP MATERIALIZED VIEW mv_articles_hash_empty;
|
||||
DROP MATERIALIZED VIEW mv_articles_hash_data;
|
||||
DROP TABLE articles_hash;
|
||||
DROP TABLE articles_single_shard_hash;
|
||||
DROP TABLE authors_hash;
|
||||
|
|
|
@ -60,10 +60,12 @@ DEBUG: assigned task 6 to node localhost:57637
|
|||
DEBUG: assigned task 2 to node localhost:57638
|
||||
DEBUG: assigned task 4 to node localhost:57637
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
||||
DEBUG: StartTransactionCommand
|
||||
|
@ -72,10 +74,12 @@ DEBUG: assigned task 6 to node localhost:57637
|
|||
DEBUG: assigned task 2 to node localhost:57638
|
||||
DEBUG: assigned task 4 to node localhost:57637
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Next test the first-replica task assignment policy
|
||||
SET citus.task_assignment_policy TO 'first-replica';
|
||||
|
@ -89,10 +93,12 @@ DEBUG: assigned task 6 to node localhost:57637
|
|||
DEBUG: assigned task 4 to node localhost:57637
|
||||
DEBUG: assigned task 2 to node localhost:57638
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
||||
DEBUG: StartTransactionCommand
|
||||
|
@ -101,10 +107,12 @@ DEBUG: assigned task 6 to node localhost:57637
|
|||
DEBUG: assigned task 4 to node localhost:57637
|
||||
DEBUG: assigned task 2 to node localhost:57638
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
-- Round-robin task assignment relies on the current jobId. We therefore need to
|
||||
-- ensure that jobIds start with an odd number here; this way, task assignment
|
||||
|
@ -132,10 +140,12 @@ DEBUG: assigned task 6 to node localhost:57637
|
|||
DEBUG: assigned task 4 to node localhost:57637
|
||||
DEBUG: assigned task 2 to node localhost:57638
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
||||
DEBUG: StartTransactionCommand
|
||||
|
@ -144,10 +154,12 @@ DEBUG: assigned task 6 to node localhost:57638
|
|||
DEBUG: assigned task 4 to node localhost:57638
|
||||
DEBUG: assigned task 2 to node localhost:57637
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
||||
DEBUG: StartTransactionCommand
|
||||
|
@ -156,10 +168,12 @@ DEBUG: assigned task 6 to node localhost:57637
|
|||
DEBUG: assigned task 4 to node localhost:57637
|
||||
DEBUG: assigned task 2 to node localhost:57638
|
||||
DEBUG: CommitTransactionCommand
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------
|
||||
explain statements for distributed queries are not enabled
|
||||
(1 row)
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
explain statements for distributed queries are not enabled
|
||||
(3 rows)
|
||||
|
||||
RESET citus.task_assignment_policy;
|
||||
DEBUG: StartTransactionCommand
|
||||
|
|
|
@ -283,12 +283,12 @@ DROP TABLE ddl_commands;
|
|||
-- Distributed SELECTs cannot appear after ALTER
|
||||
BEGIN;
|
||||
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||
SELECT l_orderkey FROM lineitem_alter LIMIT 0;
|
||||
SELECT count(*) FROM lineitem_alter;
|
||||
COMMIT;
|
||||
|
||||
-- but are allowed before
|
||||
BEGIN;
|
||||
SELECT l_orderkey FROM lineitem_alter LIMIT 0;
|
||||
SELECT count(*) FROM lineitem_alter;
|
||||
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||
COMMIT;
|
||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||
|
|
|
@ -596,15 +596,16 @@ BEGIN;
|
|||
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||
NOTICE: using one-phase commit for distributed DDL commands
|
||||
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
|
||||
SELECT l_orderkey FROM lineitem_alter LIMIT 0;
|
||||
SELECT count(*) FROM lineitem_alter;
|
||||
ERROR: cannot open new connections after the first modification command within a transaction
|
||||
COMMIT;
|
||||
-- but are allowed before
|
||||
BEGIN;
|
||||
SELECT l_orderkey FROM lineitem_alter LIMIT 0;
|
||||
l_orderkey
|
||||
------------
|
||||
(0 rows)
|
||||
SELECT count(*) FROM lineitem_alter;
|
||||
count
|
||||
-------
|
||||
18000
|
||||
(1 row)
|
||||
|
||||
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||
COMMIT;
|
||||
|
|
|
@ -764,31 +764,29 @@ FROM
|
|||
GROUP BY
|
||||
tenant_id,
|
||||
user_id) AS subquery;
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Distributed Query into pg_merge_job_270014
|
||||
Executor: Real-Time
|
||||
Task Count: 2
|
||||
Tasks Shown: One of 2
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate (cost=40.01..40.02 rows=1 width=16)
|
||||
-> GroupAggregate (cost=39.89..39.99 rows=1 width=48)
|
||||
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Merge Join (cost=39.89..39.97 rows=1 width=540)
|
||||
Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id)))
|
||||
-> Sort (cost=28.08..28.09 rows=6 width=32)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Sort (cost=11.81..11.82 rows=3 width=556)
|
||||
Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id)
|
||||
-> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556)
|
||||
Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[]))
|
||||
Master Query
|
||||
-> Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Seq Scan on pg_merge_job_270014 (cost=0.00..0.00 rows=0 width=0)
|
||||
(22 rows)
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Real-Time
|
||||
Task Count: 2
|
||||
Tasks Shown: One of 2
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Aggregate (cost=40.01..40.02 rows=1 width=16)
|
||||
-> GroupAggregate (cost=39.89..39.99 rows=1 width=48)
|
||||
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Merge Join (cost=39.89..39.97 rows=1 width=540)
|
||||
Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id)))
|
||||
-> Sort (cost=28.08..28.09 rows=6 width=32)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Sort (cost=11.81..11.82 rows=3 width=556)
|
||||
Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id)
|
||||
-> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556)
|
||||
Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[]))
|
||||
(20 rows)
|
||||
|
||||
-- Union and left join subquery pushdown
|
||||
EXPLAIN SELECT
|
||||
|
@ -853,52 +851,50 @@ FROM
|
|||
hasdone) AS subquery_top
|
||||
GROUP BY
|
||||
hasdone;
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Distributed Query into pg_merge_job_270015
|
||||
Executor: Real-Time
|
||||
Task Count: 2
|
||||
Tasks Shown: One of 2
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> GroupAggregate (cost=91.93..91.98 rows=2 width=48)
|
||||
Group Key: subquery_top.hasdone
|
||||
-> Sort (cost=91.93..91.93 rows=2 width=64)
|
||||
Sort Key: subquery_top.hasdone
|
||||
-> Subquery Scan on subquery_top (cost=91.85..91.92 rows=2 width=64)
|
||||
-> GroupAggregate (cost=91.85..91.90 rows=2 width=112)
|
||||
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text)
|
||||
-> Sort (cost=91.85..91.85 rows=2 width=88)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text)
|
||||
-> Merge Left Join (cost=91.75..91.84 rows=2 width=88)
|
||||
Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id)))
|
||||
-> Unique (cost=79.46..79.48 rows=2 width=56)
|
||||
-> Sort (cost=79.46..79.47 rows=2 width=56)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time
|
||||
-> Append (cost=0.00..79.45 rows=2 width=56)
|
||||
-> Nested Loop (cost=0.00..39.72 rows=1 width=56)
|
||||
Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id))
|
||||
-> Seq Scan on events_270009 events (cost=0.00..11.62 rows=1 width=40)
|
||||
Filter: ((event_type)::text = 'click'::text)
|
||||
-> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Nested Loop (cost=0.00..39.72 rows=1 width=56)
|
||||
Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id))
|
||||
-> Seq Scan on events_270009 events_1 (cost=0.00..11.62 rows=1 width=40)
|
||||
Filter: ((event_type)::text = 'submit'::text)
|
||||
-> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Materialize (cost=12.29..12.31 rows=1 width=48)
|
||||
-> Unique (cost=12.29..12.30 rows=1 width=80)
|
||||
-> Sort (cost=12.29..12.29 rows=1 width=80)
|
||||
Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id)
|
||||
-> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=80)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text))
|
||||
Master Query
|
||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: intermediate_column_270015_2
|
||||
-> Seq Scan on pg_merge_job_270015 (cost=0.00..0.00 rows=0 width=0)
|
||||
(43 rows)
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
||||
Group Key: hasdone
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Real-Time
|
||||
Task Count: 2
|
||||
Tasks Shown: One of 2
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> GroupAggregate (cost=91.93..91.98 rows=2 width=48)
|
||||
Group Key: subquery_top.hasdone
|
||||
-> Sort (cost=91.93..91.93 rows=2 width=64)
|
||||
Sort Key: subquery_top.hasdone
|
||||
-> Subquery Scan on subquery_top (cost=91.85..91.92 rows=2 width=64)
|
||||
-> GroupAggregate (cost=91.85..91.90 rows=2 width=112)
|
||||
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text)
|
||||
-> Sort (cost=91.85..91.85 rows=2 width=88)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text)
|
||||
-> Merge Left Join (cost=91.75..91.84 rows=2 width=88)
|
||||
Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id)))
|
||||
-> Unique (cost=79.46..79.48 rows=2 width=56)
|
||||
-> Sort (cost=79.46..79.47 rows=2 width=56)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time
|
||||
-> Append (cost=0.00..79.45 rows=2 width=56)
|
||||
-> Nested Loop (cost=0.00..39.72 rows=1 width=56)
|
||||
Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id))
|
||||
-> Seq Scan on events_270009 events (cost=0.00..11.62 rows=1 width=40)
|
||||
Filter: ((event_type)::text = 'click'::text)
|
||||
-> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Nested Loop (cost=0.00..39.72 rows=1 width=56)
|
||||
Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id))
|
||||
-> Seq Scan on events_270009 events_1 (cost=0.00..11.62 rows=1 width=40)
|
||||
Filter: ((event_type)::text = 'submit'::text)
|
||||
-> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Materialize (cost=12.29..12.31 rows=1 width=48)
|
||||
-> Unique (cost=12.29..12.30 rows=1 width=80)
|
||||
-> Sort (cost=12.29..12.29 rows=1 width=80)
|
||||
Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id)
|
||||
-> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=80)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text))
|
||||
(41 rows)
|
||||
|
||||
-- Union, left join and having subquery pushdown
|
||||
EXPLAIN SELECT
|
||||
|
@ -1022,37 +1018,35 @@ ORDER BY
|
|||
user_lastseen DESC
|
||||
LIMIT
|
||||
10;
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Distributed Query into pg_merge_job_270017
|
||||
Executor: Real-Time
|
||||
Task Count: 2
|
||||
Tasks Shown: One of 2
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit (cost=100.43..100.44 rows=6 width=56)
|
||||
-> Sort (cost=100.43..100.44 rows=6 width=56)
|
||||
Sort Key: (max(users.lastseen)) DESC
|
||||
-> GroupAggregate (cost=100.14..100.29 rows=6 width=56)
|
||||
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Sort (cost=100.14..100.16 rows=6 width=548)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548)
|
||||
-> Limit (cost=28.08..28.09 rows=6 width=24)
|
||||
-> Sort (cost=28.08..28.09 rows=6 width=24)
|
||||
Sort Key: users.lastseen DESC
|
||||
-> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=24)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Limit (cost=11.96..11.96 rows=1 width=524)
|
||||
-> Sort (cost=11.96..11.96 rows=1 width=524)
|
||||
Sort Key: events.event_time DESC
|
||||
-> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524)
|
||||
Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id)))
|
||||
Master Query
|
||||
-> Limit (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Sort (cost=0.00..0.00 rows=0 width=0)
|
||||
Sort Key: intermediate_column_270017_2 DESC
|
||||
-> Seq Scan on pg_merge_job_270017 (cost=0.00..0.00 rows=0 width=0)
|
||||
(29 rows)
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Limit (cost=0.00..0.00 rows=0 width=0)
|
||||
-> Sort (cost=0.00..0.00 rows=0 width=0)
|
||||
Sort Key: user_lastseen DESC
|
||||
-> Custom Scan (CitusScan) (cost=0.00..0.00 rows=0 width=0)
|
||||
Executor: Real-Time
|
||||
Task Count: 2
|
||||
Tasks Shown: One of 2
|
||||
-> Task
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> Limit (cost=100.43..100.44 rows=6 width=56)
|
||||
-> Sort (cost=100.43..100.44 rows=6 width=56)
|
||||
Sort Key: (max(users.lastseen)) DESC
|
||||
-> GroupAggregate (cost=100.14..100.29 rows=6 width=56)
|
||||
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Sort (cost=100.14..100.16 rows=6 width=548)
|
||||
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
|
||||
-> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548)
|
||||
-> Limit (cost=28.08..28.09 rows=6 width=24)
|
||||
-> Sort (cost=28.08..28.09 rows=6 width=24)
|
||||
Sort Key: users.lastseen DESC
|
||||
-> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=24)
|
||||
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
|
||||
-> Limit (cost=11.96..11.96 rows=1 width=524)
|
||||
-> Sort (cost=11.96..11.96 rows=1 width=524)
|
||||
Sort Key: events.event_time DESC
|
||||
-> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524)
|
||||
Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id)))
|
||||
(27 rows)
|
||||
|
||||
SET citus.enable_router_execution TO 'true';
|
||||
|
|
|
@ -11,6 +11,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 650000;
|
|||
|
||||
SET citus.explain_distributed_queries TO off;
|
||||
SET citus.log_multi_join_order TO TRUE;
|
||||
SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwhise
|
||||
SET client_min_messages TO DEBUG2;
|
||||
|
||||
-- Create new table definitions for use in testing in distributed planning and
|
||||
|
|
|
@ -11,6 +11,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 660000;
|
|||
|
||||
SET citus.explain_distributed_queries TO off;
|
||||
SET citus.log_multi_join_order TO TRUE;
|
||||
SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwhise
|
||||
SET client_min_messages TO LOG;
|
||||
|
||||
-- Change configuration to treat lineitem, orders, customer, and part tables as
|
||||
|
|
|
@ -46,25 +46,45 @@ SELECT pg_ls_dir('base/pgsql_job_cache');
|
|||
-- transaction.
|
||||
BEGIN;
|
||||
DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_00;
|
||||
DECLARE c_01 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_01;
|
||||
DECLARE c_02 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_02;
|
||||
DECLARE c_03 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_03;
|
||||
DECLARE c_04 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_04;
|
||||
DECLARE c_05 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_05;
|
||||
DECLARE c_06 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_06;
|
||||
DECLARE c_07 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_07;
|
||||
DECLARE c_08 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_08;
|
||||
DECLARE c_09 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_09;
|
||||
DECLARE c_10 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_10;
|
||||
DECLARE c_11 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_11;
|
||||
DECLARE c_12 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_12;
|
||||
DECLARE c_13 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_13;
|
||||
DECLARE c_14 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_14;
|
||||
DECLARE c_15 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_15;
|
||||
DECLARE c_16 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_16;
|
||||
DECLARE c_17 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_17;
|
||||
DECLARE c_18 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_18;
|
||||
DECLARE c_19 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem;
|
||||
FETCH 1 FROM c_19;
|
||||
SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f;
|
||||
-- close first, 17th (first after re-alloc) and last cursor.
|
||||
CLOSE c_00;
|
||||
|
|
|
@ -988,14 +988,14 @@ $$ LANGUAGE plpgsql;
|
|||
SELECT * FROM author_articles_id_word_count();
|
||||
|
||||
-- materialized views can be created for router plannable queries
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash AS
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash_empty AS
|
||||
SELECT * FROM articles_hash WHERE author_id = 1;
|
||||
SELECT * FROM mv_articles_hash_empty;
|
||||
|
||||
SELECT * FROM mv_articles_hash;
|
||||
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash_error AS
|
||||
CREATE MATERIALIZED VIEW mv_articles_hash_data AS
|
||||
SELECT * FROM articles_hash WHERE author_id in (1,2);
|
||||
|
||||
SELECT * FROM mv_articles_hash_data;
|
||||
|
||||
-- router planner/executor is now enabled for task-tracker executor
|
||||
SET citus.task_executor_type to 'task-tracker';
|
||||
SELECT id
|
||||
|
@ -1053,7 +1053,8 @@ DROP TABLE failure_test;
|
|||
DROP FUNCTION author_articles_max_id();
|
||||
DROP FUNCTION author_articles_id_word_count();
|
||||
|
||||
DROP MATERIALIZED VIEW mv_articles_hash;
|
||||
DROP MATERIALIZED VIEW mv_articles_hash_empty;
|
||||
DROP MATERIALIZED VIEW mv_articles_hash_data;
|
||||
|
||||
DROP TABLE articles_hash;
|
||||
DROP TABLE articles_single_shard_hash;
|
||||
|
|
Loading…
Reference in New Issue