Rename masterQuery to combineQuery

pull/3919/head
Marco Slot 2020-06-16 18:09:16 +02:00
parent 0259815d3a
commit 2a3234ca26
17 changed files with 104 additions and 105 deletions

View File

@ -1059,11 +1059,11 @@ IsRedistributablePlan(Plan *selectPlan)
return false;
}
if (distSelectPlan->masterQuery != NULL)
if (distSelectPlan->combineQuery != NULL)
{
Query *masterQuery = (Query *) distSelectPlan->masterQuery;
Query *combineQuery = (Query *) distSelectPlan->combineQuery;
if (contain_nextval_expression_walker((Node *) masterQuery->targetList, NULL))
if (contain_nextval_expression_walker((Node *) combineQuery->targetList, NULL))
{
/* nextval needs to be evaluated on the coordinator */
return false;

View File

@ -26,7 +26,7 @@
#include "distributed/listutils.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/multi_executor.h"
#include "distributed/merge_planner.h"
#include "distributed/combine_query_planner.h"
#include "distributed/distributed_planner.h"
#include "distributed/multi_router_planner.h"
#include "distributed/multi_resowner.h"

View File

@ -1,13 +1,11 @@
/*-------------------------------------------------------------------------
*
* merge_planner.c
* Routines for building create table and select into table statements on the
* master node.
* combine_query_planner.c
* Routines for planning the combine query that runs on the coordinator
* to combine results from the workers.
*
* Copyright (c) Citus Data, Inc.
*
* $Id$
*
*-------------------------------------------------------------------------
*/
@ -20,7 +18,7 @@
#include "distributed/insert_select_planner.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/merge_planner.h"
#include "distributed/combine_query_planner.h"
#include "distributed/multi_physical_planner.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@ -28,9 +26,9 @@
#include "optimizer/planner.h"
#include "rewrite/rewriteManip.h"
static List * MasterTargetList(List *workerTargetList);
static PlannedStmt * BuildSelectStatementViaStdPlanner(Query *masterQuery,
List *masterTargetList,
static List * RemoteScanTargetList(List *workerTargetList);
static PlannedStmt * BuildSelectStatementViaStdPlanner(Query *combineQuery,
List *remoteScanTargetList,
CustomScan *remoteScan);
static bool FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result);
@ -53,33 +51,33 @@ static CustomPathMethods CitusCustomScanPathMethods = {
/*
* MasterNodeSelectPlan takes in a distributed plan and a custom scan node which
* wraps remote part of the plan. This function finds the master node query
* structure in the multi plan, and builds the final select plan to execute on
* the tuples returned by remote scan on the master node. Note that this select
* wraps remote part of the plan. This function finds the combine query structure
* in the multi plan, and builds the final select plan to execute on the tuples
* returned by remote scan on the coordinator node. Note that this select
* plan is executed after result files are retrieved from worker nodes and
* filled into the tuple store inside provided custom scan.
*/
PlannedStmt *
MasterNodeSelectPlan(DistributedPlan *distributedPlan, CustomScan *remoteScan)
{
Query *masterQuery = distributedPlan->masterQuery;
Query *combineQuery = distributedPlan->combineQuery;
Job *workerJob = distributedPlan->workerJob;
List *workerTargetList = workerJob->jobQuery->targetList;
List *masterTargetList = MasterTargetList(workerTargetList);
return BuildSelectStatementViaStdPlanner(masterQuery, masterTargetList, remoteScan);
List *remoteScanTargetList = RemoteScanTargetList(workerTargetList);
return BuildSelectStatementViaStdPlanner(combineQuery, remoteScanTargetList,
remoteScan);
}
/*
* MasterTargetList uses the given worker target list's expressions, and creates
* a target list for the master node. This master target list keeps the
* temporary table's columns on the master node.
* RemoteScanTargetList uses the given worker target list's expressions, and creates
* a target list for the remote scan on the coordinator node.
*/
static List *
MasterTargetList(List *workerTargetList)
RemoteScanTargetList(List *workerTargetList)
{
List *masterTargetList = NIL;
List *remoteScanTargetList = NIL;
const Index tableId = 1;
AttrNumber columnId = 1;
@ -93,29 +91,30 @@ MasterTargetList(List *workerTargetList)
continue;
}
Var *masterColumn = makeVarFromTargetEntry(tableId, workerTargetEntry);
masterColumn->varattno = columnId;
masterColumn->varoattno = columnId;
Var *remoteScanColumn = makeVarFromTargetEntry(tableId, workerTargetEntry);
remoteScanColumn->varattno = columnId;
remoteScanColumn->varoattno = columnId;
columnId++;
if (masterColumn->vartype == RECORDOID || masterColumn->vartype == RECORDARRAYOID)
if (remoteScanColumn->vartype == RECORDOID || remoteScanColumn->vartype ==
RECORDARRAYOID)
{
masterColumn->vartypmod = BlessRecordExpression(workerTargetEntry->expr);
remoteScanColumn->vartypmod = BlessRecordExpression(workerTargetEntry->expr);
}
/*
* The master target entry has two pieces to it. The first piece is the
* The remote scan target entry has two pieces to it. The first piece is the
* target entry's expression, which we set to the newly created column.
* The second piece is sort and group clauses that we implicitly copy
* from the worker target entry. Note that any changes to worker target
* entry's sort and group clauses will *break* us here.
*/
TargetEntry *masterTargetEntry = flatCopyTargetEntry(workerTargetEntry);
masterTargetEntry->expr = (Expr *) masterColumn;
masterTargetList = lappend(masterTargetList, masterTargetEntry);
TargetEntry *remoteScanTargetEntry = flatCopyTargetEntry(workerTargetEntry);
remoteScanTargetEntry->expr = (Expr *) remoteScanColumn;
remoteScanTargetList = lappend(remoteScanTargetList, remoteScanTargetEntry);
}
return masterTargetList;
return remoteScanTargetList;
}
@ -147,7 +146,7 @@ CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo,
* set that would disallow hash aggregates to be used.
*
* Ideally we would be able to provide estimates close to postgres' estimates on the
* workers to let the standard planner choose an optimal solution for the masterQuery.
* workers to let the standard planner choose an optimal solution for the combineQuery.
*/
path->custom_path.path.rows = 100000;
path->remoteScan = remoteScan;
@ -195,15 +194,15 @@ CitusCustomScanPathPlan(PlannerInfo *root,
/*
* The custom_scan_tlist contains target entries for to the "output" of the call
* to citus_extradata_container, which is actually replaced by a CustomScan.
* The target entries are initialized with varno 1 (see MasterTargetList), since
* it's currently the only relation in the join tree of the masterQuery.
* The target entries are initialized with varno 1 (see RemoteScanTargetList), since
* it's currently the only relation in the join tree of the combineQuery.
*
* If the citus_extradata_container function call is not the first relation to
* appear in the flattened rtable for the entire plan, then varno is now pointing
* to the wrong relation and needs to be updated.
*
* Example:
* When the masterQuery field of the DistributedPlan is
* When the combineQuery field of the DistributedPlan is
* INSERT INTO local SELECT .. FROM citus_extradata_container.
* In that case the varno of citusdata_extradata_container should be 3, because
* it is preceded range table entries for "local" and the subquery.
@ -236,39 +235,39 @@ CitusCustomScanPathPlan(PlannerInfo *root,
/*
* BuildSelectStatementViaStdPlanner creates a PlannedStmt where it combines the
* masterQuery and the remoteScan. It utilizes the standard_planner from postgres to
* create a plan based on the masterQuery.
* combineQuery and the remoteScan. It utilizes the standard_planner from postgres to
* create a plan based on the combineQuery.
*/
static PlannedStmt *
BuildSelectStatementViaStdPlanner(Query *masterQuery, List *masterTargetList,
BuildSelectStatementViaStdPlanner(Query *combineQuery, List *remoteScanTargetList,
CustomScan *remoteScan)
{
/*
* the standard planner will scribble on the target list. Since it is essential to not
* change the custom_scan_tlist we copy the target list before adding them to any.
* The masterTargetList is used in the end to extract the column names to be added to
* The remoteScanTargetList is used in the end to extract the column names to be added to
* the alias we will create for the CustomScan, (expressed as the
* citus_extradata_container function call in the masterQuery).
* citus_extradata_container function call in the combineQuery).
*/
remoteScan->custom_scan_tlist = copyObject(masterTargetList);
remoteScan->scan.plan.targetlist = copyObject(masterTargetList);
remoteScan->custom_scan_tlist = copyObject(remoteScanTargetList);
remoteScan->scan.plan.targetlist = copyObject(remoteScanTargetList);
/*
* We will overwrite the alias of the rangetable which describes the custom scan.
* Ideally we would have set the correct column names and alias on the range table in
* the master query already when we inserted the extra data container. This could be
* the combine query already when we inserted the extra data container. This could be
* improved in the future.
*/
/* find the rangetable entry for the extradata container and overwrite its alias */
RangeTblEntry *extradataContainerRTE = NULL;
FindCitusExtradataContainerRTE((Node *) masterQuery, &extradataContainerRTE);
FindCitusExtradataContainerRTE((Node *) combineQuery, &extradataContainerRTE);
if (extradataContainerRTE != NULL)
{
/* extract column names from the masterTargetList */
/* extract column names from the remoteScanTargetList */
List *columnNameList = NIL;
TargetEntry *targetEntry = NULL;
foreach_ptr(targetEntry, masterTargetList)
foreach_ptr(targetEntry, remoteScanTargetList)
{
columnNameList = lappend(columnNameList, makeString(targetEntry->resname));
}
@ -276,15 +275,15 @@ BuildSelectStatementViaStdPlanner(Query *masterQuery, List *masterTargetList,
}
/*
* Print the master query at debug level 4. Since serializing the query is relatively
* Print the combine query at debug level 4. Since serializing the query is relatively
* cpu intensive we only perform that if we are actually logging DEBUG4.
*/
const int logMasterQueryLevel = DEBUG4;
if (IsLoggableLevel(logMasterQueryLevel))
const int logCombineQueryLevel = DEBUG4;
if (IsLoggableLevel(logCombineQueryLevel))
{
StringInfo queryString = makeStringInfo();
pg_get_query_def(masterQuery, queryString);
elog(logMasterQueryLevel, "master query: %s", queryString->data);
pg_get_query_def(combineQuery, queryString);
elog(logCombineQueryLevel, "combine query: %s", queryString->data);
}
PlannedStmt *standardStmt = NULL;
@ -296,7 +295,7 @@ BuildSelectStatementViaStdPlanner(Query *masterQuery, List *masterTargetList,
ReplaceCitusExtraDataContainer = true;
ReplaceCitusExtraDataContainerWithCustomScan = remoteScan;
standardStmt = standard_planner(masterQuery, 0, NULL);
standardStmt = standard_planner(combineQuery, 0, NULL);
ReplaceCitusExtraDataContainer = false;
ReplaceCitusExtraDataContainerWithCustomScan = NULL;

View File

@ -38,7 +38,7 @@
#include "distributed/multi_logical_planner.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/merge_planner.h"
#include "distributed/combine_query_planner.h"
#include "distributed/multi_router_planner.h"
#include "distributed/query_utils.h"
#include "distributed/recursive_planning.h"
@ -1339,13 +1339,13 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan)
* Record subplans used by distributed plan to make intermediate result
* pruning easier.
*
* We do this before finalizing the plan, because the masterQuery is
* We do this before finalizing the plan, because the combineQuery is
* rewritten by standard_planner in FinalizeNonRouterPlan.
*/
distributedPlan->usedSubPlanNodeList = FindSubPlanUsages(distributedPlan);
}
if (distributedPlan->masterQuery)
if (distributedPlan->combineQuery)
{
finalPlan = FinalizeNonRouterPlan(localPlan, distributedPlan, customScan);
}

View File

@ -378,7 +378,7 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
distributedPlan = CitusMakeNode(DistributedPlan);
distributedPlan->workerJob = job;
distributedPlan->masterQuery = NULL;
distributedPlan->combineQuery = NULL;
distributedPlan->routerExecutable = true;
distributedPlan->expectResults = true;

View File

@ -63,8 +63,8 @@ static Task * RouterModifyTaskForShardInterval(Query *originalQuery,
uint32 taskIdIndex,
bool allRelationsJoinedOnPartitionKey,
DeferredErrorMessage **routerPlannerError);
static Query * CreateMasterQueryForRouterPlan(DistributedPlan *distPlan);
static List * CreateTargetListForMasterQuery(List *targetList);
static Query * CreateCombineQueryForRouterPlan(DistributedPlan *distPlan);
static List * CreateTargetListForCombineQuery(List *targetList);
static DeferredErrorMessage * DistributedInsertSelectSupported(Query *queryTree,
RangeTblEntry *insertRte,
RangeTblEntry *subqueryRte,
@ -300,7 +300,7 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
/* and finally the multi plan */
distributedPlan->workerJob = workerJob;
distributedPlan->masterQuery = NULL;
distributedPlan->combineQuery = NULL;
distributedPlan->routerExecutable = true;
distributedPlan->expectResults = originalQuery->returningList != NIL;
distributedPlan->targetRelationId = targetRelationId;
@ -316,10 +316,10 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
* To create the plan, this function first creates a distributed plan for the SELECT
* part. Then puts it as a subquery to the original (non-distributed) INSERT query as
* a subquery. Finally, it puts this INSERT query, which now has a distributed SELECT
* subquery, in the masterQuery.
* subquery, in the combineQuery.
*
* If the SELECT query is a router query, whose distributed plan does not have a
* masterQuery, this function also creates a dummy masterQuery for that.
* combineQuery, this function also creates a dummy combineQuery for that.
*/
DistributedPlan *
CreateInsertSelectIntoLocalTablePlan(uint64 planId, Query *originalQuery, ParamListInfo
@ -347,40 +347,40 @@ CreateInsertSelectIntoLocalTablePlan(uint64 planId, Query *originalQuery, ParamL
return distPlan;
}
if (distPlan->masterQuery == NULL)
if (distPlan->combineQuery == NULL)
{
/*
* For router queries, we construct a synthetic master query that simply passes
* on the results of the remote tasks, which we can then use as the select in
* the INSERT .. SELECT.
*/
distPlan->masterQuery = CreateMasterQueryForRouterPlan(
distPlan->combineQuery = CreateCombineQueryForRouterPlan(
distPlan);
}
/*
* masterQuery of a distributed select is for combining the results from
* combineQuery of a distributed select is for combining the results from
* worker nodes on the coordinator node. Putting it as a subquery to the
* INSERT query, causes the INSERT query to insert the combined select value
* from the workers. And making the resulting insert query the masterQuery
* from the workers. And making the resulting insert query the combineQuery
* let's us execute this insert command.
*
* So this operation makes the master query insert the result of the
* distributed select instead of returning it.
*/
selectRte->subquery = distPlan->masterQuery;
distPlan->masterQuery = originalQuery;
selectRte->subquery = distPlan->combineQuery;
distPlan->combineQuery = originalQuery;
return distPlan;
}
/*
* CreateMasterQueryForRouterPlan is used for creating a dummy masterQuery
* CreateCombineQueryForRouterPlan is used for creating a dummy combineQuery
* for a router plan, since router plans normally don't have one.
*/
static Query *
CreateMasterQueryForRouterPlan(DistributedPlan *distPlan)
CreateCombineQueryForRouterPlan(DistributedPlan *distPlan)
{
const Index insertTableId = 1;
List *tableIdList = list_make1(makeInteger(insertTableId));
@ -423,7 +423,7 @@ CreateMasterQueryForRouterPlan(DistributedPlan *distPlan)
funcColumnTypeMods,
funcCollations);
List *targetList = CreateTargetListForMasterQuery(dependentTargetList);
List *targetList = CreateTargetListForCombineQuery(dependentTargetList);
RangeTblRef *rangeTableRef = makeNode(RangeTblRef);
rangeTableRef->rtindex = 1;
@ -432,23 +432,23 @@ CreateMasterQueryForRouterPlan(DistributedPlan *distPlan)
joinTree->quals = NULL;
joinTree->fromlist = list_make1(rangeTableRef);
Query *masterQuery = makeNode(Query);
masterQuery->commandType = CMD_SELECT;
masterQuery->querySource = QSRC_ORIGINAL;
masterQuery->canSetTag = true;
masterQuery->rtable = list_make1(rangeTableEntry);
masterQuery->targetList = targetList;
masterQuery->jointree = joinTree;
return masterQuery;
Query *combineQuery = makeNode(Query);
combineQuery->commandType = CMD_SELECT;
combineQuery->querySource = QSRC_ORIGINAL;
combineQuery->canSetTag = true;
combineQuery->rtable = list_make1(rangeTableEntry);
combineQuery->targetList = targetList;
combineQuery->jointree = joinTree;
return combineQuery;
}
/*
* CreateTargetListForMasterQuery is used for creating a target list for
* CreateTargetListForCombineQuery is used for creating a target list for
* master query.
*/
static List *
CreateTargetListForMasterQuery(List *targetList)
CreateTargetListForCombineQuery(List *targetList)
{
List *newTargetEntryList = NIL;
const uint32 masterTableId = 1;

View File

@ -51,9 +51,9 @@ FindSubPlanUsages(DistributedPlan *plan)
List *localSubPlans = NIL;
List *remoteSubPlans = NIL;
if (plan->masterQuery != NULL)
if (plan->combineQuery != NULL)
{
localSubPlans = FindSubPlansUsedInNode((Node *) plan->masterQuery,
localSubPlans = FindSubPlansUsedInNode((Node *) plan->combineQuery,
SUBPLAN_ACCESS_LOCAL);
}

View File

@ -34,7 +34,7 @@
#include "distributed/multi_explain.h"
#include "distributed/multi_logical_optimizer.h"
#include "distributed/multi_logical_planner.h"
#include "distributed/merge_planner.h"
#include "distributed/combine_query_planner.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_router_planner.h"
#include "distributed/distributed_planner.h"

View File

@ -249,11 +249,11 @@ CreatePhysicalDistributedPlan(MultiTreeRoot *multiTree,
/* build the final merge query to execute on the master */
List *masterDependentJobList = list_make1(workerJob);
Query *masterQuery = BuildJobQuery((MultiNode *) multiTree, masterDependentJobList);
Query *combineQuery = BuildJobQuery((MultiNode *) multiTree, masterDependentJobList);
DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan);
distributedPlan->workerJob = workerJob;
distributedPlan->masterQuery = masterQuery;
distributedPlan->combineQuery = combineQuery;
distributedPlan->routerExecutable = DistributedPlanRouterExecutable(distributedPlan);
distributedPlan->modLevel = ROW_MODIFY_READONLY;
distributedPlan->expectResults = true;
@ -273,7 +273,7 @@ CreatePhysicalDistributedPlan(MultiTreeRoot *multiTree,
static bool
DistributedPlanRouterExecutable(DistributedPlan *distributedPlan)
{
Query *masterQuery = distributedPlan->masterQuery;
Query *combineQuery = distributedPlan->combineQuery;
Job *job = distributedPlan->workerJob;
List *workerTaskList = job->taskList;
int taskCount = list_length(workerTaskList);
@ -301,7 +301,7 @@ DistributedPlanRouterExecutable(DistributedPlan *distributedPlan)
* sorting on the master query wouldn't be executed. Thus, such plans shouldn't be
* qualified as router executable.
*/
if (masterQuery != NULL && list_length(masterQuery->sortClause) > 0)
if (combineQuery != NULL && list_length(combineQuery->sortClause) > 0)
{
return false;
}
@ -311,8 +311,8 @@ DistributedPlanRouterExecutable(DistributedPlan *distributedPlan)
* have either an aggregate or a function expression which has to be executed for
* the correct results.
*/
bool masterQueryHasAggregates = job->jobQuery->hasAggs;
if (masterQueryHasAggregates)
bool combineQueryHasAggregates = job->jobQuery->hasAggs;
if (combineQueryHasAggregates)
{
return false;
}

View File

@ -238,7 +238,7 @@ CreateModifyPlan(Query *originalQuery, Query *query,
ereport(DEBUG2, (errmsg("Creating router plan")));
distributedPlan->workerJob = job;
distributedPlan->masterQuery = NULL;
distributedPlan->combineQuery = NULL;
distributedPlan->routerExecutable = true;
distributedPlan->expectResults = originalQuery->returningList != NIL;
distributedPlan->targetRelationId = ResultRelationOidForQuery(query);
@ -277,7 +277,7 @@ CreateSingleTaskRouterSelectPlan(DistributedPlan *distributedPlan, Query *origin
ereport(DEBUG2, (errmsg("Creating router plan")));
distributedPlan->workerJob = job;
distributedPlan->masterQuery = NULL;
distributedPlan->combineQuery = NULL;
distributedPlan->routerExecutable = true;
distributedPlan->expectResults = true;
}

View File

@ -15,7 +15,7 @@
#include "distributed/pg_version_constants.h"
#include "distributed/merge_planner.h"
#include "distributed/combine_query_planner.h"
#include "nodes/plannodes.h"
#if PG_VERSION_NUM >= PG_VERSION_12
#include "nodes/nodeFuncs.h"

View File

@ -48,7 +48,7 @@
#include "distributed/multi_join_order.h"
#include "distributed/multi_logical_optimizer.h"
#include "distributed/distributed_planner.h"
#include "distributed/merge_planner.h"
#include "distributed/combine_query_planner.h"
#include "distributed/multi_router_planner.h"
#include "distributed/multi_server_executor.h"
#include "distributed/pg_dist_partition.h"

View File

@ -124,7 +124,7 @@ CopyNodeDistributedPlan(COPYFUNC_ARGS)
COPY_SCALAR_FIELD(routerExecutable);
COPY_NODE_FIELD(workerJob);
COPY_NODE_FIELD(masterQuery);
COPY_NODE_FIELD(combineQuery);
COPY_SCALAR_FIELD(queryId);
COPY_NODE_FIELD(relationIdList);
COPY_SCALAR_FIELD(targetRelationId);

View File

@ -188,7 +188,7 @@ OutDistributedPlan(OUTFUNC_ARGS)
WRITE_BOOL_FIELD(routerExecutable);
WRITE_NODE_FIELD(workerJob);
WRITE_NODE_FIELD(masterQuery);
WRITE_NODE_FIELD(combineQuery);
WRITE_UINT64_FIELD(queryId);
WRITE_NODE_FIELD(relationIdList);
WRITE_OID_FIELD(targetRelationId);

View File

@ -405,7 +405,7 @@ typedef struct DistributedPlan
Job *workerJob;
/* local query that merges results from the workers */
Query *masterQuery;
Query *combineQuery;
/* query identifier (copied from the top-level PlannedStmt) */
uint64 queryId;

View File

@ -114,7 +114,7 @@ DETAIL: Creating dependency on merge taskId 11
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: master query: SELECT l_partkey, o_orderkey, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 3 2 4)'::cstring(0)) remote_scan(l_partkey integer, o_orderkey bigint, count bigint) GROUP BY l_partkey, o_orderkey ORDER BY l_partkey, o_orderkey
DEBUG: combine query: SELECT l_partkey, o_orderkey, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 3 2 4)'::cstring(0)) remote_scan(l_partkey integer, o_orderkey bigint, count bigint) GROUP BY l_partkey, o_orderkey ORDER BY l_partkey, o_orderkey
DEBUG: completed cleanup query for job 3
DEBUG: completed cleanup query for job 3
DEBUG: completed cleanup query for job 2
@ -222,7 +222,7 @@ DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: master query: SELECT l_partkey, o_orderkey, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(l_partkey integer, o_orderkey bigint, count bigint) GROUP BY l_partkey, o_orderkey ORDER BY l_partkey, o_orderkey
DEBUG: combine query: SELECT l_partkey, o_orderkey, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(l_partkey integer, o_orderkey bigint, count bigint) GROUP BY l_partkey, o_orderkey ORDER BY l_partkey, o_orderkey
DEBUG: completed cleanup query for job 6
DEBUG: completed cleanup query for job 6
DEBUG: completed cleanup query for job 4
@ -301,7 +301,7 @@ DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: master query: SELECT o_orderkey, o_shippriority, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, o_shippriority integer, count bigint) GROUP BY o_orderkey ORDER BY o_orderkey
DEBUG: combine query: SELECT o_orderkey, o_shippriority, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, o_shippriority integer, count bigint) GROUP BY o_orderkey ORDER BY o_orderkey
DEBUG: completed cleanup query for job 9
DEBUG: completed cleanup query for job 9
DEBUG: completed cleanup query for job 7
@ -382,7 +382,7 @@ DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: master query: SELECT o_orderkey, o_shippriority, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, o_shippriority integer, count bigint) GROUP BY o_orderkey ORDER BY o_orderkey
DEBUG: combine query: SELECT o_orderkey, o_shippriority, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, o_shippriority integer, count bigint) GROUP BY o_orderkey ORDER BY o_orderkey
DEBUG: completed cleanup query for job 12
DEBUG: completed cleanup query for job 12
DEBUG: completed cleanup query for job 10
@ -461,7 +461,7 @@ DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: master query: SELECT o_orderkey, any_value(any_value) AS any_value FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, any_value integer, worker_column_3 integer) GROUP BY o_orderkey ORDER BY o_orderkey
DEBUG: combine query: SELECT o_orderkey, any_value(any_value) AS any_value FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, any_value integer, worker_column_3 integer) GROUP BY o_orderkey ORDER BY o_orderkey
DEBUG: completed cleanup query for job 15
DEBUG: completed cleanup query for job 15
DEBUG: completed cleanup query for job 13
@ -554,7 +554,7 @@ DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: assigned task to node localhost:xxxxx
DEBUG: master query: SELECT s_i_id FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(s_i_id integer, worker_column_2 integer, worker_column_3 numeric)
DEBUG: combine query: SELECT s_i_id FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(s_i_id integer, worker_column_2 integer, worker_column_3 numeric)
DEBUG: completed cleanup query for job 18
DEBUG: completed cleanup query for job 18
DEBUG: completed cleanup query for job 16