mirror of https://github.com/citusdata/citus.git
Merge pull request #4041 from citusdata/remove_router_executable_flag
Remove `routerExecutable` flag from `DistributedPlan`pull/4045/head
commit
32d9cce8a2
|
@ -46,40 +46,11 @@ MultiExecutorType
|
|||
JobExecutorType(DistributedPlan *distributedPlan)
|
||||
{
|
||||
Job *job = distributedPlan->workerJob;
|
||||
MultiExecutorType executorType = TaskExecutorType;
|
||||
bool routerExecutablePlan = distributedPlan->routerExecutable;
|
||||
|
||||
/* debug distribution column value */
|
||||
if (routerExecutablePlan)
|
||||
{
|
||||
if (IsLoggableLevel(DEBUG2))
|
||||
{
|
||||
Const *partitionValueConst = job->partitionKeyValue;
|
||||
|
||||
if (partitionValueConst != NULL && !partitionValueConst->constisnull)
|
||||
{
|
||||
Datum partitionColumnValue = partitionValueConst->constvalue;
|
||||
Oid partitionColumnType = partitionValueConst->consttype;
|
||||
char *partitionColumnString = DatumToString(partitionColumnValue,
|
||||
partitionColumnType);
|
||||
|
||||
ereport(DEBUG2, (errmsg("Plan is router executable"),
|
||||
errdetail("distribution column value: %s",
|
||||
ApplyLogRedaction(partitionColumnString))));
|
||||
}
|
||||
else
|
||||
{
|
||||
ereport(DEBUG2, (errmsg("Plan is router executable")));
|
||||
}
|
||||
}
|
||||
|
||||
return MULTI_EXECUTOR_ADAPTIVE;
|
||||
}
|
||||
|
||||
if (distributedPlan->insertSelectQuery != NULL)
|
||||
{
|
||||
/*
|
||||
* Even if adaptiveExecutorEnabled, we go through
|
||||
* We go through
|
||||
* MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT because
|
||||
* the executor already knows how to handle adaptive
|
||||
* executor when necessary.
|
||||
|
@ -87,29 +58,40 @@ JobExecutorType(DistributedPlan *distributedPlan)
|
|||
return MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT;
|
||||
}
|
||||
|
||||
Assert(distributedPlan->modLevel == ROW_MODIFY_READONLY);
|
||||
|
||||
|
||||
if (executorType == MULTI_EXECUTOR_ADAPTIVE)
|
||||
/*
|
||||
* If we have repartition jobs with adaptive executor and repartition
|
||||
* joins are not enabled, error out.
|
||||
*/
|
||||
int dependentJobCount = list_length(job->dependentJobList);
|
||||
if (!EnableRepartitionJoins && dependentJobCount > 0)
|
||||
{
|
||||
/* if we have repartition jobs with adaptive executor and repartition
|
||||
* joins are not enabled, error out. Otherwise, switch to task-tracker
|
||||
*/
|
||||
int dependentJobCount = list_length(job->dependentJobList);
|
||||
if (dependentJobCount > 0)
|
||||
ereport(ERROR, (errmsg("the query contains a join that requires repartitioning"),
|
||||
errhint("Set citus.enable_repartition_joins to on to enable "
|
||||
"repartitioning")));
|
||||
}
|
||||
|
||||
/*
|
||||
* Debug distribution column value if possible. The distributed planner sometimes
|
||||
* defers creating the tasks, so the task list might be NIL. Still, it sets the
|
||||
* partitionKeyValue and we print it here.
|
||||
*/
|
||||
if (list_length(job->taskList) <= 1 && IsLoggableLevel(DEBUG2))
|
||||
{
|
||||
Const *partitionValueConst = job->partitionKeyValue;
|
||||
|
||||
if (partitionValueConst != NULL && !partitionValueConst->constisnull)
|
||||
{
|
||||
if (!EnableRepartitionJoins)
|
||||
{
|
||||
ereport(ERROR, (errmsg(
|
||||
"the query contains a join that requires repartitioning"),
|
||||
errhint("Set citus.enable_repartition_joins to on "
|
||||
"to enable repartitioning")));
|
||||
}
|
||||
return MULTI_EXECUTOR_ADAPTIVE;
|
||||
Datum partitionColumnValue = partitionValueConst->constvalue;
|
||||
Oid partitionColumnType = partitionValueConst->consttype;
|
||||
char *partitionColumnString = DatumToString(partitionColumnValue,
|
||||
partitionColumnType);
|
||||
|
||||
ereport(DEBUG2, (errmsg("query has a single distribution column value: "
|
||||
"%s", partitionColumnString)));
|
||||
}
|
||||
}
|
||||
|
||||
return executorType;
|
||||
return MULTI_EXECUTOR_ADAPTIVE;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -377,7 +377,6 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
|
|||
distributedPlan = CitusMakeNode(DistributedPlan);
|
||||
distributedPlan->workerJob = job;
|
||||
distributedPlan->combineQuery = NULL;
|
||||
distributedPlan->routerExecutable = true;
|
||||
distributedPlan->expectResults = true;
|
||||
|
||||
/* worker will take care of any necessary locking, treat query as read-only */
|
||||
|
|
|
@ -354,7 +354,6 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
|
|||
/* and finally the multi plan */
|
||||
distributedPlan->workerJob = workerJob;
|
||||
distributedPlan->combineQuery = NULL;
|
||||
distributedPlan->routerExecutable = true;
|
||||
distributedPlan->expectResults = originalQuery->returningList != NIL;
|
||||
distributedPlan->targetRelationId = targetRelationId;
|
||||
|
||||
|
|
|
@ -156,7 +156,6 @@ static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
|
|||
uint32 shardIntervalCount);
|
||||
|
||||
/* Local functions forward declarations for task list creation and helper functions */
|
||||
static bool DistributedPlanRouterExecutable(DistributedPlan *distributedPlan);
|
||||
static Job * BuildJobTreeTaskList(Job *jobTree,
|
||||
PlannerRestrictionContext *plannerRestrictionContext);
|
||||
static bool IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction);
|
||||
|
@ -250,7 +249,6 @@ CreatePhysicalDistributedPlan(MultiTreeRoot *multiTree,
|
|||
DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan);
|
||||
distributedPlan->workerJob = workerJob;
|
||||
distributedPlan->combineQuery = combineQuery;
|
||||
distributedPlan->routerExecutable = DistributedPlanRouterExecutable(distributedPlan);
|
||||
distributedPlan->modLevel = ROW_MODIFY_READONLY;
|
||||
distributedPlan->expectResults = true;
|
||||
|
||||
|
@ -258,65 +256,6 @@ CreatePhysicalDistributedPlan(MultiTreeRoot *multiTree,
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* DistributedPlanRouterExecutable returns true if the input distributedPlan is
|
||||
* router executable.
|
||||
*
|
||||
* Note that all the multi plans that are created by router planner are
|
||||
* already router executable. Thus, this function should only be called
|
||||
* for multi plans that are not generated by router planner.
|
||||
*/
|
||||
static bool
|
||||
DistributedPlanRouterExecutable(DistributedPlan *distributedPlan)
|
||||
{
|
||||
Query *combineQuery = distributedPlan->combineQuery;
|
||||
Job *job = distributedPlan->workerJob;
|
||||
List *workerTaskList = job->taskList;
|
||||
int taskCount = list_length(workerTaskList);
|
||||
int dependentJobCount = list_length(job->dependentJobList);
|
||||
|
||||
if (!EnableRouterExecution)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/* router executor cannot execute SELECT queries that hit more than one shard */
|
||||
if (taskCount != 1)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/* router executor cannot execute repartition jobs */
|
||||
if (dependentJobCount > 0)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
* Router executor does not run master query. This means that aggregation and
|
||||
* sorting on the master query wouldn't be executed. Thus, such plans shouldn't be
|
||||
* qualified as router executable.
|
||||
*/
|
||||
if (combineQuery != NULL && list_length(combineQuery->sortClause) > 0)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
* Note that worker query having an aggregate means that the master query should
|
||||
* have either an aggregate or a function expression which has to be executed for
|
||||
* the correct results.
|
||||
*/
|
||||
bool combineQueryHasAggregates = job->jobQuery->hasAggs;
|
||||
if (combineQueryHasAggregates)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* BuildJobTree builds the physical job tree from the given logical plan tree.
|
||||
* The function walks over the logical plan from the bottom up, finds boundaries
|
||||
|
|
|
@ -240,7 +240,6 @@ CreateModifyPlan(Query *originalQuery, Query *query,
|
|||
|
||||
distributedPlan->workerJob = job;
|
||||
distributedPlan->combineQuery = NULL;
|
||||
distributedPlan->routerExecutable = true;
|
||||
distributedPlan->expectResults = originalQuery->returningList != NIL;
|
||||
distributedPlan->targetRelationId = ResultRelationOidForQuery(query);
|
||||
|
||||
|
@ -279,7 +278,6 @@ CreateSingleTaskRouterSelectPlan(DistributedPlan *distributedPlan, Query *origin
|
|||
|
||||
distributedPlan->workerJob = job;
|
||||
distributedPlan->combineQuery = NULL;
|
||||
distributedPlan->routerExecutable = true;
|
||||
distributedPlan->expectResults = true;
|
||||
}
|
||||
|
||||
|
|
|
@ -121,7 +121,6 @@ CopyNodeDistributedPlan(COPYFUNC_ARGS)
|
|||
COPY_SCALAR_FIELD(planId);
|
||||
COPY_SCALAR_FIELD(modLevel);
|
||||
COPY_SCALAR_FIELD(expectResults);
|
||||
COPY_SCALAR_FIELD(routerExecutable);
|
||||
|
||||
COPY_NODE_FIELD(workerJob);
|
||||
COPY_NODE_FIELD(combineQuery);
|
||||
|
|
|
@ -185,7 +185,6 @@ OutDistributedPlan(OUTFUNC_ARGS)
|
|||
WRITE_UINT64_FIELD(planId);
|
||||
WRITE_ENUM_FIELD(modLevel, RowModifyLevel);
|
||||
WRITE_BOOL_FIELD(expectResults);
|
||||
WRITE_BOOL_FIELD(routerExecutable);
|
||||
|
||||
WRITE_NODE_FIELD(workerJob);
|
||||
WRITE_NODE_FIELD(combineQuery);
|
||||
|
|
|
@ -384,9 +384,6 @@ typedef struct DistributedPlan
|
|||
*/
|
||||
bool expectResults;
|
||||
|
||||
/* a router executable query is executed entirely on a worker */
|
||||
bool routerExecutable;
|
||||
|
||||
/* job tree containing the tasks to be executed on workers */
|
||||
Job *workerJob;
|
||||
|
||||
|
|
|
@ -359,7 +359,6 @@ PREPARE router_select_with_param(int) AS SELECT DISTINCT value FROM coordinator_
|
|||
EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -368,7 +367,6 @@ DEBUG: Plan is router executable
|
|||
EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -377,7 +375,6 @@ DEBUG: Plan is router executable
|
|||
EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -386,7 +383,6 @@ DEBUG: Plan is router executable
|
|||
EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -395,7 +391,6 @@ DEBUG: Plan is router executable
|
|||
EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -404,7 +399,6 @@ DEBUG: Plan is router executable
|
|||
EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -545,27 +539,21 @@ SET client_min_messages TO DEBUG2;
|
|||
CALL test_procedure(100);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
CALL test_procedure(100);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
CALL test_procedure(100);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
CALL test_procedure(100);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
CALL test_procedure(100);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
CALL test_procedure(100);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
CALL test_procedure(100);
|
||||
CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure_2(int)
|
||||
LANGUAGE plpgsql
|
||||
|
|
|
@ -498,17 +498,14 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinat
|
|||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
\set VERBOSITY terse
|
||||
DROP TABLE ref_table;
|
||||
NOTICE: executing the command locally: DROP TABLE IF EXISTS coordinator_shouldhaveshards.ref_table_xxxxx CASCADE
|
||||
CONTEXT: SQL statement "SELECT master_drop_all_shards(v_obj.objid, v_obj.schema_name, v_obj.object_name)"
|
||||
PL/pgSQL function citus_drop_trigger() line 19 at PERFORM
|
||||
DELETE FROM test;
|
||||
DROP TABLE test;
|
||||
DROP TABLE dist_table;
|
||||
DROP TABLE ref;
|
||||
NOTICE: executing the command locally: DROP TABLE IF EXISTS coordinator_shouldhaveshards.ref_xxxxx CASCADE
|
||||
CONTEXT: SQL statement "SELECT master_drop_all_shards(v_obj.objid, v_obj.schema_name, v_obj.object_name)"
|
||||
PL/pgSQL function citus_drop_trigger() line 19 at PERFORM
|
||||
DROP SCHEMA coordinator_shouldhaveshards CASCADE;
|
||||
NOTICE: drop cascades to table local
|
||||
SELECT 1 FROM master_set_node_property('localhost', :master_port, 'shouldhaveshards', false);
|
||||
|
|
|
@ -34,7 +34,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test9 | {"f1": 9, "f2": 162, "f3": "test9"} | 1
|
||||
|
@ -56,7 +55,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY value DESC LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | {"f1": 99, "f2": 1782, "f3": "test99"} | 1
|
||||
|
@ -80,8 +78,7 @@ WHERE
|
|||
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT (SELECT 1) FROM cte_inline.test_table WHERE (key OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table) cte_1 WHERE (key OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result."?column?" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -133,7 +130,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT cte_1.key, cte_1.value, cte_1.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1) foo ORDER BY value DESC LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | {"f1": 99, "f2": 1782, "f3": "test99"} | 1
|
||||
|
@ -163,11 +159,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for CTE cte_1: SELECT key, value, other_value FROM cte_inline.test_table
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT key, value, other_value, (SELECT 1) FROM (SELECT cte_1.key, cte_1.value, cte_1.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1) foo
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) top_cte, (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result."?column?" FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, "?column?" integer)) bar(key, value, other_value, "?column?")
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10201
|
||||
|
@ -197,7 +191,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value FROM cte_inline.test_table
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT (SELECT 1) FROM (SELECT foo.key, foo.value, foo.other_value, random() AS random FROM (SELECT cte_1.key, cte_1.value, cte_1.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1) foo) bar
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM cte_inline.test_table WHERE (key OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result."?column?" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -224,7 +217,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_2: SELECT key, value, other_value,
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT (SELECT 1) AS key FROM (SELECT cte_2.key, cte_2.value, cte_2.other_value, cte_2.random, cte_2.random_1 AS random, random() AS random FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result.random, intermediate_result.random_1 AS random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, random double precision, random_1 double precision)) cte_2(key, value, other_value, random, random_1)) bar(key, value, other_value, random, random_1, random_2) ORDER BY (SELECT 1) DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -268,8 +260,7 @@ WHERE
|
|||
ORDER BY 3 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value | other_value | row_number
|
||||
---------------------------------------------------------------------
|
||||
1 | test91 | {"f1": 91, "f2": 1638, "f3": "test91"} | 10
|
||||
|
@ -292,8 +283,7 @@ ORDER BY 1 DESC
|
|||
LIMIT 5;
|
||||
DEBUG: CTE a is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
1 | test1 | {"f1": 1, "f2": 18, "f3": "test1"} | 1
|
||||
|
@ -314,8 +304,7 @@ WHERE
|
|||
key = 1;
|
||||
DEBUG: CTE a is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10
|
||||
|
@ -331,8 +320,7 @@ WHERE
|
|||
key = 1;
|
||||
DEBUG: CTE a is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10
|
||||
|
@ -351,7 +339,6 @@ DEBUG: generating subplan XXX_1 for CTE a: SELECT key, value, other_value FROM
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) a WHERE (key OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10
|
||||
|
@ -367,8 +354,7 @@ WHERE
|
|||
key = 1;
|
||||
DEBUG: CTE a is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -393,7 +379,6 @@ DEBUG: generating subplan XXX_1 for CTE a: SELECT key, value, other_value FROM
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) a WHERE (key OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -427,7 +412,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) first_entry JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) second_entry USING (key))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1021
|
||||
|
@ -475,7 +459,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) first_entry JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) second_entry USING (key))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -552,7 +535,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value,
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, random double precision)) cte_1 ORDER BY value DESC LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value
|
||||
---------------------------------------------------------------------
|
||||
9 | test99
|
||||
|
@ -569,7 +551,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value,
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, random double precision)) cte_1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
101
|
||||
|
@ -676,7 +657,6 @@ DEBUG: push down of limit count: 5
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT key, value, other_value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table) cte_1 ORDER BY key, value, other_value LIMIT 5
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value FROM (SELECT ct2.key, ct2.value, ct2.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) ct2) foo ORDER BY key DESC, value DESC, other_value DESC LIMIT 5
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value
|
||||
---------------------------------------------------------------------
|
||||
0 | test30 | {"f1": 30, "f2": 540, "f3": "test30"}
|
||||
|
@ -709,8 +689,7 @@ DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
|||
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
||||
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count | key
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -733,11 +712,9 @@ DEBUG: generating subplan XXX_2 for subquery SELECT key, value, other_value FRO
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT key, value, other_value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table) cte_2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_4 for subquery (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb) EXCEPT SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) UNION SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) foo
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
101
|
||||
|
@ -763,13 +740,11 @@ DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT key, value, other_value, 1 FROM cte_inline.test_table
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result."?column?" FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, "?column?" integer) EXCEPT SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result."?column?" FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, "?column?" integer)) UNION SELECT cte_2.key, cte_2.value, cte_2.other_value, 1 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test29 | {"f1": 29, "f2": 522, "f3": "test29"} | 1
|
||||
|
@ -899,7 +874,6 @@ DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960000 AS citu
|
|||
DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960001 AS citus_table_alias (key, value) SELECT key, value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table_1960001 test_table) fist_table_cte WHERE ((worker_hash(key) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(key) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960002 AS citus_table_alias (key, value) SELECT key, value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table_1960002 test_table) fist_table_cte WHERE ((worker_hash(key) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(key) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960003 AS citus_table_alias (key, value) SELECT key, value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table_1960003 test_table) fist_table_cte WHERE ((worker_hash(key) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(key) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- update/delete/modifying ctes
|
||||
-- we don't support any cte inlining in modifications
|
||||
-- queries and modifying CTEs
|
||||
|
@ -910,7 +884,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM cte_inline.test_table WHERE (NOT (key OPERATOR(pg_catalog.=) ANY (SELECT cte_1.key FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- NOT MATERIALIZED should not CTEs that are used in a modifying query, because
|
||||
-- we de still don't support it
|
||||
WITH cte_1 AS NOT MATERIALIZED (SELECT * FROM test_table)
|
||||
|
@ -920,17 +893,14 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM cte_inline.test_table WHERE (NOT (key OPERATOR(pg_catalog.=) ANY (SELECT cte_1.key FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- we don't inline CTEs if they are modifying CTEs
|
||||
WITH cte_1 AS (DELETE FROM test_table WHERE key % 3 = 1 RETURNING key)
|
||||
SELECT * FROM cte_1 ORDER BY 1 DESC LIMIT 3;
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for CTE cte_1: DELETE FROM cte_inline.test_table WHERE ((key OPERATOR(pg_catalog.%) 3) OPERATOR(pg_catalog.=) 1) RETURNING key
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_1 ORDER BY key DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key
|
||||
---------------------------------------------------------------------
|
||||
7
|
||||
|
@ -944,10 +914,8 @@ SELECT count(*) FROM cte_1;
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for CTE cte_1: DELETE FROM cte_inline.test_table WHERE ((key OPERATOR(pg_catalog.%) 3) OPERATOR(pg_catalog.=) 0) RETURNING key
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
164
|
||||
|
@ -1110,7 +1078,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1279,7 +1246,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1295,7 +1261,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1311,7 +1276,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1327,7 +1291,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1343,7 +1306,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1361,7 +1323,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
8 | test98 | | 1
|
||||
|
@ -1436,7 +1397,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT key AS x FROM cte_inline.test_table OFFSET 0
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) ref LEFT JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) b ON ((ref.x OPERATOR(pg_catalog.=) b.key)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
4800
|
||||
|
@ -1479,7 +1439,6 @@ DEBUG: generating subplan XXX_2 for CTE b: SELECT key, value, other_value FROM
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT min(a.key) AS min FROM ((SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) a LEFT JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) b ON ((a.value OPERATOR(pg_catalog.=) b.value)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
min
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -1523,7 +1482,6 @@ DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT value FROM cte_inline.tes
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT value FROM ((SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) cte_1 FULL JOIN (SELECT intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(value text)) cte_2 USING (value)) ORDER BY value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
test98
|
||||
|
|
|
@ -34,7 +34,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test9 | {"f1": 9, "f2": 162, "f3": "test9"} | 1
|
||||
|
@ -68,8 +67,7 @@ WHERE
|
|||
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT (SELECT 1) FROM cte_inline.test_table WHERE (key OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table) cte_1 WHERE (key OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result."?column?" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -121,7 +119,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT cte_1.key, cte_1.value, cte_1.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1) foo ORDER BY value DESC LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | {"f1": 99, "f2": 1782, "f3": "test99"} | 1
|
||||
|
@ -151,11 +148,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for CTE cte_1: SELECT key, value, other_value FROM cte_inline.test_table
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT key, value, other_value, (SELECT 1) FROM (SELECT cte_1.key, cte_1.value, cte_1.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1) foo
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) top_cte, (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result."?column?" FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, "?column?" integer)) bar(key, value, other_value, "?column?")
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10201
|
||||
|
@ -185,7 +180,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value FROM cte_inline.test_table
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT (SELECT 1) FROM (SELECT foo.key, foo.value, foo.other_value, random() AS random FROM (SELECT cte_1.key, cte_1.value, cte_1.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1) foo) bar
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM cte_inline.test_table WHERE (key OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result."?column?" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -212,7 +206,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_2: SELECT key, value, other_value,
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT (SELECT 1) AS key FROM (SELECT cte_2.key, cte_2.value, cte_2.other_value, cte_2.random, cte_2.random_1 AS random, random() AS random FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result.random, intermediate_result.random_1 AS random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, random double precision, random_1 double precision)) cte_2(key, value, other_value, random, random_1)) bar(key, value, other_value, random, random_1, random_2) ORDER BY (SELECT 1) DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -256,8 +249,7 @@ WHERE
|
|||
ORDER BY 3 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value | other_value | row_number
|
||||
---------------------------------------------------------------------
|
||||
1 | test91 | {"f1": 91, "f2": 1638, "f3": "test91"} | 10
|
||||
|
@ -280,8 +272,7 @@ ORDER BY 1 DESC
|
|||
LIMIT 5;
|
||||
DEBUG: CTE a is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
1 | test1 | {"f1": 1, "f2": 18, "f3": "test1"} | 1
|
||||
|
@ -370,7 +361,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) first_entry JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) second_entry USING (key))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1021
|
||||
|
@ -400,7 +390,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) first_entry JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) second_entry USING (key))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -448,7 +437,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value,
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, random double precision)) cte_1 ORDER BY value DESC LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value
|
||||
---------------------------------------------------------------------
|
||||
9 | test99
|
||||
|
@ -562,7 +550,6 @@ DEBUG: push down of limit count: 5
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT key, value, other_value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table) cte_1 ORDER BY key, value, other_value LIMIT 5
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value FROM (SELECT ct2.key, ct2.value, ct2.other_value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) ct2) foo ORDER BY key DESC, value DESC, other_value DESC LIMIT 5
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value
|
||||
---------------------------------------------------------------------
|
||||
0 | test30 | {"f1": 30, "f2": 540, "f3": "test30"}
|
||||
|
@ -617,11 +604,9 @@ DEBUG: generating subplan XXX_2 for subquery SELECT key, value, other_value FRO
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT key, value, other_value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table) cte_2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_4 for subquery (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb) EXCEPT SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) UNION SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) foo
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
101
|
||||
|
@ -647,13 +632,11 @@ DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT key, value, other_value, 1 FROM cte_inline.test_table
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result."?column?" FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, "?column?" integer) EXCEPT SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value, intermediate_result."?column?" FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb, "?column?" integer)) UNION SELECT cte_2.key, cte_2.value, cte_2.other_value, 1 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test29 | {"f1": 29, "f2": 522, "f3": "test29"} | 1
|
||||
|
@ -781,7 +764,6 @@ DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960000 AS citu
|
|||
DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960001 AS citus_table_alias (key, value) SELECT key, value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table_1960001 test_table) fist_table_cte WHERE ((worker_hash(key) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(key) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960002 AS citus_table_alias (key, value) SELECT key, value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table_1960002 test_table) fist_table_cte WHERE ((worker_hash(key) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(key) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO cte_inline.test_table_1960003 AS citus_table_alias (key, value) SELECT key, value FROM (SELECT test_table.key, test_table.value, test_table.other_value FROM cte_inline.test_table_1960003 test_table) fist_table_cte WHERE ((worker_hash(key) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(key) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- update/delete/modifying ctes
|
||||
-- we don't support any cte inlining in modifications
|
||||
-- queries and modifying CTEs
|
||||
|
@ -792,7 +774,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM cte_inline.test_table WHERE (NOT (key OPERATOR(pg_catalog.=) ANY (SELECT cte_1.key FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- NOT MATERIALIZED should not CTEs that are used in a modifying query, because
|
||||
-- we de still don't support it
|
||||
WITH cte_1 AS NOT MATERIALIZED (SELECT * FROM test_table)
|
||||
|
@ -804,10 +785,8 @@ SELECT * FROM cte_1 ORDER BY 1 DESC LIMIT 3;
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for CTE cte_1: DELETE FROM cte_inline.test_table WHERE ((key OPERATOR(pg_catalog.%) 3) OPERATOR(pg_catalog.=) 1) RETURNING key
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_1 ORDER BY key DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key
|
||||
---------------------------------------------------------------------
|
||||
7
|
||||
|
@ -978,7 +957,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1155,7 +1133,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1171,7 +1148,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1187,7 +1163,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1203,7 +1178,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1219,7 +1193,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1237,7 +1210,6 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT key, value, other_value F
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value, other_value, (SELECT 1) FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) cte_1 ORDER BY key DESC, value DESC, other_value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value | other_value | ?column?
|
||||
---------------------------------------------------------------------
|
||||
9 | test99 | | 1
|
||||
|
@ -1312,7 +1284,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT key AS x FROM cte_inline.test_table OFFSET 0
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) ref LEFT JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) b ON ((ref.x OPERATOR(pg_catalog.=) b.key)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
11536
|
||||
|
@ -1355,7 +1326,6 @@ DEBUG: generating subplan XXX_2 for CTE b: SELECT key, value, other_value FROM
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT min(a.key) AS min FROM ((SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) a LEFT JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) b ON ((a.value OPERATOR(pg_catalog.=) b.value)))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
min
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -1399,7 +1369,6 @@ DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT value FROM cte_inline.tes
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT value FROM ((SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) cte_1 FULL JOIN (SELECT intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(value text)) cte_2 USING (value)) ORDER BY value DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
value
|
||||
---------------------------------------------------------------------
|
||||
test99
|
||||
|
|
|
@ -78,7 +78,6 @@ INSERT INTO test_range VALUES (U&'\00E4sop', 1), (U&'Vo\1E9Er', 2);
|
|||
SET client_min_messages TO debug;
|
||||
SELECT * FROM test_range WHERE key > 'Ab' AND key < U&'\00E4z';
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | val
|
||||
---------------------------------------------------------------------
|
||||
äsop | 1
|
||||
|
|
|
@ -35,69 +35,55 @@ SET client_min_messages TO DEBUG;
|
|||
DELETE FROM modify_fast_path WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
UPDATE modify_fast_path SET value_1 = 1 WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
UPDATE modify_fast_path SET value_1 = value_1 + value_2::int WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DELETE FROM modify_fast_path WHERE value_1 = 15 AND (key = 1 AND value_2 = 'citus');
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DELETE FROM modify_fast_path WHERE key = 1 and FALSE;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- UPDATE may include complex target entries
|
||||
UPDATE modify_fast_path SET value_1 = value_1 + 12 * value_1 WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
UPDATE modify_fast_path SET value_1 = abs(-19) WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- cannot go through fast-path because there are multiple keys
|
||||
DELETE FROM modify_fast_path WHERE key = 1 AND key = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DELETE FROM modify_fast_path WHERE key = 1 AND (key = 2 AND value_1 = 15);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- cannot go through fast-path because key is not on the top level
|
||||
DELETE FROM modify_fast_path WHERE value_1 = 15 OR (key = 1 AND value_2 = 'citus');
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DELETE FROM modify_fast_path WHERE value_1 = 15 AND (key = 1 OR value_2 = 'citus');
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- goes through fast-path planning even if the key is updated to the same value
|
||||
UPDATE modify_fast_path SET key = 1 WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
UPDATE modify_fast_path SET key = 1::float WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- cannot support if key changes
|
||||
UPDATE modify_fast_path SET key = 2 WHERE key = 1;
|
||||
DEBUG: modifying the partition value of rows is not allowed
|
||||
|
@ -108,13 +94,11 @@ ERROR: modifying the partition value of rows is not allowed
|
|||
-- returning is supported via fast-path
|
||||
INSERT INTO modify_fast_path (key, value_1) VALUES (1,1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DELETE FROM modify_fast_path WHERE key = 1 RETURNING *;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value_1 | value_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 |
|
||||
|
@ -122,8 +106,7 @@ DETAIL: distribution column value: 1
|
|||
|
||||
INSERT INTO modify_fast_path (key, value_1) VALUES (2,1) RETURNING value_1, key;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
value_1 | key
|
||||
---------------------------------------------------------------------
|
||||
1 | 2
|
||||
|
@ -132,8 +115,7 @@ DETAIL: distribution column value: 2
|
|||
DELETE FROM modify_fast_path WHERE key = 2 RETURNING value_1 * 15, value_1::numeric * 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
?column? | ?column?
|
||||
---------------------------------------------------------------------
|
||||
15 | 16
|
||||
|
@ -150,8 +132,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM fast_path_router_modify.modify_fast_path WHERE (key OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value_1, value_2 FROM (SELECT modify_fast_path.key, modify_fast_path.value_1, modify_fast_path.value_2 FROM fast_path_router_modify.modify_fast_path) t2
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
key | value_1 | value_2
|
||||
|
@ -162,8 +143,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
SELECT * FROM modify_fast_path WHERE key = 1 FOR UPDATE;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value_1 | value_2
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -171,8 +151,7 @@ DETAIL: distribution column value: 1
|
|||
SELECT * FROM modify_fast_path WHERE key = 1 FOR SHARE;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
key | value_1 | value_2
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -180,7 +159,6 @@ DETAIL: distribution column value: 1
|
|||
SELECT * FROM modify_fast_path_reference WHERE key = 1 FOR UPDATE;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value_1 | value_2
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -188,7 +166,6 @@ DEBUG: Plan is router executable
|
|||
SELECT * FROM modify_fast_path_reference WHERE key = 1 FOR SHARE;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
key | value_1 | value_2
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -206,19 +183,15 @@ HINT: Consider using an equality filter on the distributed table's partition co
|
|||
DELETE FROM modify_fast_path_reference WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
UPDATE modify_fast_path_reference SET value_1 = 1 WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
UPDATE modify_fast_path_reference SET value_1 = value_1 + 1 WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
UPDATE modify_fast_path_reference SET value_1 = value_1 + value_2::int WHERE key = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- joins are not supported via fast-path
|
||||
UPDATE modify_fast_path
|
||||
SET value_1 = 1
|
||||
|
@ -228,34 +201,27 @@ UPDATE modify_fast_path
|
|||
modify_fast_path.key = 1 AND
|
||||
modify_fast_path_reference.key = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
PREPARE p1 (int, int, int) AS
|
||||
UPDATE modify_fast_path SET value_1 = value_1 + $1 WHERE key = $2 AND value_1 = $3;
|
||||
EXECUTE p1(1,1,1);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE p1(2,2,2);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE p1(3,3,3);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE p1(4,4,4);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE p1(5,5,5);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE p1(6,6,6);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE p1(7,7,7);
|
||||
CREATE FUNCTION modify_fast_path_plpsql(int, int) RETURNS void as $$
|
||||
BEGIN
|
||||
|
@ -268,9 +234,6 @@ CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1
|
|||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Creating router plan
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
modify_fast_path_plpsql
|
||||
---------------------------------------------------------------------
|
||||
|
@ -283,9 +246,6 @@ CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1
|
|||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Creating router plan
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
modify_fast_path_plpsql
|
||||
---------------------------------------------------------------------
|
||||
|
@ -298,9 +258,6 @@ CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1
|
|||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Creating router plan
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
modify_fast_path_plpsql
|
||||
---------------------------------------------------------------------
|
||||
|
@ -313,9 +270,6 @@ CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1
|
|||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Creating router plan
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
modify_fast_path_plpsql
|
||||
---------------------------------------------------------------------
|
||||
|
@ -328,9 +282,6 @@ CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1
|
|||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Creating router plan
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
modify_fast_path_plpsql
|
||||
---------------------------------------------------------------------
|
||||
|
@ -343,9 +294,6 @@ CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1
|
|||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Creating router plan
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"
|
||||
PL/pgSQL function modify_fast_path_plpsql(integer,integer) line 3 at SQL statement
|
||||
modify_fast_path_plpsql
|
||||
---------------------------------------------------------------------
|
||||
|
@ -366,7 +314,6 @@ SET citus.log_remote_commands TO ON;
|
|||
EXECUTE prepared_zero_shard_select(1);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) 1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -376,7 +323,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
EXECUTE prepared_zero_shard_select(2);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) 2) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -386,7 +332,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
EXECUTE prepared_zero_shard_select(3);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) 3) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -396,7 +341,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
EXECUTE prepared_zero_shard_select(4);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) 4) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -406,7 +350,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
EXECUTE prepared_zero_shard_select(5);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) 5) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -416,7 +359,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
EXECUTE prepared_zero_shard_select(6);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) 6) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -433,27 +375,21 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
EXECUTE prepared_zero_shard_update(1);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(2);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(3);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(4);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(5);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(6);
|
||||
DEBUG: Deferred pruning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(7);
|
||||
-- same test with fast-path disabled
|
||||
SET citus.enable_fast_path_router_planner TO FALSE;
|
||||
|
@ -479,7 +415,6 @@ PREPARE prepared_zero_shard_select(int) AS SELECT count(*) FROM modify_fast_path
|
|||
PREPARE prepared_zero_shard_update(int) AS UPDATE modify_fast_path SET value_1 = 1 WHERE key = $1 AND false;
|
||||
EXECUTE prepared_zero_shard_select(1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) $1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -488,7 +423,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
|
||||
EXECUTE prepared_zero_shard_select(2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) $1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -497,7 +431,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
|
||||
EXECUTE prepared_zero_shard_select(3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) $1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -506,7 +439,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
|
||||
EXECUTE prepared_zero_shard_select(4);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) $1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -515,7 +447,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
|
||||
EXECUTE prepared_zero_shard_select(5);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) $1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -524,7 +455,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
|
||||
EXECUTE prepared_zero_shard_select(6);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::integer AS value_1, NULL::text AS value_2 WHERE false) modify_fast_path(key, value_1, value_2) WHERE ((key OPERATOR(pg_catalog.=) $1) AND false)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -540,22 +470,16 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NU
|
|||
|
||||
EXECUTE prepared_zero_shard_update(1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(4);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(5);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(6);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
EXECUTE prepared_zero_shard_update(7);
|
||||
-- same test with fast-path disabled
|
||||
-- reset back to the original value, in case any new test comes after
|
||||
|
|
|
@ -1199,13 +1199,13 @@ SET client_min_messages TO DEBUG;
|
|||
SET citus.enable_unique_job_ids TO off;
|
||||
INSERT INTO source_table VALUES (1,2, '2020-02-02', 3, 4, 5);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO source_table VALUES (1,2, '2020-02-02', 3, 4, 5);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO source_table VALUES (3,4, '2020-02-02', 3, 4, 5);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
INSERT INTO target_table AS enriched(c1, c2, c3, c4, c5, c6, cardinality, sum)
|
||||
SELECT c1, c2, c3, c4, -1::float AS c5,
|
||||
dist_func(c1, 4) c6,
|
||||
|
|
|
@ -101,8 +101,7 @@ FROM interesting_squares JOIN (SELECT * FROM read_intermediate_result('squares',
|
|||
WHERE user_id = 'jon'
|
||||
ORDER BY x;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: jon
|
||||
DEBUG: query has a single distribution column value: jon
|
||||
ERROR: Query could not find the intermediate result file "squares", it was mostly likely deleted due to an error in a parallel process within the same distributed transaction
|
||||
RESET client_min_messages;
|
||||
-- try to read the file as text, will fail because of binary encoding
|
||||
|
|
|
@ -304,8 +304,7 @@ step s1-select-from-t1-with-subquery:
|
|||
SELECT * FROM (SELECT * FROM test_table_1_rf1 FOR UPDATE) foo WHERE id = 1;
|
||||
RESET client_min_messages;
|
||||
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id val_1
|
||||
|
||||
1 2
|
||||
|
|
|
@ -2056,7 +2056,7 @@ NOTICE: executing the command locally: UPDATE public.event_responses_1480001 ev
|
|||
SELECT count(*) FROM event_responses WHERE event_id = 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM public.event_responses_1480001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2066,7 +2066,7 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM public.eve
|
|||
SELECT count(*) FROM event_responses WHERE event_id = 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM public.event_responses_1480001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16)
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2076,13 +2076,13 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM public.eve
|
|||
UPDATE event_responses SET response = 'no' WHERE event_id = 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: UPDATE public.event_responses_1480001 event_responses SET response = 'no'::public.invite_resp WHERE (event_id OPERATOR(pg_catalog.=) 16)
|
||||
INSERT INTO event_responses VALUES (16, 666, 'maybe')
|
||||
ON CONFLICT (event_id, user_id)
|
||||
DO UPDATE SET response = EXCLUDED.response RETURNING *;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: INSERT INTO public.event_responses_1480001 AS citus_table_alias (event_id, user_id, response) VALUES (16, 666, 'maybe'::public.invite_resp) ON CONFLICT(event_id, user_id) DO UPDATE SET response = excluded.response RETURNING citus_table_alias.event_id, citus_table_alias.user_id, citus_table_alias.response
|
||||
event_id | user_id | response
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2094,7 +2094,6 @@ INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no')
|
|||
ON CONFLICT (event_id, user_id)
|
||||
DO UPDATE SET response = EXCLUDED.response RETURNING *;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: INSERT INTO public.event_responses_1480001 AS citus_table_alias (event_id, user_id, response) VALUES (16,666,'maybe'::public.invite_resp), (17,777,'no'::public.invite_resp) ON CONFLICT(event_id, user_id) DO UPDATE SET response = excluded.response RETURNING citus_table_alias.event_id, citus_table_alias.user_id, citus_table_alias.response
|
||||
event_id | user_id | response
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2109,7 +2108,7 @@ CALL register_for_event(19, 1, 'yes');
|
|||
-- should be fine even if no parameters exists in the query
|
||||
SELECT count(*) FROM event_responses WHERE event_id = 16;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -2117,7 +2116,7 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT count(*) FROM event_responses WHERE event_id = 16;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -2125,12 +2124,12 @@ DEBUG: Plan is router executable
|
|||
|
||||
UPDATE event_responses SET response = 'no' WHERE event_id = 16;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
INSERT INTO event_responses VALUES (16, 666, 'maybe')
|
||||
ON CONFLICT (event_id, user_id)
|
||||
DO UPDATE SET response = EXCLUDED.response RETURNING *;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
event_id | user_id | response
|
||||
---------------------------------------------------------------------
|
||||
16 | 666 | maybe
|
||||
|
@ -2141,7 +2140,6 @@ INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no')
|
|||
ON CONFLICT (event_id, user_id)
|
||||
DO UPDATE SET response = EXCLUDED.response RETURNING *;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
event_id | user_id | response
|
||||
---------------------------------------------------------------------
|
||||
16 | 666 | maybe
|
||||
|
|
|
@ -70,8 +70,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
|
||||
SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -79,8 +78,7 @@ DETAIL: distribution column value: 1
|
|||
|
||||
SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -88,8 +86,7 @@ DETAIL: distribution column value: 2
|
|||
|
||||
SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -97,8 +94,7 @@ DETAIL: distribution column value: 3
|
|||
|
||||
SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 4;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 4
|
||||
DEBUG: query has a single distribution column value: 4
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -107,8 +103,7 @@ DETAIL: distribution column value: 4
|
|||
SELECT count(*) FROM orders_hash_partitioned
|
||||
WHERE o_orderkey = 1 AND o_clerk = 'aaa';
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -116,8 +111,7 @@ DETAIL: distribution column value: 1
|
|||
|
||||
SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = abs(-1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -232,8 +226,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
SELECT count(*) FROM
|
||||
(SELECT o_orderkey FROM orders_hash_partitioned WHERE o_orderkey = 1) AS orderkeys;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -283,7 +276,6 @@ DEBUG: assigned task to node localhost:xxxxx
|
|||
SELECT count(*) FROM lineitem_hash_part
|
||||
WHERE l_orderkey IN (NULL);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -447,7 +439,6 @@ DEBUG: constraint value: '2'::bigint
|
|||
DEBUG: constraint value: '3'::bigint
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
13
|
||||
|
@ -460,7 +451,6 @@ DEBUG: constraint value: '2'::bigint
|
|||
DEBUG: constraint value: '3'::bigint
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
13
|
||||
|
@ -553,8 +543,7 @@ SELECT count(*) FROM orders_hash_partitioned
|
|||
DEBUG: constraint value: 1
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -604,8 +593,7 @@ DEBUG: shard count: 1
|
|||
DEBUG: constraint value: 1
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -633,8 +621,7 @@ SELECT count(*) FROM orders_hash_partitioned
|
|||
DEBUG: constraint value: 1
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -646,8 +633,7 @@ SELECT count(*) FROM orders_hash_partitioned
|
|||
DEBUG: constraint value: 1
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -912,7 +898,6 @@ SELECT count(*) FROM orders_hash_partitioned
|
|||
DEBUG: constraint value: 1
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -924,8 +909,7 @@ SELECT count(*) FROM orders_hash_partitioned
|
|||
DEBUG: no valid constraints found
|
||||
DEBUG: shard count: 0
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -1023,7 +1007,6 @@ DEBUG: assigned task to node localhost:xxxxx
|
|||
SELECT count(*) FROM orders_hash_partitioned
|
||||
WHERE NOT (o_orderkey != 2 OR o_orderkey != 3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -1092,8 +1075,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: constraint value: 1
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT o_custkey FROM public.orders_hash_partitioned WHERE (o_orderkey OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.orders_hash_partitioned WHERE ((o_orderkey OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2])) AND (NOT (o_custkey OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.o_custkey FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(o_custkey integer)))))
|
||||
DEBUG: constraint value: 1
|
||||
|
@ -1121,8 +1103,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: constraint value: 3
|
||||
DEBUG: shard count: 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT o_custkey FROM public.orders_hash_partitioned WHERE (o_orderkey OPERATOR(pg_catalog.=) 3)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.orders_hash_partitioned WHERE ((o_orderkey OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2])) OR (NOT (o_custkey OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.o_custkey FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(o_custkey integer)))))
|
||||
DEBUG: no valid constraints found
|
||||
|
|
|
@ -67,7 +67,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- see that our first multi shard INSERT...SELECT works expected
|
||||
SET client_min_messages TO INFO;
|
||||
SELECT
|
||||
|
@ -154,7 +153,6 @@ DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, "time") SELECT user_id, "time" FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) 7) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
-- add one more row
|
||||
INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES
|
||||
|
@ -172,7 +170,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
-- a zero shard select
|
||||
INSERT INTO raw_events_second (value_2, value_1, value_3, value_4, user_id, time)
|
||||
SELECT
|
||||
|
@ -185,7 +182,6 @@ DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
-- another zero shard select
|
||||
INSERT INTO raw_events_second (value_2, value_1, value_3, value_4, user_id, time)
|
||||
SELECT
|
||||
|
@ -198,7 +194,6 @@ DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
-- add one more row
|
||||
SET client_min_messages TO INFO;
|
||||
INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES
|
||||
|
@ -217,7 +212,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300001 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300002 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300003 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4
|
||||
DEBUG: Plan is router executable
|
||||
user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
---------------------------------------------------------------------
|
||||
9 | | 90 | | 9000 |
|
||||
|
@ -237,7 +231,6 @@ DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300001 raw_events_first WHERE (((user_id OPERATOR(pg_catalog.=) 9) OR (user_id OPERATOR(pg_catalog.=) 16)) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300003 raw_events_first WHERE (((user_id OPERATOR(pg_catalog.=) 9) OR (user_id OPERATOR(pg_catalog.=) 16)) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4
|
||||
DEBUG: Plan is router executable
|
||||
ERROR: duplicate key value violates unique constraint "raw_events_second_user_id_value_1_key_xxxxxxx"
|
||||
-- now do some aggregations
|
||||
INSERT INTO agg_events
|
||||
|
@ -251,7 +244,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY user_id
|
||||
DEBUG: Plan is router executable
|
||||
-- group by column not exists on the SELECT target list
|
||||
INSERT INTO agg_events (value_3_agg, value_4_agg, value_1_agg, user_id)
|
||||
SELECT
|
||||
|
@ -265,7 +257,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time
|
||||
DEBUG: Plan is router executable
|
||||
ERROR: duplicate key value violates unique constraint "agg_events_user_id_value_1_agg_key_xxxxxxx"
|
||||
-- some subquery tests
|
||||
INSERT INTO agg_events
|
||||
|
@ -284,7 +275,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY id ORDER BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY id ORDER BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY id ORDER BY id
|
||||
DEBUG: Plan is router executable
|
||||
ERROR: duplicate key value violates unique constraint "agg_events_user_id_value_1_agg_key_xxxxxxx"
|
||||
-- subquery one more level depth
|
||||
INSERT INTO agg_events
|
||||
|
@ -306,7 +296,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) ORDER BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) ORDER BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) ORDER BY id
|
||||
DEBUG: Plan is router executable
|
||||
ERROR: duplicate key value violates unique constraint "agg_events_user_id_value_1_agg_key_xxxxxxx"
|
||||
\set VERBOSITY DEFAULT
|
||||
-- join between subqueries
|
||||
|
@ -336,7 +325,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- add one more level subqueris on top of subquery JOINs
|
||||
INSERT INTO agg_events
|
||||
(user_id, value_4_agg)
|
||||
|
@ -370,7 +358,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY id
|
||||
DEBUG: Plan is router executable
|
||||
-- subqueries in WHERE clause
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
|
@ -383,7 +370,6 @@ DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) 2))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
SELECT user_id
|
||||
|
@ -396,7 +382,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) ON CONFLICT(user_id, value_1) DO NOTHING
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) ON CONFLICT(user_id, value_1) DO NOTHING
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) ON CONFLICT(user_id, value_1) DO NOTHING
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
SELECT user_id
|
||||
|
@ -407,7 +392,6 @@ DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
SELECT user_id
|
||||
|
@ -419,7 +403,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
-- lets mix subqueries in FROM clause and subqueries in WHERE
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
|
@ -449,7 +432,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
-- some UPSERTS
|
||||
INSERT INTO agg_events AS ae
|
||||
(
|
||||
|
@ -469,7 +451,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS ae (use
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time)
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time)
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time)
|
||||
DEBUG: Plan is router executable
|
||||
-- upserts with returning
|
||||
INSERT INTO agg_events AS ae
|
||||
(
|
||||
|
@ -490,7 +471,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS ae (use
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg
|
||||
DEBUG: Plan is router executable
|
||||
user_id | value_1_agg
|
||||
---------------------------------------------------------------------
|
||||
7 |
|
||||
|
@ -505,7 +485,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY user_id
|
||||
DEBUG: Plan is router executable
|
||||
-- FILTER CLAUSE
|
||||
INSERT INTO agg_events (user_id, value_1_agg)
|
||||
SELECT
|
||||
|
@ -516,7 +495,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY user_id
|
||||
DEBUG: Plan is router executable
|
||||
-- a test with reference table JOINs
|
||||
INSERT INTO
|
||||
agg_events (user_id, value_1_agg)
|
||||
|
@ -532,7 +510,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300001 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) GROUP BY raw_events_first.user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300002 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823))) GROUP BY raw_events_first.user_id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300003 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647))) GROUP BY raw_events_first.user_id
|
||||
DEBUG: Plan is router executable
|
||||
-- a note on the outer joins is that
|
||||
-- we filter out outer join results
|
||||
-- where partition column returns
|
||||
|
@ -581,7 +558,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300001 t1 FULL JOIN public.raw_events_second_13300005 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300002 t1 FULL JOIN public.raw_events_second_13300006 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300003 t1 FULL JOIN public.raw_events_second_13300007 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
-- see that the results are different from the SELECT query
|
||||
SELECT
|
||||
|
@ -617,7 +593,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- we don't want to see constraint violations, so truncate first
|
||||
SET client_min_messages TO INFO;
|
||||
truncate agg_events;
|
||||
|
@ -663,7 +638,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2;
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
user_id | value_1_agg
|
||||
|
@ -698,7 +672,6 @@ DEBUG: generating subplan XXX_1 for CTE fist_table_agg: SELECT (max(value_1) OP
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, v1_agg AS value_1_agg FROM (SELECT fist_table_agg.v1_agg, fist_table_agg.user_id FROM (SELECT intermediate_result.v1_agg, intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v1_agg integer, user_id integer)) fist_table_agg) citus_insert_select_subquery
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ROLLBACK;
|
||||
-- We don't support CTEs that are referenced in the target list
|
||||
|
@ -752,11 +725,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.raw_eve
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.raw_events_first
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) INTERSECT SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) citus_insert_select_subquery
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ROLLBACK;
|
||||
-- If the query is router plannable then it is executed via the coordinator
|
||||
|
@ -769,7 +740,6 @@ FROM
|
|||
(SELECT user_id FROM raw_events_second where user_id = 17)) as foo;
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- some supported LEFT joins
|
||||
INSERT INTO agg_events (user_id)
|
||||
|
@ -781,7 +751,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN public.raw_events_second_13300005 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN public.raw_events_second_13300006 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300003 raw_events_first LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_second.user_id
|
||||
|
@ -791,7 +760,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300005 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300006 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_first.user_id
|
||||
|
@ -802,7 +770,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: Skipping target shard interval 13300009 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300010 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300011 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_first.user_id
|
||||
|
@ -813,7 +780,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN (SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_second(user_id, "time", value_1, value_2, value_3, value_4) ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN (SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_second(user_id, "time", value_1, value_2, value_3, value_4) ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300003 raw_events_first LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_first.user_id
|
||||
|
@ -824,7 +790,6 @@ DEBUG: Skipping target shard interval 13300008 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300009 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300010 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300011 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_first.user_id
|
||||
|
@ -835,7 +800,6 @@ DEBUG: Skipping target shard interval 13300008 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300009 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300010 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300011 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_first.user_id
|
||||
|
@ -846,7 +810,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN public.raw_events_second_13300005 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN public.raw_events_second_13300006 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM ((SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_first(user_id, "time", value_1, value_2, value_3, value_4) LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
raw_events_first.user_id
|
||||
|
@ -857,7 +820,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first JOIN public.raw_events_second_13300005 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first JOIN public.raw_events_second_13300006 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300003 raw_events_first JOIN (SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_second(user_id, "time", value_1, value_2, value_3, value_4) ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO WARNING;
|
||||
-- following query should use repartitioned joins and results should
|
||||
-- be routed via coordinator
|
||||
|
@ -1173,7 +1135,6 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.value_3 AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_3
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(id) AS user_id, int4(v1) AS value_1_agg, int8(v4) AS value_4_agg FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 bigint, id double precision)) foo
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ERROR: the partition column of table public.agg_events cannot be NULL
|
||||
-- error cases
|
||||
|
@ -1287,7 +1248,6 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition
|
|||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- foo2 is recursively planned and INSERT...SELECT is done via coordinator
|
||||
INSERT INTO agg_events
|
||||
|
@ -1685,7 +1645,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
RESET client_min_messages;
|
||||
-- we cannot push this down since it is NOT IN
|
||||
-- we use repartition insert/select instead
|
||||
|
@ -1722,7 +1681,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300005 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300006 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300007 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
-- we cannot push down
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
|
@ -1735,7 +1693,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300005 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300006 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300007 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
-- more complex LEFT JOINs
|
||||
INSERT INTO agg_events
|
||||
(user_id, value_4_agg)
|
||||
|
@ -1769,7 +1726,6 @@ DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_t
|
|||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY id
|
||||
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300003 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY id
|
||||
DEBUG: Plan is router executable
|
||||
RESET client_min_messages;
|
||||
-- cannot push down since the f.id IN is matched with value_1
|
||||
-- we use repartition insert/select instead
|
||||
|
@ -1847,7 +1803,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first_1, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first_1, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first_1, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)))
|
||||
DEBUG: Plan is router executable
|
||||
RESET client_min_messages;
|
||||
-- cannot push down since top level user_id is matched with NOT IN
|
||||
INSERT INTO raw_events_second
|
||||
|
@ -2057,7 +2012,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS c
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300005 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300006 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300007 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
truncate raw_events_first;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
|
@ -2067,7 +2021,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS c
|
|||
DEBUG: Skipping target shard interval 13300001 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300002 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300003 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
-- if a single shard of the SELECT is unhealty, the query should fail
|
||||
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 13300004 AND nodeport = :worker_1_port;
|
||||
|
@ -2087,7 +2040,6 @@ DEBUG: Skipping target shard interval 13300000 since SELECT query for it pruned
|
|||
DEBUG: Skipping target shard interval 13300001 since SELECT query for it pruned away
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300006 raw_events_second WHERE ((user_id OPERATOR(pg_catalog.=) 6) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)))
|
||||
DEBUG: Skipping target shard interval 13300003 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
-- mark the unhealthy placement as healthy again for the next tests
|
||||
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE shardid = 13300004 AND nodeport = :worker_1_port;
|
||||
|
@ -2101,7 +2053,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS c
|
|||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300005 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300006 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300007 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
truncate raw_events_first;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
|
@ -2111,7 +2062,6 @@ DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS c
|
|||
DEBUG: Skipping target shard interval 13300001 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300002 since SELECT query for it pruned away
|
||||
DEBUG: Skipping target shard interval 13300003 since SELECT query for it pruned away
|
||||
DEBUG: Plan is router executable
|
||||
SET client_min_messages TO INFO;
|
||||
-- now do some tests with varchars
|
||||
INSERT INTO insert_select_varchar_test VALUES ('test_1', 10);
|
||||
|
@ -2164,7 +2114,6 @@ SET client_min_messages TO DEBUG2;
|
|||
INSERT INTO table_with_defaults SELECT * FROM table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, default_1, last_name, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, default_1, last_name, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- see that defaults are filled
|
||||
INSERT INTO table_with_defaults (store_id, first_name)
|
||||
SELECT
|
||||
|
@ -2173,7 +2122,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, '2'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, '2'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- shuffle one of the defaults and skip the other
|
||||
INSERT INTO table_with_defaults (default_2, store_id, first_name)
|
||||
SELECT
|
||||
|
@ -2182,7 +2130,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- shuffle both defaults
|
||||
INSERT INTO table_with_defaults (default_2, store_id, default_1, first_name)
|
||||
SELECT
|
||||
|
@ -2191,7 +2138,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, default_1, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, default_1, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- use constants instead of non-default column
|
||||
INSERT INTO table_with_defaults (default_2, last_name, store_id, first_name)
|
||||
SELECT
|
||||
|
@ -2200,7 +2146,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- use constants instead of non-default column and skip both defauls
|
||||
INSERT INTO table_with_defaults (last_name, store_id, first_name)
|
||||
SELECT
|
||||
|
@ -2209,7 +2154,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, '2'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, '2'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- use constants instead of default columns
|
||||
INSERT INTO table_with_defaults (default_2, last_name, store_id, first_name, default_1)
|
||||
SELECT
|
||||
|
@ -2218,7 +2162,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, 10, last_name, 20 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, 10, last_name, 20 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- use constants instead of both default columns and non-default columns
|
||||
INSERT INTO table_with_defaults (default_2, last_name, store_id, first_name, default_1)
|
||||
SELECT
|
||||
|
@ -2227,7 +2170,6 @@ FROM
|
|||
table_with_defaults;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 10, 'Freund'::text AS last_name, 20 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer))
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 10, 'Freund'::text AS last_name, 20 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647))
|
||||
DEBUG: Plan is router executable
|
||||
-- some of the ultimate queries where we have constants,
|
||||
-- defaults and group by entry is not on the target entry
|
||||
INSERT INTO table_with_defaults (default_2, store_id, first_name)
|
||||
|
@ -2239,7 +2181,6 @@ GROUP BY
|
|||
last_name, store_id;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, '2000'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, '2000'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO table_with_defaults (default_1, store_id, first_name, default_2)
|
||||
SELECT
|
||||
1000, store_id, 'Andres', '2000'
|
||||
|
@ -2249,7 +2190,6 @@ GROUP BY
|
|||
last_name, store_id, first_name;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id, first_name
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id, first_name
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO table_with_defaults (default_1, store_id, first_name, default_2)
|
||||
SELECT
|
||||
1000, store_id, 'Andres', '2000'
|
||||
|
@ -2259,7 +2199,6 @@ GROUP BY
|
|||
last_name, store_id, first_name, default_2;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id, first_name, default_2
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id, first_name, default_2
|
||||
DEBUG: Plan is router executable
|
||||
INSERT INTO table_with_defaults (default_1, store_id, first_name)
|
||||
SELECT
|
||||
1000, store_id, 'Andres'
|
||||
|
@ -2269,7 +2208,6 @@ GROUP BY
|
|||
last_name, store_id, first_name, default_2;
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id, first_name, default_2
|
||||
DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id, first_name, default_2
|
||||
DEBUG: Plan is router executable
|
||||
RESET client_min_messages;
|
||||
-- Stable function in default should be allowed
|
||||
ALTER TABLE table_with_defaults ADD COLUMN t timestamptz DEFAULT now();
|
||||
|
|
|
@ -105,7 +105,6 @@ INSERT INTO append_partitioned VALUES (414123, 'AAPL', 9580, '2004-10-19 10:23:5
|
|||
SET client_min_messages TO 'DEBUG2';
|
||||
SELECT * FROM range_partitioned WHERE id = 32743;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | symbol | bidder_id | placed_at | kind | limit_price
|
||||
---------------------------------------------------------------------
|
||||
32743 | AAPL | 9580 | Tue Oct 19 10:23:54 2004 | buy | 20.69
|
||||
|
@ -113,7 +112,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT * FROM append_partitioned WHERE id = 414123;
|
||||
DEBUG: Router planner does not support append-partitioned tables.
|
||||
DEBUG: Plan is router executable
|
||||
id | symbol | bidder_id | placed_at | kind | limit_price
|
||||
---------------------------------------------------------------------
|
||||
414123 | AAPL | 9580 | Tue Oct 19 10:23:54 2004 | buy | 20.69
|
||||
|
|
|
@ -77,7 +77,6 @@ SET client_min_messages TO DEBUG;
|
|||
SELECT count(*) FROM ref;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM mx_add_coordinator.ref_7000000 ref
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -87,7 +86,6 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM mx_add_coo
|
|||
SELECT count(*) FROM ref;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM mx_add_coordinator.ref_7000000 ref
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -99,7 +97,6 @@ SET citus.task_assignment_policy TO "round-robin";
|
|||
SELECT count(*) FROM ref;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -108,7 +105,6 @@ DEBUG: Plan is router executable
|
|||
SELECT count(*) FROM ref;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -117,7 +113,6 @@ DEBUG: Plan is router executable
|
|||
SELECT count(*) FROM ref;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -126,7 +121,6 @@ DEBUG: Plan is router executable
|
|||
-- modifications always go through local shard as well as remote ones
|
||||
INSERT INTO ref VALUES (1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: INSERT INTO mx_add_coordinator.ref_7000000 (a) VALUES (1)
|
||||
-- get it ready for the next executions
|
||||
TRUNCATE ref;
|
||||
|
|
|
@ -921,7 +921,6 @@ SELECT create_reference_table('numbers');
|
|||
SET client_min_messages TO debug4;
|
||||
INSERT INTO numbers VALUES (1), (2), (3), (4);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: query before rebuilding: (null)
|
||||
DEBUG: query after rebuilding: INSERT INTO public.numbers_1250015 AS citus_table_alias (a) VALUES (1), (2), (3), (4)
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
|
@ -934,7 +933,6 @@ DEBUG: Total number of commands sent over the session 5: 1
|
|||
SELECT count(*) FROM numbers;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
4
|
||||
|
|
|
@ -63,8 +63,7 @@ SET client_min_messages TO 'DEBUG2';
|
|||
-- insert a single row for the test
|
||||
INSERT INTO articles_single_shard_hash_mx VALUES (50, 10, 'anjanette', 19519);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
-- single-shard tests
|
||||
-- many of the tests in this file is intended for testing non-fast-path
|
||||
-- router planner, so we're explicitly disabling it in this file.
|
||||
|
@ -73,8 +72,7 @@ SET citus.enable_fast_path_router_planner TO false;
|
|||
-- test simple select for a single row
|
||||
SELECT * FROM articles_hash_mx WHERE author_id = 10 AND id = 50;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
50 | 10 | anjanette | 19519
|
||||
|
@ -83,8 +81,7 @@ DETAIL: distribution column value: 10
|
|||
-- get all titles by a single author
|
||||
SELECT title FROM articles_hash_mx WHERE author_id = 10;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
title
|
||||
---------------------------------------------------------------------
|
||||
aggrandize
|
||||
|
@ -99,8 +96,7 @@ SELECT title, word_count FROM articles_hash_mx
|
|||
WHERE author_id = 10
|
||||
ORDER BY word_count DESC NULLS LAST;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
title | word_count
|
||||
---------------------------------------------------------------------
|
||||
anjanette | 19519
|
||||
|
@ -116,8 +112,7 @@ SELECT title, id FROM articles_hash_mx
|
|||
ORDER BY id
|
||||
LIMIT 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
title | id
|
||||
---------------------------------------------------------------------
|
||||
aruru | 5
|
||||
|
@ -130,7 +125,6 @@ SELECT title, author_id FROM articles_hash_mx
|
|||
WHERE author_id = 7 OR author_id = 8
|
||||
ORDER BY author_id ASC, id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
title | author_id
|
||||
---------------------------------------------------------------------
|
||||
aseptic | 7
|
||||
|
@ -149,7 +143,6 @@ DEBUG: Plan is router executable
|
|||
SELECT title, author_id FROM articles_hash_mx
|
||||
WHERE author_id = 7 OR author_id = 8;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
title | author_id
|
||||
---------------------------------------------------------------------
|
||||
aseptic | 7
|
||||
|
@ -172,7 +165,6 @@ SELECT author_id, sum(word_count) AS corpus_size FROM articles_hash_mx
|
|||
HAVING sum(word_count) > 1000
|
||||
ORDER BY sum(word_count) DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
author_id | corpus_size
|
||||
---------------------------------------------------------------------
|
||||
10 | 59955
|
||||
|
@ -188,8 +180,7 @@ SELECT author_id, sum(word_count) AS corpus_size FROM articles_hash_mx
|
|||
HAVING sum(word_count) > 1000
|
||||
ORDER BY sum(word_count) DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
author_id | corpus_size
|
||||
---------------------------------------------------------------------
|
||||
1 | 35894
|
||||
|
@ -210,7 +201,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
|
||||
SELECT * FROM articles_hash_mx WHERE author_id IN (1, 3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -230,8 +220,7 @@ WITH first_author AS ( SELECT id FROM articles_hash_mx WHERE author_id = 1)
|
|||
SELECT * FROM first_author;
|
||||
DEBUG: CTE first_author is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -245,8 +234,7 @@ DETAIL: distribution column value: 1
|
|||
WITH first_author AS ( SELECT id FROM articles_hash_mx WHERE author_id = 1)
|
||||
SELECT title FROM articles_hash_mx WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
title
|
||||
---------------------------------------------------------------------
|
||||
arsenous
|
||||
|
@ -263,8 +251,7 @@ SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id;
|
|||
DEBUG: CTE id_author is going to be inlined via distributed planning
|
||||
DEBUG: CTE id_title is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | 1 | arsenous
|
||||
|
@ -280,7 +267,6 @@ SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id;
|
|||
DEBUG: CTE id_author is going to be inlined via distributed planning
|
||||
DEBUG: CTE id_title is going to be inlined via distributed planning
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -293,13 +279,11 @@ DEBUG: CTE id_author is going to be inlined via distributed planning
|
|||
DEBUG: CTE id_title is going to be inlined via distributed planning
|
||||
DEBUG: cannot run command which targets multiple shards
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM public.articles_hash_mx WHERE (author_id OPERATOR(pg_catalog.=) 2)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id_author.id, id_author.author_id, id_title.id, id_title.title FROM (SELECT articles_hash_mx.id, articles_hash_mx.author_id FROM public.articles_hash_mx WHERE (articles_hash_mx.author_id OPERATOR(pg_catalog.=) 1)) id_author, (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title character varying(20))) id_title WHERE (id_author.id OPERATOR(pg_catalog.=) id_title.id)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -307,36 +291,28 @@ DETAIL: distribution column value: 1
|
|||
-- recursive CTEs are supported when filtered on partition column
|
||||
INSERT INTO company_employees_mx values(1, 1, 0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees_mx values(1, 2, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees_mx values(1, 3, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees_mx values(1, 4, 2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees_mx values(1, 5, 4);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees_mx values(3, 1, 0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
INSERT INTO company_employees_mx values(3, 15, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
INSERT INTO company_employees_mx values(3, 3, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
-- find employees at top 2 level within company hierarchy
|
||||
WITH RECURSIVE hierarchy as (
|
||||
SELECT *, 1 AS level
|
||||
|
@ -350,8 +326,7 @@ WITH RECURSIVE hierarchy as (
|
|||
ce.company_id = 1))
|
||||
SELECT * FROM hierarchy WHERE LEVEL <= 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
company_id | employee_id | manager_id | level
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | 0 | 1
|
||||
|
@ -397,7 +372,6 @@ SELECT
|
|||
GROUP BY GROUPING SETS ((id),(subtitle))
|
||||
ORDER BY id, subtitle;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | subtitle | count
|
||||
---------------------------------------------------------------------
|
||||
1 | | 1
|
||||
|
@ -433,8 +407,7 @@ HINT: Consider using an equality filter on the distributed table's partition co
|
|||
-- queries which involve functions in FROM clause are supported if it goes to a single worker.
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count | position
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572 | 3
|
||||
|
@ -446,7 +419,6 @@ DETAIL: distribution column value: 1
|
|||
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count | position
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572 | 3
|
||||
|
@ -502,8 +474,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT id, word_count FROM public.articles_hash_mx
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT articles_hash_mx.id, test.word_count FROM public.articles_hash_mx, (SELECT intermediate_result.id, intermediate_result.word_count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, word_count integer)) test WHERE ((test.id OPERATOR(pg_catalog.=) articles_hash_mx.id) AND (articles_hash_mx.author_id OPERATOR(pg_catalog.=) 1)) ORDER BY articles_hash_mx.id
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 9572
|
||||
|
@ -525,8 +496,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -541,7 +511,6 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1 OR author_id = 17;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -571,8 +540,7 @@ SELECT id as article_id, word_count * id as random_value
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
article_id | random_value
|
||||
---------------------------------------------------------------------
|
||||
1 | 9572
|
||||
|
@ -588,8 +556,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 17277
|
||||
|
@ -604,8 +571,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 19519
|
||||
|
@ -633,8 +599,7 @@ SELECT *
|
|||
WHERE author_id = 1
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -649,8 +614,7 @@ SELECT *
|
|||
LIMIT 2
|
||||
OFFSET 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
11 | 1 | alamo | 1347
|
||||
|
@ -665,8 +629,7 @@ SELECT *
|
|||
LIMIT 2
|
||||
OFFSET 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
31 | 1 | athwartships | 7271
|
||||
|
@ -680,8 +643,7 @@ SELECT id
|
|||
GROUP BY id
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -697,8 +659,7 @@ SELECT distinct id
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -713,8 +674,7 @@ SELECT avg(word_count)
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
avg
|
||||
---------------------------------------------------------------------
|
||||
12356.400000000000
|
||||
|
@ -726,8 +686,7 @@ SELECT max(word_count) as max, min(word_count) as min,
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
max | min | sum | cnt
|
||||
---------------------------------------------------------------------
|
||||
18185 | 2728 | 61782 | 5
|
||||
|
@ -739,8 +698,7 @@ SELECT max(word_count)
|
|||
WHERE author_id = 1
|
||||
GROUP BY author_id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
max
|
||||
---------------------------------------------------------------------
|
||||
11814
|
||||
|
@ -754,7 +712,6 @@ SELECT * FROM (
|
|||
) AS combination
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -773,7 +730,6 @@ DEBUG: Plan is router executable
|
|||
UNION
|
||||
(SELECT LEFT(title, 1) FROM articles_hash_mx WHERE author_id = 3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
left
|
||||
---------------------------------------------------------------------
|
||||
a
|
||||
|
@ -783,7 +739,6 @@ DEBUG: Plan is router executable
|
|||
INTERSECT
|
||||
(SELECT LEFT(title, 1) FROM articles_hash_mx WHERE author_id = 3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
left
|
||||
---------------------------------------------------------------------
|
||||
a
|
||||
|
@ -796,7 +751,6 @@ SELECT * FROM (
|
|||
) AS combination
|
||||
ORDER BY 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
left
|
||||
---------------------------------------------------------------------
|
||||
al
|
||||
|
@ -883,8 +837,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1 and author_id >= 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -913,8 +866,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1 and (id = 1 or id = 41);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -926,8 +878,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1 and (id = random()::int * 0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -965,8 +916,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = abs(-1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1009,8 +959,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1 and (id = abs(id - 2));
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1035,8 +984,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE (author_id = 1) = true;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1051,8 +999,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE (author_id = 1) and id between 0 and 20;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1064,8 +1011,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE (author_id = 1) and (id = 1 or id = 31) and title like '%s';
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1077,8 +1023,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE (id = 1 or id = 31) and title like '%s' and (author_id = 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1090,8 +1035,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE (title like '%s' or title like 'a%') and (author_id = 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1106,8 +1050,7 @@ SELECT *
|
|||
FROM articles_hash_mx
|
||||
WHERE (title like '%s' or title like 'a%') and (author_id = 1) and (word_count < 3000 or word_count > 8000);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1120,8 +1063,7 @@ SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
prev | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
| afrasia | 864
|
||||
|
@ -1136,8 +1078,7 @@ SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count
|
|||
WHERE author_id = 5
|
||||
ORDER BY word_count DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
prev | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
aminate | aruru | 11389
|
||||
|
@ -1151,8 +1092,7 @@ SELECT id, MIN(id) over (order by word_count)
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | min
|
||||
---------------------------------------------------------------------
|
||||
11 | 11
|
||||
|
@ -1166,8 +1106,7 @@ SELECT id, word_count, AVG(word_count) over (order by word_count)
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | word_count | avg
|
||||
---------------------------------------------------------------------
|
||||
11 | 1347 | 1347.0000000000000000
|
||||
|
@ -1181,8 +1120,7 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count)
|
|||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
word_count | rank
|
||||
---------------------------------------------------------------------
|
||||
1347 | 1
|
||||
|
@ -1245,8 +1183,7 @@ SELECT
|
|||
WHERE
|
||||
author_id = 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
c
|
||||
---------------------------------------------------------------------
|
||||
5
|
||||
|
@ -1289,8 +1226,7 @@ SELECT *
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1309,8 +1245,7 @@ DECLARE test_cursor CURSOR FOR
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
FETCH test_cursor;
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1337,8 +1272,7 @@ COPY (
|
|||
WHERE author_id = 1
|
||||
ORDER BY id) TO STDOUT;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
1 1 arsenous 9572
|
||||
11 1 alamo 1347
|
||||
21 1 arcading 5890
|
||||
|
@ -1351,15 +1285,13 @@ CREATE TEMP TABLE temp_articles_hash_mx as
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- router plannable queries may include filter for aggregates
|
||||
SELECT count(*), count(*) FILTER (WHERE id < 3)
|
||||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count | count
|
||||
---------------------------------------------------------------------
|
||||
5 | 1
|
||||
|
@ -1382,8 +1314,7 @@ PREPARE author_1_articles as
|
|||
WHERE author_id = 1;
|
||||
EXECUTE author_1_articles;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1400,8 +1331,7 @@ PREPARE author_articles(int) as
|
|||
WHERE author_id = $1;
|
||||
EXECUTE author_articles(1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1427,8 +1357,7 @@ DEBUG: Creating router plan
|
|||
CONTEXT: SQL statement "SELECT MAX(id) FROM articles_hash_mx ah
|
||||
WHERE author_id = 1"
|
||||
PL/pgSQL function author_articles_max_id() line 5 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
CONTEXT: SQL statement "SELECT MAX(id) FROM articles_hash_mx ah
|
||||
WHERE author_id = 1"
|
||||
PL/pgSQL function author_articles_max_id() line 5 at SQL statement
|
||||
|
@ -1454,8 +1383,7 @@ CONTEXT: SQL statement "SELECT ah.id, ah.word_count
|
|||
FROM articles_hash_mx ah
|
||||
WHERE author_id = 1"
|
||||
PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
CONTEXT: SQL statement "SELECT ah.id, ah.word_count
|
||||
FROM articles_hash_mx ah
|
||||
WHERE author_id = 1"
|
||||
|
@ -1473,8 +1401,7 @@ PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY
|
|||
CREATE MATERIALIZED VIEW mv_articles_hash_mx AS
|
||||
SELECT * FROM articles_hash_mx WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
SELECT * FROM mv_articles_hash_mx;
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1500,15 +1427,13 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
-- insert query is router plannable even under task-tracker
|
||||
INSERT INTO articles_hash_mx VALUES (51, 1, 'amateus', 1814);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- verify insert is successfull (not router plannable and executable)
|
||||
SELECT id
|
||||
FROM articles_hash_mx
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
|
|
@ -210,8 +210,6 @@ $Q$);
|
|||
DEBUG: Router planner does not support append-partitioned tables.
|
||||
CONTEXT: PL/pgSQL function coordinator_plan(text) line 3 at FOR over EXECUTE statement
|
||||
LOG: join order: [ "lineitem" ]
|
||||
CONTEXT: PL/pgSQL function coordinator_plan(text) line 3 at FOR over EXECUTE statement
|
||||
DEBUG: Plan is router executable
|
||||
CONTEXT: PL/pgSQL function coordinator_plan(text) line 3 at FOR over EXECUTE statement
|
||||
coordinator_plan
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -127,14 +127,12 @@ SET client_min_messages TO 'DEBUG2';
|
|||
-- insert a single row for the test
|
||||
INSERT INTO articles_single_shard_hash VALUES (50, 10, 'anjanette', 19519);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
-- single-shard tests
|
||||
-- test simple select for a single row
|
||||
SELECT * FROM articles_hash WHERE author_id = 10 AND id = 50;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
50 | 10 | anjanette | 19519
|
||||
|
@ -143,8 +141,7 @@ DETAIL: distribution column value: 10
|
|||
-- get all titles by a single author
|
||||
SELECT title FROM articles_hash WHERE author_id = 10;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
title
|
||||
---------------------------------------------------------------------
|
||||
aggrandize
|
||||
|
@ -159,8 +156,7 @@ SELECT title, word_count FROM articles_hash
|
|||
WHERE author_id = 10
|
||||
ORDER BY word_count DESC NULLS LAST;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
title | word_count
|
||||
---------------------------------------------------------------------
|
||||
anjanette | 19519
|
||||
|
@ -176,8 +172,7 @@ SELECT title, id FROM articles_hash
|
|||
ORDER BY id
|
||||
LIMIT 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
title | id
|
||||
---------------------------------------------------------------------
|
||||
aruru | 5
|
||||
|
@ -190,7 +185,6 @@ SELECT title, author_id FROM articles_hash
|
|||
WHERE author_id = 7 OR author_id = 8
|
||||
ORDER BY author_id ASC, id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
title | author_id
|
||||
---------------------------------------------------------------------
|
||||
aseptic | 7
|
||||
|
@ -209,7 +203,6 @@ DEBUG: Plan is router executable
|
|||
SELECT title, author_id FROM articles_hash
|
||||
WHERE author_id = 7 OR author_id = 8;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
title | author_id
|
||||
---------------------------------------------------------------------
|
||||
aseptic | 7
|
||||
|
@ -232,7 +225,6 @@ SELECT author_id, sum(word_count) AS corpus_size FROM articles_hash
|
|||
HAVING sum(word_count) > 1000
|
||||
ORDER BY sum(word_count) DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
author_id | corpus_size
|
||||
---------------------------------------------------------------------
|
||||
10 | 59955
|
||||
|
@ -248,8 +240,7 @@ SELECT author_id, sum(word_count) AS corpus_size FROM articles_hash
|
|||
HAVING sum(word_count) > 1000
|
||||
ORDER BY sum(word_count) DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
author_id | corpus_size
|
||||
---------------------------------------------------------------------
|
||||
1 | 35894
|
||||
|
@ -270,7 +261,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
|
||||
SELECT * FROM articles_hash WHERE author_id IN (1, 3) ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -287,8 +277,7 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT * FROM articles_hash WHERE author_id IN (1, NULL) ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -302,8 +291,7 @@ DETAIL: distribution column value: 1
|
|||
WITH first_author AS ( SELECT id FROM articles_hash WHERE author_id = 1)
|
||||
SELECT * FROM first_author;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -325,12 +313,10 @@ FROM first_author WHERE articles_hash.author_id = 2 AND articles_hash.id = first
|
|||
DEBUG: Router planner doesn't support SELECT FOR UPDATE in common table expressions involving reference tables.
|
||||
DEBUG: generating subplan XXX_1 for CTE first_author: SELECT articles_hash.id, auref.name FROM public.articles_hash, public.authors_reference auref WHERE ((articles_hash.author_id OPERATOR(pg_catalog.=) 2) AND (auref.id OPERATOR(pg_catalog.=) articles_hash.author_id)) FOR UPDATE OF articles_hash FOR UPDATE OF auref
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE public.articles_hash SET title = first_author.name FROM (SELECT intermediate_result.id, intermediate_result.name FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name character varying(20))) first_author WHERE ((articles_hash.author_id OPERATOR(pg_catalog.=) 2) AND (articles_hash.id OPERATOR(pg_catalog.=) first_author.id))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
WITH first_author AS (
|
||||
SELECT id, word_count FROM articles_hash WHERE author_id = 2
|
||||
FOR UPDATE
|
||||
|
@ -338,8 +324,7 @@ WITH first_author AS (
|
|||
UPDATE articles_hash SET title = first_author.word_count::text
|
||||
FROM first_author WHERE articles_hash.author_id = 2 AND articles_hash.id = first_author.id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
-- Without FOR UPDATE this is router plannable
|
||||
WITH first_author AS (
|
||||
SELECT articles_hash.id, auref.name FROM articles_hash, authors_reference auref
|
||||
|
@ -348,15 +333,13 @@ WITH first_author AS (
|
|||
UPDATE articles_hash SET title = first_author.name
|
||||
FROM first_author WHERE articles_hash.author_id = 2 AND articles_hash.id = first_author.id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
ROLLBACK;
|
||||
-- queries with CTEs are supported even if CTE is not referenced inside query
|
||||
WITH first_author AS ( SELECT id FROM articles_hash WHERE author_id = 1)
|
||||
SELECT title FROM articles_hash WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
title
|
||||
---------------------------------------------------------------------
|
||||
arsenous
|
||||
|
@ -371,8 +354,7 @@ WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1)
|
|||
id_title AS (SELECT id, title from articles_hash WHERE author_id = 1)
|
||||
SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | 1 | arsenous
|
||||
|
@ -386,7 +368,6 @@ WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1)
|
|||
id_title AS (SELECT id, title from articles_hash WHERE author_id = 3)
|
||||
SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -398,15 +379,12 @@ SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id;
|
|||
DEBUG: cannot run command which targets multiple shards
|
||||
DEBUG: generating subplan XXX_1 for CTE id_author: SELECT id, author_id FROM public.articles_hash WHERE (author_id OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: generating subplan XXX_2 for CTE id_title: SELECT id, title FROM public.articles_hash WHERE (author_id OPERATOR(pg_catalog.=) 2)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id_author.id, id_author.author_id, id_title.id, id_title.title FROM (SELECT intermediate_result.id, intermediate_result.author_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, author_id bigint)) id_author, (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title character varying(20))) id_title WHERE (id_author.id OPERATOR(pg_catalog.=) id_title.id)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -427,36 +405,28 @@ SELECT master_create_worker_shards('company_employees', 4, 1);
|
|||
|
||||
INSERT INTO company_employees values(1, 1, 0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees values(1, 2, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees values(1, 3, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees values(1, 4, 2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees values(1, 5, 4);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
INSERT INTO company_employees values(3, 1, 0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
INSERT INTO company_employees values(3, 15, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
INSERT INTO company_employees values(3, 3, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
-- find employees at top 2 level within company hierarchy
|
||||
WITH RECURSIVE hierarchy as (
|
||||
SELECT *, 1 AS level
|
||||
|
@ -470,8 +440,7 @@ WITH RECURSIVE hierarchy as (
|
|||
ce.company_id = 1))
|
||||
SELECT * FROM hierarchy WHERE LEVEL <= 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
company_id | employee_id | manager_id | level
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | 0 | 1
|
||||
|
@ -516,11 +485,9 @@ SELECT * FROM new_article;
|
|||
DEBUG: only SELECT, UPDATE, or DELETE common table expressions may be router planned
|
||||
DEBUG: generating subplan XXX_1 for CTE new_article: INSERT INTO public.articles_hash (id, author_id, title, word_count) VALUES (1, 1, 'arsenous'::character varying, 9) RETURNING id, author_id, title, word_count
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id, author_id, title, word_count FROM (SELECT intermediate_result.id, intermediate_result.author_id, intermediate_result.title, intermediate_result.word_count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, author_id bigint, title character varying(20), word_count integer)) new_article
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9
|
||||
|
@ -533,10 +500,8 @@ SELECT * FROM update_article;
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for CTE update_article: UPDATE public.articles_hash SET word_count = 10 WHERE ((id OPERATOR(pg_catalog.=) 1) AND (word_count OPERATOR(pg_catalog.=) 9)) RETURNING id, author_id, title, word_count
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id, author_id, title, word_count FROM (SELECT intermediate_result.id, intermediate_result.author_id, intermediate_result.title, intermediate_result.word_count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, author_id bigint, title character varying(20), word_count integer)) update_article
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 10
|
||||
|
@ -549,10 +514,8 @@ SELECT * FROM delete_article;
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for CTE delete_article: DELETE FROM public.articles_hash WHERE ((id OPERATOR(pg_catalog.=) 1) AND (word_count OPERATOR(pg_catalog.=) 10)) RETURNING id, author_id, title, word_count
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id, author_id, title, word_count FROM (SELECT intermediate_result.id, intermediate_result.author_id, intermediate_result.title, intermediate_result.word_count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, author_id bigint, title character varying(20), word_count integer)) delete_article
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 10
|
||||
|
@ -583,7 +546,6 @@ SELECT
|
|||
GROUP BY GROUPING SETS ((id),(subtitle))
|
||||
ORDER BY id, subtitle;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | subtitle | count
|
||||
---------------------------------------------------------------------
|
||||
1 | | 1
|
||||
|
@ -619,8 +581,7 @@ HINT: Consider using an equality filter on the distributed table's partition co
|
|||
-- queries which involve functions in FROM clause are supported if it goes to a single worker.
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count | position
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572 | 3
|
||||
|
@ -632,7 +593,6 @@ DETAIL: distribution column value: 1
|
|||
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count | position
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572 | 3
|
||||
|
@ -665,8 +625,7 @@ SELECT * FROM articles_hash
|
|||
WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 2)
|
||||
ORDER BY articles_hash.id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
2 | 2 | abducing | 13642
|
||||
|
@ -704,8 +663,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT id, word_count FROM public.articles_hash
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT articles_hash.id, test.word_count FROM public.articles_hash, (SELECT intermediate_result.id, intermediate_result.word_count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, word_count integer)) test WHERE ((test.id OPERATOR(pg_catalog.=) articles_hash.id) AND (articles_hash.author_id OPERATOR(pg_catalog.=) 1)) ORDER BY articles_hash.id
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 9572
|
||||
|
@ -727,8 +685,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -743,7 +700,6 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 OR author_id = 17;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -774,8 +730,7 @@ SELECT id as article_id, word_count * id as random_value
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
article_id | random_value
|
||||
---------------------------------------------------------------------
|
||||
1 | 9572
|
||||
|
@ -791,8 +746,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 17277
|
||||
|
@ -807,8 +761,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 19519
|
||||
|
@ -827,11 +780,9 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
DEBUG: found no worker with all shard placements
|
||||
DEBUG: generating subplan XXX_1 for CTE single_shard: SELECT id, author_id, title, word_count FROM public.articles_single_shard_hash
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a.author_id AS first_author, b.word_count AS second_word_count FROM public.articles_hash a, (SELECT intermediate_result.id, intermediate_result.author_id, intermediate_result.title, intermediate_result.word_count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, author_id bigint, title character varying(20), word_count integer)) b WHERE ((a.author_id OPERATOR(pg_catalog.=) 2) AND (a.author_id OPERATOR(pg_catalog.=) b.author_id)) LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -842,8 +793,7 @@ SELECT *
|
|||
WHERE author_id = 1
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -858,8 +808,7 @@ SELECT *
|
|||
LIMIT 2
|
||||
OFFSET 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
11 | 1 | alamo | 1347
|
||||
|
@ -874,8 +823,7 @@ SELECT *
|
|||
LIMIT 2
|
||||
OFFSET 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
31 | 1 | athwartships | 7271
|
||||
|
@ -889,8 +837,7 @@ SELECT id
|
|||
GROUP BY id
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -906,8 +853,7 @@ SELECT DISTINCT id
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -922,8 +868,7 @@ SELECT avg(word_count)
|
|||
FROM articles_hash
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
avg
|
||||
---------------------------------------------------------------------
|
||||
12356.400000000000
|
||||
|
@ -935,8 +880,7 @@ SELECT max(word_count) as max, min(word_count) as min,
|
|||
FROM articles_hash
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
max | min | sum | cnt
|
||||
---------------------------------------------------------------------
|
||||
18185 | 2728 | 61782 | 5
|
||||
|
@ -948,8 +892,7 @@ SELECT max(word_count)
|
|||
WHERE author_id = 1
|
||||
GROUP BY author_id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
max
|
||||
---------------------------------------------------------------------
|
||||
11814
|
||||
|
@ -963,7 +906,6 @@ SELECT * FROM (
|
|||
) AS combination
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -982,7 +924,6 @@ DEBUG: Plan is router executable
|
|||
UNION
|
||||
(SELECT LEFT(title, 1) FROM articles_hash WHERE author_id = 3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
left
|
||||
---------------------------------------------------------------------
|
||||
a
|
||||
|
@ -992,7 +933,6 @@ DEBUG: Plan is router executable
|
|||
INTERSECT
|
||||
(SELECT LEFT(title, 1) FROM articles_hash WHERE author_id = 3);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
left
|
||||
---------------------------------------------------------------------
|
||||
a
|
||||
|
@ -1005,7 +945,6 @@ SELECT * FROM (
|
|||
) AS combination
|
||||
ORDER BY 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
left
|
||||
---------------------------------------------------------------------
|
||||
al
|
||||
|
@ -1114,8 +1053,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and author_id >= 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1144,8 +1082,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and (id = 1 or id = 41);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1157,8 +1094,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and (id = random()::int * 0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1196,8 +1132,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = abs(-1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1240,8 +1175,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and (id = abs(id - 2));
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1266,8 +1200,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE (author_id = 1) = true;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1282,8 +1215,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE (author_id = 1) and id between 0 and 20;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1295,8 +1227,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE (author_id = 1) and (id = 1 or id = 31) and title like '%s';
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1308,8 +1239,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE (id = 1 or id = 31) and title like '%s' and (author_id = 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1321,8 +1251,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE (title like '%s' or title like 'a%') and (author_id = 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1337,8 +1266,7 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE (title like '%s' or title like 'a%') and (author_id = 1) and (word_count < 3000 or word_count > 8000);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1350,8 +1278,7 @@ SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count
|
|||
FROM articles_hash
|
||||
WHERE author_id = 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
prev | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
| afrasia | 864
|
||||
|
@ -1366,8 +1293,7 @@ SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count
|
|||
WHERE author_id = 5
|
||||
ORDER BY word_count DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
prev | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
aminate | aruru | 11389
|
||||
|
@ -1381,8 +1307,7 @@ SELECT id, MIN(id) over (order by word_count)
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | min
|
||||
---------------------------------------------------------------------
|
||||
11 | 11
|
||||
|
@ -1396,8 +1321,7 @@ SELECT id, word_count, AVG(word_count) over (order by word_count)
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | word_count | avg
|
||||
---------------------------------------------------------------------
|
||||
11 | 1347 | 1347.0000000000000000
|
||||
|
@ -1411,8 +1335,7 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count)
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
word_count | rank
|
||||
---------------------------------------------------------------------
|
||||
1347 | 1
|
||||
|
@ -1465,7 +1388,6 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE false;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1474,7 +1396,6 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and false;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1483,7 +1404,6 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and 1=0;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1492,7 +1412,6 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
FROM articles_hash a, articles_single_shard_hash b
|
||||
WHERE a.author_id = 10 and a.author_id = b.author_id and false;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1501,7 +1420,6 @@ SELECT *
|
|||
FROM articles_hash
|
||||
WHERE null;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1511,7 +1429,6 @@ SELECT *
|
|||
FROM articles_hash a
|
||||
WHERE a.author_id = 10 and int4eq(1, 2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1520,7 +1437,6 @@ SELECT *
|
|||
FROM articles_hash a
|
||||
WHERE int4eq(1, 2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1529,8 +1445,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
FROM articles_hash a, articles_single_shard_hash b
|
||||
WHERE a.author_id = 10 and a.author_id = b.author_id and int4eq(1, 1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 19519
|
||||
|
@ -1544,7 +1459,6 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
FROM articles_hash a, articles_single_shard_hash b
|
||||
WHERE a.author_id = 10 and a.author_id = b.author_id and int4eq(1, 2);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1565,7 +1479,6 @@ SELECT *
|
|||
FROM articles_hash a
|
||||
WHERE a.author_id = null;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1575,7 +1488,6 @@ SELECT *
|
|||
FROM articles_hash a
|
||||
WHERE date_ne_timestamp('1954-04-11', '1954-04-11'::timestamp);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1585,7 +1497,6 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id and
|
||||
date_ne_timestamp('1954-04-11', '1954-04-11'::timestamp);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1600,8 +1511,7 @@ SELECT * FROM (
|
|||
) AS combination
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1618,8 +1528,7 @@ SELECT * FROM (
|
|||
) AS combination
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -1633,8 +1542,7 @@ DETAIL: distribution column value: 1
|
|||
INTERSECT
|
||||
(SELECT * FROM articles_hash WHERE author_id = 2 and 1=0);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1642,11 +1550,10 @@ DETAIL: distribution column value: 1
|
|||
-- CTEs with where false
|
||||
-- terse because distribution column inference varies between pg11 & pg12
|
||||
\set VERBOSITY terse
|
||||
RESET client_min_messages;
|
||||
WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1),
|
||||
id_title AS (SELECT id, title from articles_hash WHERE author_id = 1 and 1=0)
|
||||
SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1654,12 +1561,11 @@ DEBUG: Plan is router executable
|
|||
WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1),
|
||||
id_title AS (SELECT id, title from articles_hash WHERE author_id = 1)
|
||||
SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id and 1=0;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | id | title
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
\set VERBOSITY DEFAULT
|
||||
WITH RECURSIVE hierarchy as (
|
||||
SELECT *, 1 AS level
|
||||
|
@ -1673,8 +1579,7 @@ WITH RECURSIVE hierarchy as (
|
|||
ce.company_id = 1))
|
||||
SELECT * FROM hierarchy WHERE LEVEL <= 2 and 1=0;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
company_id | employee_id | manager_id | level
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1691,8 +1596,7 @@ WITH RECURSIVE hierarchy as (
|
|||
ce.company_id = 1 AND 1=0))
|
||||
SELECT * FROM hierarchy WHERE LEVEL <= 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
company_id | employee_id | manager_id | level
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | 0 | 1
|
||||
|
@ -1710,8 +1614,7 @@ WITH RECURSIVE hierarchy as (
|
|||
ce.company_id = 1))
|
||||
SELECT * FROM hierarchy WHERE LEVEL <= 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
company_id | employee_id | manager_id | level
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1721,7 +1624,6 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count)
|
|||
FROM articles_hash
|
||||
WHERE author_id = 1 and 1=0;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
word_count | rank
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1748,7 +1650,6 @@ SELECT author_id FROM articles_hash
|
|||
author_id
|
||||
LIMIT 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
author_id
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1789,7 +1690,6 @@ SET citus.log_remote_commands TO on;
|
|||
-- single shard select queries are router plannable
|
||||
SELECT * FROM articles_range where author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: issuing SELECT id, author_id, title, word_count FROM public.articles_range_840012 articles_range WHERE (author_id OPERATOR(pg_catalog.=) 1)
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
id | author_id | title | word_count
|
||||
|
@ -1798,7 +1698,6 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|||
|
||||
SELECT * FROM articles_range where author_id = 1 or author_id = 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: issuing SELECT id, author_id, title, word_count FROM public.articles_range_840012 articles_range WHERE ((author_id OPERATOR(pg_catalog.=) 1) OR (author_id OPERATOR(pg_catalog.=) 5))
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
id | author_id | title | word_count
|
||||
|
@ -1808,7 +1707,6 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|||
-- zero shard select query is router plannable
|
||||
SELECT * FROM articles_range where author_id = 1 and author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT id, author_id, title, word_count FROM (SELECT NULL::bigint AS id, NULL::bigint AS author_id, NULL::character varying(20) AS title, NULL::integer AS word_count WHERE false) articles_range(id, author_id, title, word_count) WHERE ((author_id OPERATOR(pg_catalog.=) 1) AND (author_id OPERATOR(pg_catalog.=) 2))
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1818,7 +1716,6 @@ NOTICE: executing the command locally: SELECT id, author_id, title, word_count
|
|||
SELECT * FROM articles_range ar join authors_range au on (ar.author_id = au.id)
|
||||
WHERE ar.author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: issuing SELECT ar.id, ar.author_id, ar.title, ar.word_count, au.name, au.id FROM (public.articles_range_840012 ar JOIN public.authors_range_840008 au ON ((ar.author_id OPERATOR(pg_catalog.=) au.id))) WHERE (ar.author_id OPERATOR(pg_catalog.=) 1)
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
id | author_id | title | word_count | name | id
|
||||
|
@ -1829,7 +1726,6 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|||
SELECT * FROM articles_range ar join authors_range au on (ar.author_id = au.id)
|
||||
WHERE ar.author_id = 1 and au.id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
NOTICE: executing the command locally: SELECT ar.id, ar.author_id, ar.title, ar.word_count, au.name, au.id FROM ((SELECT NULL::bigint AS id, NULL::bigint AS author_id, NULL::character varying(20) AS title, NULL::integer AS word_count WHERE false) ar(id, author_id, title, word_count) JOIN (SELECT NULL::character varying(20) AS name, NULL::bigint AS id WHERE false) au(name, id) ON ((ar.author_id OPERATOR(pg_catalog.=) au.id))) WHERE ((ar.author_id OPERATOR(pg_catalog.=) 1) AND (au.id OPERATOR(pg_catalog.=) 2))
|
||||
id | author_id | title | word_count | name | id
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1916,7 +1812,6 @@ DETAIL: Creating dependency on merge taskId 20
|
|||
SELECT * FROM articles_range ar join authors_range au on (ar.id = au.id)
|
||||
WHERE ar.author_id = 1 and au.id < 10;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count | name | id
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1928,8 +1823,7 @@ DEBUG: Plan is router executable
|
|||
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
|
||||
WHERE ar.author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
id | author_id | title | word_count | name | id
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1966,7 +1860,6 @@ DETAIL: Creating dependency on merge taskId 8
|
|||
SELECT * FROM articles_range ar join authors_reference au on (ar.author_id = au.id)
|
||||
WHERE ar.author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count | name | id
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1975,7 +1868,6 @@ DEBUG: Plan is router executable
|
|||
SELECT * FROM articles_range ar join authors_reference au on (ar.author_id = au.id)
|
||||
WHERE ar.author_id = 1 or ar.author_id = 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count | name | id
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -2110,8 +2002,7 @@ SELECT
|
|||
WHERE
|
||||
author_id = 5;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 5
|
||||
DEBUG: query has a single distribution column value: 5
|
||||
c
|
||||
---------------------------------------------------------------------
|
||||
5
|
||||
|
@ -2154,8 +2045,7 @@ SELECT *
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -2174,8 +2064,7 @@ SELECT *
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -2194,8 +2083,7 @@ DECLARE test_cursor CURSOR FOR
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
FETCH test_cursor;
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2230,8 +2118,7 @@ COPY (
|
|||
WHERE author_id = 1
|
||||
ORDER BY id) TO STDOUT;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
1 1 arsenous 9572
|
||||
11 1 alamo 1347
|
||||
21 1 arcading 5890
|
||||
|
@ -2244,15 +2131,13 @@ CREATE TEMP TABLE temp_articles_hash as
|
|||
WHERE author_id = 1
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- router plannable queries may include filter for aggregates
|
||||
SELECT count(*), count(*) FILTER (WHERE id < 3)
|
||||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count | count
|
||||
---------------------------------------------------------------------
|
||||
5 | 1
|
||||
|
@ -2276,8 +2161,7 @@ PREPARE author_1_articles as
|
|||
ORDER BY 1;
|
||||
EXECUTE author_1_articles;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -2295,8 +2179,7 @@ PREPARE author_articles(int) as
|
|||
ORDER BY 1;
|
||||
EXECUTE author_articles(1);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -2322,8 +2205,7 @@ DEBUG: Creating router plan
|
|||
CONTEXT: SQL statement "SELECT MAX(id) FROM articles_hash ah
|
||||
WHERE author_id = 1"
|
||||
PL/pgSQL function author_articles_max_id() line 5 at SQL statement
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
CONTEXT: SQL statement "SELECT MAX(id) FROM articles_hash ah
|
||||
WHERE author_id = 1"
|
||||
PL/pgSQL function author_articles_max_id() line 5 at SQL statement
|
||||
|
@ -2349,8 +2231,7 @@ CONTEXT: SQL statement "SELECT ah.id, ah.word_count
|
|||
FROM articles_hash ah
|
||||
WHERE author_id = 1"
|
||||
PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
CONTEXT: SQL statement "SELECT ah.id, ah.word_count
|
||||
FROM articles_hash ah
|
||||
WHERE author_id = 1"
|
||||
|
@ -2368,8 +2249,7 @@ PL/pgSQL function author_articles_id_word_count() line 4 at RETURN QUERY
|
|||
CREATE MATERIALIZED VIEW mv_articles_hash_empty AS
|
||||
SELECT * FROM articles_hash WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
SELECT * FROM mv_articles_hash_empty;
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2404,8 +2284,7 @@ SELECT id
|
|||
WHERE author_id = 1
|
||||
ORDER BY 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -2418,16 +2297,14 @@ DETAIL: distribution column value: 1
|
|||
-- insert query is router plannable even under task-tracker
|
||||
INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814), (52, 1, 'second amateus', 2824);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
-- verify insert is successful (not router plannable and executable)
|
||||
SELECT id
|
||||
FROM articles_hash
|
||||
WHERE author_id = 1
|
||||
ORDER BY 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -2443,8 +2320,7 @@ DETAIL: distribution column value: 1
|
|||
UPDATE articles_hash SET id = id
|
||||
WHERE author_id = 1 AND title IN (SELECT name FROM authors_reference WHERE random() > 0.5);
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
SET client_min_messages to 'NOTICE';
|
||||
-- test that a connection failure marks placements invalid
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -143,8 +143,7 @@ WARNING: master_modify_multiple_shards is deprecated and will be removed in a f
|
|||
HINT: Run the command directly
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 15
|
||||
DEBUG: query has a single distribution column value: 15
|
||||
master_modify_multiple_shards
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
|
|
@ -417,8 +417,7 @@ SELECT *
|
|||
FROM articles
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -433,7 +432,6 @@ SELECT *
|
|||
FROM articles
|
||||
WHERE author_id = 1 OR author_id = 17;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -462,8 +460,7 @@ SELECT id as article_id, word_count * id as random_value
|
|||
FROM articles
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
article_id | random_value
|
||||
---------------------------------------------------------------------
|
||||
1 | 9572
|
||||
|
@ -480,8 +477,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 17277
|
||||
|
@ -496,8 +492,7 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
|
|||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 10
|
||||
DEBUG: query has a single distribution column value: 10
|
||||
first_author | second_word_count
|
||||
---------------------------------------------------------------------
|
||||
10 | 19519
|
||||
|
@ -511,8 +506,7 @@ SELECT *
|
|||
WHERE author_id = 1
|
||||
LIMIT 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -528,8 +522,7 @@ SELECT id
|
|||
GROUP BY id
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -546,8 +539,7 @@ SELECT avg(word_count)
|
|||
FROM articles
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
avg
|
||||
---------------------------------------------------------------------
|
||||
12356.400000000000
|
||||
|
@ -570,8 +562,7 @@ SELECT max(word_count) as max, min(word_count) as min,
|
|||
FROM articles
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 2
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
max | min | sum | cnt
|
||||
---------------------------------------------------------------------
|
||||
18185 | 2728 | 61782 | 5
|
||||
|
@ -632,24 +623,21 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
-- tablesample is supported
|
||||
SELECT * FROM articles TABLESAMPLE SYSTEM (0) WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM articles TABLESAMPLE BERNOULLI (0) WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM articles TABLESAMPLE SYSTEM (100) WHERE author_id = 1 ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -661,8 +649,7 @@ DETAIL: distribution column value: 1
|
|||
|
||||
SELECT * FROM articles TABLESAMPLE BERNOULLI (100) WHERE author_id = 1 ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -677,8 +664,7 @@ SET citus.enable_fast_path_router_planner TO true;
|
|||
SELECT * FROM articles TABLESAMPLE SYSTEM (0) WHERE author_id = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -686,8 +672,7 @@ DETAIL: distribution column value: 1
|
|||
SELECT * FROM articles TABLESAMPLE BERNOULLI (0) WHERE author_id = 1;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -695,8 +680,7 @@ DETAIL: distribution column value: 1
|
|||
SELECT * FROM articles TABLESAMPLE SYSTEM (100) WHERE author_id = 1 ORDER BY id;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
@ -709,8 +693,7 @@ DETAIL: distribution column value: 1
|
|||
SELECT * FROM articles TABLESAMPLE BERNOULLI (100) WHERE author_id = 1 ORDER BY id;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
id | author_id | title | word_count
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | arsenous | 9572
|
||||
|
|
|
@ -85,11 +85,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT l_orderkey FROM public.lineitem_subquery
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.l_orderkey FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(l_orderkey bigint) UNION ALL SELECT (1)::bigint AS int8
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.l_orderkey FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(l_orderkey bigint)) b
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
12001
|
||||
|
@ -107,11 +105,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT l_orderkey FROM public.line
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT l_partkey FROM public.lineitem_subquery
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.l_orderkey FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(l_orderkey bigint) UNION SELECT intermediate_result.l_partkey FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(l_partkey integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.l_orderkey FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(l_orderkey bigint)) b
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
14496
|
||||
|
|
|
@ -312,11 +312,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.user_buy_test_table
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION ALL SELECT users_ref_test_table.id FROM generate_series(1, 10) users_ref_test_table(id)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) subquery_1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
14
|
||||
|
@ -364,11 +362,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.user_buy_test_table
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION ALL SELECT users_ref_test_table.id FROM (SELECT 5 AS id) users_ref_test_table
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) subquery_1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
5
|
||||
|
@ -385,11 +381,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.user_buy_test_table
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT users_ref_test_table.id FROM public.users_ref_test_table UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) sub ORDER BY id DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id
|
||||
---------------------------------------------------------------------
|
||||
7
|
||||
|
@ -411,11 +405,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT user_id, (random() OPERATOR(pg_catalog.*) (0)::double precision) FROM (SELECT user_buy_test_table.user_id FROM public.user_buy_test_table) sub2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT sub1.id, (random() OPERATOR(pg_catalog.*) (0)::double precision) FROM (SELECT users_ref_test_table.id FROM public.users_ref_test_table) sub1 UNION SELECT intermediate_result.user_id, intermediate_result."?column?" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "?column?" double precision)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id, "?column?" FROM (SELECT intermediate_result.id, intermediate_result."?column?" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer, "?column?" double precision)) sub(id, "?column?") ORDER BY id DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | ?column?
|
||||
---------------------------------------------------------------------
|
||||
7 | 0
|
||||
|
|
|
@ -142,7 +142,6 @@ SET LOCAL citus.task_assignment_policy TO 'greedy';
|
|||
EXPLAIN (COSTS FALSE) SELECT * FROM task_assignment_reference_table;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -152,7 +151,6 @@ DEBUG: Plan is router executable
|
|||
EXPLAIN (COSTS FALSE) SELECT * FROM task_assignment_reference_table;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -163,7 +161,6 @@ SET LOCAL citus.task_assignment_policy TO 'first-replica';
|
|||
EXPLAIN (COSTS FALSE) SELECT * FROM task_assignment_reference_table;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
@ -173,7 +170,6 @@ DEBUG: Plan is router executable
|
|||
EXPLAIN (COSTS FALSE) SELECT * FROM task_assignment_reference_table;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus Adaptive)
|
||||
|
|
|
@ -982,14 +982,12 @@ DEBUG: generating subplan XXX_1 for subquery SELECT user_id, "time", value_1, v
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.>) 2)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.value_1, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, value_1 integer, value_2 integer, value_3 double precision, value_4 bigint) EXCEPT SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.value_1, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, value_1 integer, value_2 integer, value_3 double precision, value_4 bigint)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT user_id, "time", event_type, value_2, value_3, value_4 FROM public.events_table WHERE (value_3 OPERATOR(pg_catalog.>) (4)::double precision)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_5 for subquery SELECT user_id, "time", event_type, value_2, value_3, value_4 FROM public.events_table WHERE (value_2 OPERATOR(pg_catalog.>) 2)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_6 for subquery SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.event_type, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, event_type integer, value_2 integer, value_3 double precision, value_4 bigint) INTERSECT SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.event_type, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, event_type integer, value_2 integer, value_3 double precision, value_4 bigint)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT users_table_limited.user_id, users_table_limited."time", users_table_limited.value_1, users_table_limited.value_2, users_table_limited.value_3, users_table_limited.value_4, foo.user_id, foo."time", foo.event_type, foo.value_2, foo.value_3, foo.value_4, foo.user_id_1 AS user_id, foo.time_1 AS "time", foo.value_1, foo.value_2_1 AS value_2, foo.value_3_1 AS value_3, foo.value_4_1 AS value_4 FROM ((SELECT users_table_union.user_id, users_table_union."time", users_table_union.value_1, users_table_union.value_2, users_table_union.value_3, users_table_union.value_4 FROM (SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.value_1, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, value_1 integer, value_2 integer, value_3 double precision, value_4 bigint)) users_table_union) users_table_limited JOIN LATERAL (SELECT bar.user_id, bar."time", bar.event_type, bar.value_2, bar.value_3, bar.value_4, u2.user_id, u2."time", u2.value_1, u2.value_2, u2.value_3, u2.value_4 FROM ((SELECT events_table.user_id, events_table."time", events_table.event_type, events_table.value_2, events_table.value_3, events_table.value_4 FROM (SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.event_type, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_6'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, event_type integer, value_2 integer, value_3 double precision, value_4 bigint)) events_table WHERE (events_table.user_id OPERATOR(pg_catalog.=) users_table_limited.user_id)) bar LEFT JOIN public.users_table u2 ON ((u2.user_id OPERATOR(pg_catalog.=) bar.value_2)))) foo(user_id, "time", event_type, value_2, value_3, value_4, user_id_1, time_1, value_1, value_2_1, value_3_1, value_4_1) ON (true))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -1029,14 +1027,12 @@ DEBUG: generating subplan XXX_1 for subquery SELECT user_id, "time", value_1, v
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.>) 2)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.value_1, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, value_1 integer, value_2 integer, value_3 double precision, value_4 bigint) EXCEPT SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.value_1, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, value_1 integer, value_2 integer, value_3 double precision, value_4 bigint)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT user_id, "time", event_type, value_2, value_3, value_4 FROM public.events_table WHERE (value_3 OPERATOR(pg_catalog.>) (4)::double precision)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_5 for subquery SELECT user_id, "time", event_type, value_2, value_3, value_4 FROM public.events_table WHERE (value_2 OPERATOR(pg_catalog.>) 2)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_6 for subquery SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.event_type, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, event_type integer, value_2 integer, value_3 double precision, value_4 bigint) INTERSECT SELECT intermediate_result.user_id, intermediate_result."time", intermediate_result.event_type, intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4 FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, "time" timestamp without time zone, event_type integer, value_2 integer, value_3 double precision, value_4 bigint)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
ERROR: cannot pushdown the subquery
|
||||
|
@ -1068,14 +1064,12 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT table1.id, table1.tenant_id FROM non_colocated_subquery.table1 WHERE (table1.id OPERATOR(pg_catalog.<) 100)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE non_colocated_subquery.table2 SET id = 20 FROM (SELECT intermediate_result.id, intermediate_result.tenant_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer, tenant_id integer)) table1_view WHERE (table1_view.id OPERATOR(pg_catalog.=) table2.id)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
UPDATE table2_p1 SET id=20 FROM table1_view WHERE table1_view.id=table2_p1.id;
|
||||
DEBUG: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT table1.id, table1.tenant_id FROM non_colocated_subquery.table1 WHERE (table1.id OPERATOR(pg_catalog.<) 100)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE non_colocated_subquery.table2_p1 SET id = 20 FROM (SELECT intermediate_result.id, intermediate_result.tenant_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer, tenant_id integer)) table1_view WHERE (table1_view.id OPERATOR(pg_catalog.=) table2_p1.id)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
RESET client_min_messages;
|
||||
DROP FUNCTION explain_json_2(text);
|
||||
SET search_path TO 'public';
|
||||
|
|
|
@ -28,7 +28,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -40,7 +39,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT i.i FROM generate_series(0, 100) i(i) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -56,7 +54,6 @@ DEBUG: push down of limit count: 5
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_set_local.test LIMIT 5
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT i.i FROM generate_series(0, 100) i(i) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -66,7 +63,6 @@ DEBUG: Plan is router executable
|
|||
-- this doesn't require any recursive planning
|
||||
(SELECT a FROM ref) INTERSECT (SELECT i FROM generate_series(0, 100) i) ORDER BY 1 DESC;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a
|
||||
---------------------------------------------------------------------
|
||||
3
|
||||
|
@ -80,7 +76,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT (i.i OPERATOR(pg_catalog./) 0) FROM generate_series(0, 100) i(i) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
ERROR: division by zero
|
||||
-- we should be able to run set operations with generate series and local tables as well
|
||||
((SELECT x FROM local_test) UNION ALL (SELECT x FROM test)) INTERSECT (SELECT i FROM generate_series(0, 100) i) ORDER BY 1 DESC;
|
||||
|
@ -90,7 +85,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION ALL SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) INTERSECT SELECT i.i FROM generate_series(0, 100) i(i) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
4
|
||||
|
@ -108,7 +102,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_3 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION ALL SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
4
|
||||
|
@ -131,7 +124,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_3 for subquery SELECT user_id FROM (SELECT events_table.user_id FROM public.events_table) cte_2
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) INTERSECT SELECT i.i FROM generate_series(0, 100) i(i) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
user_id
|
||||
---------------------------------------------------------------------
|
||||
6
|
||||
|
@ -161,7 +153,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_set_local.
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM (SELECT test.x FROM recursive_set_local.test) cte_1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery (SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT cte_1.a FROM (SELECT ref.a FROM recursive_set_local.ref) cte_1) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) foo, recursive_set_local.test WHERE (test.y OPERATOR(pg_catalog.=) foo.x)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -189,11 +180,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_set_local.
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM (SELECT test.x FROM recursive_set_local.test) cte_1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery (SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT cte_1.a FROM (SELECT ref.a FROM recursive_set_local.ref) cte_1) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) foo, recursive_set_local.ref WHERE (ref.a OPERATOR(pg_catalog.=) foo.x)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -208,7 +197,6 @@ DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_set_local.
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT y FROM recursive_set_local.test c
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(y integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM recursive_set_local.test a WHERE (x OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.x FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(x integer))) ORDER BY x, y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -231,7 +219,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_3 for subquery SELECT y FROM recursive_set_local.test c
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(y integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(y integer)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM recursive_set_local.test a WHERE (x OPERATOR(pg_catalog.=) ANY (SELECT cte.x FROM (SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) cte)) ORDER BY x, y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
x | y
|
||||
|
@ -250,11 +237,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_set_loc
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_set_local.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, local_test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_set_local.local_test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y | y
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -267,7 +252,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: push down of limit count: 1
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_set_local.test ORDER BY x LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u LEFT JOIN recursive_set_local.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -281,7 +265,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: push down of limit count: 1
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_set_local.test ORDER BY x LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT test.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u RIGHT JOIN recursive_set_local.test USING (x)) ORDER BY test.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -299,7 +282,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: push down of limit count: 1
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_set_local.test ORDER BY x LIMIT 1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_set_local.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -314,16 +296,13 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_set_local.
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT x FROM (SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) foo WHERE (x OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_5 for subquery SELECT intermediate_result.x FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x FROM (SELECT intermediate_result.x FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) u ORDER BY x
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -371,7 +350,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_4 for subquery SELECT x FROM recursive_set_local.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT ((SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION ALL SELECT intermediate_result.x FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) INTERSECT SELECT i.i FROM generate_series(0, 100) i(i)) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
|
|
@ -34,7 +34,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -47,7 +46,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT ref.a, ref.b FROM recursive_union.ref ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -57,7 +55,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
(SELECT * FROM ref) UNION (SELECT * FROM ref) ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -72,7 +69,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION ALL SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -87,7 +83,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION ALL SELECT ref.a, ref.b FROM recursive_union.ref ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -98,7 +93,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
(SELECT * FROM ref) UNION ALL (SELECT * FROM ref) ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -115,7 +109,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -128,7 +121,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT ref.a, ref.b FROM recursive_union.ref ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -136,7 +128,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
(SELECT * FROM ref) INTERSECT (SELECT * FROM ref) ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -151,7 +142,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT ALL SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -164,7 +154,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT ALL SELECT ref.a, ref.b FROM recursive_union.ref ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -172,7 +161,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
(SELECT * FROM ref) INTERSECT ALL (SELECT * FROM ref) ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -187,7 +175,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -198,7 +185,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT ref.a, ref.b FROM recursive_union.ref ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -206,7 +192,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
(SELECT * FROM ref) EXCEPT (SELECT * FROM ref) ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -219,7 +204,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT ALL SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -230,7 +214,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT ALL SELECT ref.a, ref.b FROM recursive_union.ref ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -238,7 +221,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
(SELECT * FROM ref) EXCEPT ALL (SELECT * FROM ref) ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -261,7 +243,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT test.x, ref.a FROM (recursive_union.test LEFT JOIN recursive_union.ref ON ((test.x OPERATOR(pg_catalog.=) ref.a)))
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT ref.a, ref.b FROM recursive_union.ref) UNION ALL SELECT s.s, s.s FROM generate_series(1, 10) s(s)) EXCEPT SELECT 1, 1) UNION SELECT intermediate_result.x, intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, a integer) ORDER BY 1, 2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 |
|
||||
|
@ -292,11 +273,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y, x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.y, intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer, x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -308,11 +287,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT ref.a, ref.b FROM recursive_union.ref
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -322,7 +299,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT * FROM ((SELECT * FROM ref) UNION (SELECT * FROM ref)) u ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -354,11 +330,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION ALL SELECT ref.a, ref.b FROM recursive_union.ref
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -369,7 +343,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT * FROM ((SELECT * FROM ref) UNION ALL (SELECT * FROM ref)) u ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -385,11 +358,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -403,11 +374,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y, x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.y, intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer, x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -419,11 +388,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT ref.a, ref.b FROM recursive_union.ref
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -431,7 +398,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT * FROM ((SELECT * FROM ref) INTERSECT (SELECT * FROM ref)) u ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -445,11 +411,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -461,11 +425,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y, x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT intermediate_result.y, intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer, x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -475,11 +437,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT ref.a, ref.b FROM recursive_union.ref
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -487,7 +447,6 @@ DEBUG: Plan is router executable
|
|||
|
||||
SELECT * FROM ((SELECT * FROM ref) EXCEPT (SELECT * FROM ref)) u ORDER BY 1,2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -520,7 +479,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_union.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -538,7 +496,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION ALL SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u LEFT JOIN recursive_union.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -552,7 +509,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y, x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.y, intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer, x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_union.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -569,7 +525,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT 1, 1 FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result."?column?", intermediate_result."?column?_1" AS "?column?" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer, "?column?_1" integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_union.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -596,11 +551,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test ORDER BY x
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN (SELECT x_1.x FROM generate_series(1, 10) x_1(x)) x USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -635,11 +588,9 @@ DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT y, x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.y, intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(y integer, x integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) foo WHERE (x OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(y integer))) ORDER BY x
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -654,13 +605,11 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y, x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.y, intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer, x integer)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) foo WHERE (x OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.y FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(y integer))) ORDER BY x
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -679,7 +628,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y, rnk FROM (SELECT test.x, test.y, rank() OVER my_win AS rnk FROM recursive_union.test WINDOW my_win AS (PARTITION BY test.x ORDER BY test.y DESC)) bar
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y, intermediate_result.rnk FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer, rnk bigint) UNION SELECT intermediate_result.x, intermediate_result.y, intermediate_result.rnk FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer, rnk bigint) ORDER BY 1 DESC, 2 DESC, 3 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y | rnk
|
||||
---------------------------------------------------------------------
|
||||
2 | 2 | 1
|
||||
|
@ -698,7 +646,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y, rank() OVER my_win AS rnk FROM recursive_union.test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.x, foo.y, foo.rnk FROM (SELECT intermediate_result.x, intermediate_result.y, intermediate_result.rnk FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer, rnk bigint)) foo UNION SELECT bar.x, bar.y, bar.rnk FROM (SELECT intermediate_result.x, intermediate_result.y, intermediate_result.rnk FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer, rnk bigint)) bar ORDER BY 1 DESC, 2 DESC, 3 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y | rnk
|
||||
---------------------------------------------------------------------
|
||||
2 | 2 | 1
|
||||
|
@ -714,7 +661,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) EXCEPT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_union.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -731,7 +677,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) INTERSECT SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT u.x, u.y, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u LEFT JOIN recursive_union.test USING (x)) ORDER BY u.x, u.y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -743,16 +688,13 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT a, b FROM recursive_union.ref WHERE (a OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -778,7 +720,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_union.test c WHERE (y OPERATOR(pg_catalog.=) 2)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT b.x FROM recursive_union.test b WHERE (b.y OPERATOR(pg_catalog.=) 1) UNION SELECT c.x FROM recursive_union.test c WHERE (c.y OPERATOR(pg_catalog.=) 2)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM recursive_union.test a WHERE (NOT (x OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer)))) ORDER BY x, y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -794,7 +735,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x FROM recursive_union.test
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y FROM recursive_union.test c
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM recursive_union.test a WHERE (x OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer))) ORDER BY x, y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -827,11 +767,9 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) ORDER BY 1, 2 LIMIT 5
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT test.x, test.y FROM recursive_union.test UNION SELECT test.x, test.y FROM recursive_union.test ORDER BY 1, 2 LIMIT 5
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) foo ORDER BY x DESC LIMIT 3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
2 | 2
|
||||
|
@ -846,11 +784,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT y FROM recursive_u
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT DISTINCT x FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(DISTINCT x) AS count FROM (SELECT intermediate_result.x FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer)) t(x) ORDER BY (count(DISTINCT x))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -863,11 +799,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT count(DISTINCT x) AS count
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT count(DISTINCT y) AS count FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint) UNION SELECT intermediate_result.count FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(DISTINCT x) AS count FROM (SELECT intermediate_result.count FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) t(x) ORDER BY (count(DISTINCT x))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -881,11 +815,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT avg(DISTINCT y) AS avg FROM
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT avg(DISTINCT y) AS avg FROM recursive_union.test GROUP BY x
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.avg FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric) UNION SELECT intermediate_result.avg FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT avg(DISTINCT x) AS avg FROM (SELECT intermediate_result.avg FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) t(x) ORDER BY (avg(DISTINCT x))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
avg
|
||||
---------------------------------------------------------------------
|
||||
1.50000000000000000000
|
||||
|
@ -937,7 +869,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -979,7 +910,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT x FROM recursive_union.test
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.x FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT intermediate_result.x FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer) ORDER BY 1 DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -996,11 +926,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT y FROM recursive_union.test
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(y integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT y FROM (SELECT intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(y integer)) set_view_recursive ORDER BY y DESC
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
y
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -1026,7 +954,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT 1, 1 FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result."?column?", intermediate_result."?column?_1" AS "?column?" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer, "?column?_1" integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT u.x, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_union.test USING (x)) ORDER BY u.x, test.y) set_view_recursive_second ORDER BY x, y
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
|
@ -1044,14 +971,12 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT 1, 1 FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result."?column?", intermediate_result."?column?_1" AS "?column?" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer, "?column?_1" integer)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_4 for subquery SELECT y FROM recursive_union.test
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_5 for subquery SELECT y FROM recursive_union.test
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_6 for subquery SELECT intermediate_result.y FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(y integer) UNION SELECT intermediate_result.y FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(y integer)
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_7 for subquery SELECT x FROM (SELECT u.x, test.y FROM ((SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u JOIN recursive_union.test USING (x)) ORDER BY u.x, test.y) set_view_recursive_second
|
||||
|
@ -1059,7 +984,6 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: generating subplan XXX_8 for subquery SELECT x FROM (SELECT test.x FROM recursive_union.test UNION SELECT test.x FROM recursive_union.test) set_view_pushdown
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: (SELECT intermediate_result.x FROM read_intermediate_result('XXX_7'::text, 'binary'::citus_copy_format) intermediate_result(x integer) INTERSECT SELECT set_view_recursive.y FROM (SELECT intermediate_result.y FROM read_intermediate_result('XXX_6'::text, 'binary'::citus_copy_format) intermediate_result(y integer)) set_view_recursive) EXCEPT SELECT intermediate_result.x FROM read_intermediate_result('XXX_8'::text, 'binary'::citus_copy_format) intermediate_result(x integer)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1072,11 +996,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test_not_colocated
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -1090,11 +1012,9 @@ DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM recursive_union.t
|
|||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT x, y FROM recursive_union.test_not_colocated
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer) UNION ALL SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT x, y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) u ORDER BY x, y
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
x | y
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
|
|
@ -894,8 +894,7 @@ SET client_min_messages TO debug2;
|
|||
WITH mb AS (UPDATE modify_table SET val = 3 WHERE id = 3 RETURNING NULL) INSERT INTO modify_table WITH ma AS (SELECT * FROM modify_table LIMIT 10) SELECT count(*) FROM mb;
|
||||
DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 3
|
||||
DEBUG: query has a single distribution column value: 3
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- not pushed down due to volatile
|
||||
WITH ma AS (SELECT count(*) FROM modify_table where id = 1), mu AS (WITH allref AS (SELECT random() a FROM modify_table limit 4) UPDATE modify_table SET val = 3 WHERE id = 1 AND val IN (SELECT a FROM allref) RETURNING id+1) SELECT count(*) FROM mu, ma;
|
||||
|
@ -908,16 +907,13 @@ DEBUG: Router planner cannot handle multi-shard select queries
|
|||
DEBUG: push down of limit count: 4
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE with_modifying.modify_table SET val = 3 WHERE ((id OPERATOR(pg_catalog.=) 1) AND ((val)::double precision OPERATOR(pg_catalog.=) ANY (SELECT allref.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a double precision)) allref))) RETURNING (id OPERATOR(pg_catalog.+) 1)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT count(*) AS count FROM with_modifying.modify_table WHERE (id OPERATOR(pg_catalog.=) 1)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result."?column?" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)) mu("?column?"), (SELECT intermediate_result.count FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) ma
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -930,14 +926,11 @@ DEBUG: Router planner doesn't support VOLATILE functions in common table expres
|
|||
DEBUG: generating subplan XXX_1 for CTE allref: SELECT random() AS a FROM with_modifying.anchor_table
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE with_modifying.modify_table SET val = 3 WHERE ((id OPERATOR(pg_catalog.=) 1) AND ((val)::double precision OPERATOR(pg_catalog.=) ANY (SELECT allref.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a double precision)) allref))) RETURNING (id OPERATOR(pg_catalog.+) 1)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result."?column?" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)) mu("?column?")
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -946,8 +939,7 @@ DEBUG: Plan is router executable
|
|||
-- pushed down
|
||||
WITH mu AS (WITH allref AS (SELECT id a FROM anchor_table) UPDATE modify_table SET val = 3 WHERE id = 1 AND val IN (SELECT a FROM allref) RETURNING id+1) SELECT count(*) FROM mu;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -956,8 +948,7 @@ DETAIL: distribution column value: 1
|
|||
-- pushed down and stable function evaluated
|
||||
WITH mu AS (WITH allref AS (SELECT now() a FROM anchor_table) UPDATE modify_table SET val = 3 WHERE id = 1 AND now() IN (SELECT a FROM allref) RETURNING id+1) SELECT count(*) FROM mu;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DETAIL: distribution column value: 1
|
||||
DEBUG: query has a single distribution column value: 1
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
|
|
@ -214,6 +214,7 @@ INSERT INTO ref_table SELECT *, * FROM generate_series(1, 100);
|
|||
SELECT COUNT(*) FROM test JOIN ref_table USING(x);
|
||||
ROLLBACK;
|
||||
|
||||
\set VERBOSITY terse
|
||||
DROP TABLE ref_table;
|
||||
|
||||
DELETE FROM test;
|
||||
|
|
|
@ -712,6 +712,7 @@ INTERSECT
|
|||
-- CTEs with where false
|
||||
-- terse because distribution column inference varies between pg11 & pg12
|
||||
\set VERBOSITY terse
|
||||
RESET client_min_messages;
|
||||
|
||||
WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1),
|
||||
id_title AS (SELECT id, title from articles_hash WHERE author_id = 1 and 1=0)
|
||||
|
@ -721,6 +722,7 @@ WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1)
|
|||
id_title AS (SELECT id, title from articles_hash WHERE author_id = 1)
|
||||
SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id and 1=0;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
\set VERBOSITY DEFAULT
|
||||
|
||||
WITH RECURSIVE hierarchy as (
|
||||
|
|
Loading…
Reference in New Issue