From 8d929d3bf8aa20490cf4bb3117f129337c91fe00 Mon Sep 17 00:00:00 2001 From: eaydingol <60466783+eaydingol@users.noreply.github.com> Date: Mon, 18 Aug 2025 14:03:44 +0300 Subject: [PATCH] Push down recurring outer joins when possible (#7973) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit DESCRIPTION: Adds support for pushing down LEFT/RIGHT outer joins having a reference table in the outer side and a distributed table on the inner side (e.g., LEFT JOIN ) Partially addresses #6546 1) `` LEFT JOIN `` 2) `` RIGHT JOIN `` Previously, for outer joins of types (1) and (2), the distributed side was computed recursively. This was necessary because, when the inner side of a recurring outer join is a distributed table, it is not possible to directly distribute the join; the preserved (outer and recurring) side may generate rows with join keys that hash to different shards. To implement distributed planning while maintaining consistency with global execution semantics, this PR restricts the outer side only to those partition key values that route to the selected shard during distributed shard query computation. This method is employed )when the following criteria are met: (recursive planning applied otherwise) - The join type is (1) or (2) (lateral joins are not supported). - The outer side is a reference table. - The outer join qualifications include an equality condition between the partition column of a distributed table and the recurring table. - The join is not part of a chained join. - The “enable_recurring_outer_join_pushdown” GUC is enabled (default is on). --------- Co-authored-by: ebruaydingol Co-authored-by: Onur Tirtir --- .../distributed/metadata/metadata_cache.c | 26 + .../distributed/planner/deparse_shard_query.c | 250 +++++ .../distributed/planner/distributed_planner.c | 14 +- .../planner/insert_select_planner.c | 3 +- .../distributed/planner/merge_planner.c | 3 +- .../planner/multi_physical_planner.c | 98 +- .../planner/multi_router_planner.c | 3 +- .../planner/query_pushdown_planning.c | 74 +- .../distributed/planner/recursive_planning.c | 418 +++++++- src/backend/distributed/shared_library_init.c | 15 + src/include/distributed/deparse_shard_query.h | 7 + src/include/distributed/distributed_planner.h | 12 + src/include/distributed/metadata_cache.h | 1 + .../distributed/query_pushdown_planning.h | 11 +- src/include/distributed/recursive_planning.h | 14 +- src/test/regress/citus_tests/run_test.py | 14 + .../regress/expected/multi_insert_select.out | 41 +- .../multi_level_recursive_queries.out | 14 +- .../expected/multi_outer_join_reference.out | 6 +- ...ulti_subquery_complex_reference_clause.out | 10 +- ...lti_subquery_in_where_reference_clause.out | 14 +- src/test/regress/expected/pg17.out | 35 +- src/test/regress/expected/pg17_0.out | 46 +- .../expected/query_single_shard_table.out | 31 +- .../expected/recurring_join_pushdown.out | 983 ++++++++++++++++++ .../regress/expected/recurring_outer_join.out | 308 +++--- src/test/regress/expected/stat_counters.out | 2 +- src/test/regress/multi_1_schedule | 1 + src/test/regress/sql/multi_insert_select.sql | 11 +- .../sql/multi_outer_join_reference.sql | 4 +- ...ulti_subquery_complex_reference_clause.sql | 3 +- ...lti_subquery_in_where_reference_clause.sql | 6 +- src/test/regress/sql/pg17.sql | 20 +- .../regress/sql/recurring_join_pushdown.sql | 142 +++ src/test/regress/sql/recurring_outer_join.sql | 82 +- src/test/regress/sql/stat_counters.sql | 2 +- 36 files changed, 2377 insertions(+), 347 deletions(-) create mode 100644 src/test/regress/expected/recurring_join_pushdown.out create mode 100644 src/test/regress/sql/recurring_join_pushdown.sql diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 8fd39d3b7..51eaa5c93 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -521,6 +521,32 @@ IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEntry, CitusTableType tabl } +/* + * IsFirstShard returns true if the given shardId is the first shard. + */ +bool +IsFirstShard(CitusTableCacheEntry *tableEntry, uint64 shardId) +{ + if (tableEntry == NULL || tableEntry->sortedShardIntervalArray == NULL) + { + return false; + } + if (tableEntry->sortedShardIntervalArray[0]->shardId == INVALID_SHARD_ID) + { + return false; + } + + if (shardId == tableEntry->sortedShardIntervalArray[0]->shardId) + { + return true; + } + else + { + return false; + } +} + + /* * HasDistributionKey returns true if given Citus table has a distribution key. */ diff --git a/src/backend/distributed/planner/deparse_shard_query.c b/src/backend/distributed/planner/deparse_shard_query.c index 1aedbac17..b22bb8028 100644 --- a/src/backend/distributed/planner/deparse_shard_query.c +++ b/src/backend/distributed/planner/deparse_shard_query.c @@ -16,6 +16,8 @@ #include "access/heapam.h" #include "access/htup_details.h" #include "catalog/pg_constraint.h" +#include "catalog/pg_namespace.h" +#include "catalog/pg_operator.h" #include "lib/stringinfo.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" @@ -38,6 +40,8 @@ #include "distributed/metadata_cache.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_router_planner.h" +#include "distributed/query_utils.h" +#include "distributed/recursive_planning.h" #include "distributed/shard_utils.h" #include "distributed/stats/stat_tenants.h" #include "distributed/version_compat.h" @@ -204,6 +208,252 @@ UpdateTaskQueryString(Query *query, Task *task) } +/* + * CreateQualsForShardInterval creates the necessary qual conditions over the + * given attnum and rtindex for the given shard interval. + */ +Node * +CreateQualsForShardInterval(RelationShard *relationShard, int attnum, int rtindex) +{ + uint64 shardId = relationShard->shardId; + Oid relationId = relationShard->relationId; + + CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); + Var *partitionColumnVar = cacheEntry->partitionColumn; + + /* + * Add constraints for the relation identified by rtindex, specifically on its column at attnum. + * Create a Var node representing this column, which will be used to compare against the bounds + * from the partition column of shard interval. + */ + + Var *outerTablePartitionColumnVar = makeVar( + rtindex, attnum, partitionColumnVar->vartype, + partitionColumnVar->vartypmod, + partitionColumnVar->varcollid, + 0); + + bool isFirstShard = IsFirstShard(cacheEntry, shardId); + + /* load the interval for the shard and create constant nodes for the upper/lower bounds */ + ShardInterval *shardInterval = LoadShardInterval(shardId); + Const *constNodeLowerBound = makeConst(INT4OID, -1, InvalidOid, sizeof(int32), + shardInterval->minValue, false, true); + Const *constNodeUpperBound = makeConst(INT4OID, -1, InvalidOid, sizeof(int32), + shardInterval->maxValue, false, true); + Const *constNodeZero = makeConst(INT4OID, -1, InvalidOid, sizeof(int32), + Int32GetDatum(0), false, true); + + /* create a function expression node for the hash partition column */ + FuncExpr *hashFunction = makeNode(FuncExpr); + hashFunction->funcid = cacheEntry->hashFunction->fn_oid; + hashFunction->args = list_make1(outerTablePartitionColumnVar); + hashFunction->funcresulttype = get_func_rettype(cacheEntry->hashFunction->fn_oid); + hashFunction->funcretset = false; + + /* create a function expression for the lower bound of the shard interval */ + Oid resultTypeOid = get_func_rettype( + cacheEntry->shardIntervalCompareFunction->fn_oid); + FuncExpr *lowerBoundFuncExpr = makeNode(FuncExpr); + lowerBoundFuncExpr->funcid = cacheEntry->shardIntervalCompareFunction->fn_oid; + lowerBoundFuncExpr->args = list_make2((Node *) constNodeLowerBound, + (Node *) hashFunction); + lowerBoundFuncExpr->funcresulttype = resultTypeOid; + lowerBoundFuncExpr->funcretset = false; + + Oid lessThan = GetSysCacheOid(OPERNAMENSP, Anum_pg_operator_oid, CStringGetDatum("<"), + resultTypeOid, resultTypeOid, ObjectIdGetDatum( + PG_CATALOG_NAMESPACE)); + + /* + * Finally, check if the comparison result is less than 0, i.e., + * shardInterval->minValue < hash(partitionColumn) + * See SearchCachedShardInterval for the behavior at the boundaries. + */ + Expr *lowerBoundExpr = make_opclause(lessThan, BOOLOID, false, + (Expr *) lowerBoundFuncExpr, + (Expr *) constNodeZero, InvalidOid, InvalidOid); + + /* create a function expression for the upper bound of the shard interval */ + FuncExpr *upperBoundFuncExpr = makeNode(FuncExpr); + upperBoundFuncExpr->funcid = cacheEntry->shardIntervalCompareFunction->fn_oid; + upperBoundFuncExpr->args = list_make2((Node *) hashFunction, + (Expr *) constNodeUpperBound); + upperBoundFuncExpr->funcresulttype = resultTypeOid; + upperBoundFuncExpr->funcretset = false; + + Oid lessThanOrEqualTo = GetSysCacheOid(OPERNAMENSP, Anum_pg_operator_oid, + CStringGetDatum("<="), + resultTypeOid, resultTypeOid, + ObjectIdGetDatum(PG_CATALOG_NAMESPACE)); + + + /* + * Finally, check if the comparison result is less than or equal to 0, i.e., + * hash(partitionColumn) <= shardInterval->maxValue + * See SearchCachedShardInterval for the behavior at the boundaries. + */ + Expr *upperBoundExpr = make_opclause(lessThanOrEqualTo, BOOLOID, false, + (Expr *) upperBoundFuncExpr, + (Expr *) constNodeZero, InvalidOid, InvalidOid); + + + /* create a node for both upper and lower bound */ + Node *shardIntervalBoundQuals = make_and_qual((Node *) lowerBoundExpr, + (Node *) upperBoundExpr); + + /* + * Add a null test for the partition column for the first shard. + * This is because we need to include the null values in exactly one of the shard queries. + * The null test is added as an OR clause to the existing AND clause. + */ + if (isFirstShard) + { + /* null test for the first shard */ + NullTest *nullTest = makeNode(NullTest); + nullTest->nulltesttype = IS_NULL; /* Check for IS NULL */ + nullTest->arg = (Expr *) outerTablePartitionColumnVar; /* The variable to check */ + nullTest->argisrow = false; + shardIntervalBoundQuals = (Node *) make_orclause(list_make2(nullTest, + shardIntervalBoundQuals)); + } + return shardIntervalBoundQuals; +} + + +/* + * UpdateWhereClauseToPushdownRecurringOuterJoinWalker walks over the query tree and + * updates the WHERE clause for outer joins satisfying feasibility conditions. + */ +bool +UpdateWhereClauseToPushdownRecurringOuterJoinWalker(Node *node, List *relationShardList) +{ + if (node == NULL) + { + return false; + } + + if (IsA(node, Query)) + { + UpdateWhereClauseToPushdownRecurringOuterJoin((Query *) node, relationShardList); + return query_tree_walker((Query *) node, + UpdateWhereClauseToPushdownRecurringOuterJoinWalker, + relationShardList, QTW_EXAMINE_RTES_BEFORE); + } + + if (!IsA(node, RangeTblEntry)) + { + return expression_tree_walker(node, + UpdateWhereClauseToPushdownRecurringOuterJoinWalker, + relationShardList); + } + + return false; +} + + +/* + * UpdateWhereClauseToPushdownRecurringOuterJoin + * + * Inject shard interval predicates into the query WHERE clause for certain + * outer joins to make the join semantically correct when distributed. + * + * Why this is needed: + * When an inner side of an OUTER JOIN is a distributed table that has been + * routed to a single shard, we cannot simply replace the RTE with the shard + * name and rely on implicit pruning: the preserved (outer) side could still + * produce rows whose join keys would hash to other shards. To keep results + * consistent with the global execution semantics we restrict the preserved + * (outer) side to only those partition key values that would route to the + * chosen shard (plus NULLs, which are assigned to exactly one shard). + * + * What the function does: + * 1. Iterate over the top-level jointree->fromlist. + * 2. For each JoinExpr call CanPushdownRecurringOuterJoinExtended() which: + * - Verifies shape / join type is eligible. + * - Returns: + * outerRtIndex : RT index whose column we will constrain, + * outerRte / innerRte, + * attnum : attribute number (partition column) on outer side. + * This is compared to partition column of innerRte. + * 3. Find the RelationShard for the inner distributed table (innerRte->relid) + * in relationShardList; skip if absent (no fixed shard chosen). + * 4. Build the shard qualification with CreateQualsForShardInterval(): + * (minValue < hash(partcol) AND hash(partcol) <= maxValue) + * and, for the first shard only, OR (partcol IS NULL). + * The Var refers to (outerRtIndex, attnum) so the restriction applies to + * the preserved outer input. + * 5. AND the new quals into jointree->quals (creating it if NULL). + * + * The function does not return anything, it modifies the query in place. + */ +void +UpdateWhereClauseToPushdownRecurringOuterJoin(Query *query, List *relationShardList) +{ + if (query == NULL) + { + return; + } + + FromExpr *fromExpr = query->jointree; + if (fromExpr == NULL || fromExpr->fromlist == NIL) + { + return; + } + + ListCell *fromExprCell; + foreach(fromExprCell, fromExpr->fromlist) + { + Node *fromItem = (Node *) lfirst(fromExprCell); + if (!IsA(fromItem, JoinExpr)) + { + continue; + } + JoinExpr *joinExpr = (JoinExpr *) fromItem; + + /* + * We will check if we need to add constraints to the WHERE clause. + */ + RangeTblEntry *innerRte = NULL; + RangeTblEntry *outerRte = NULL; + int outerRtIndex = -1; + int attnum; + if (!CanPushdownRecurringOuterJoinExtended(joinExpr, query, &outerRtIndex, + &outerRte, &innerRte, &attnum)) + { + continue; + } + + if (attnum == InvalidAttrNumber) + { + continue; + } + ereport(DEBUG5, (errmsg( + "Distributed table from the inner part of the outer join: %s.", + innerRte->eref->aliasname))); + + RelationShard *relationShard = FindRelationShard(innerRte->relid, + relationShardList); + + if (relationShard == NULL || relationShard->shardId == INVALID_SHARD_ID) + { + continue; + } + + Node *shardIntervalBoundQuals = CreateQualsForShardInterval(relationShard, attnum, + outerRtIndex); + if (fromExpr->quals == NULL) + { + fromExpr->quals = (Node *) shardIntervalBoundQuals; + } + else + { + fromExpr->quals = make_and_qual(fromExpr->quals, shardIntervalBoundQuals); + } + } +} + + /* * UpdateRelationToShardNames walks over the query tree and appends shard ids to * relations. It uses unique identity value to establish connection between a diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index e22296ec7..098aaeb13 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -75,17 +75,6 @@ #endif -/* RouterPlanType is used to determine the router plan to invoke */ -typedef enum RouterPlanType -{ - INSERT_SELECT_INTO_CITUS_TABLE, - INSERT_SELECT_INTO_LOCAL_TABLE, - DML_QUERY, - SELECT_QUERY, - MERGE_QUERY, - REPLAN_WITH_BOUND_PARAMETERS -} RouterPlanType; - static List *plannerRestrictionContextList = NIL; int MultiTaskQueryLogLevel = CITUS_LOG_LEVEL_OFF; /* multi-task query log level */ static uint64 NextPlanId = 1; @@ -1097,7 +1086,8 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina * set_plan_references>add_rtes_to_flat_rtable>add_rte_to_flat_rtable. */ List *subPlanList = GenerateSubplansForSubqueriesAndCTEs(planId, originalQuery, - plannerRestrictionContext); + plannerRestrictionContext, + routerPlan); /* * If subqueries were recursively planned then we need to replan the query diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 3bf0bb327..1cf996b77 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -766,7 +766,8 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte, { /* first apply toplevel pushdown checks to SELECT query */ error = - DeferErrorIfUnsupportedSubqueryPushdown(subquery, plannerRestrictionContext); + DeferErrorIfUnsupportedSubqueryPushdown(subquery, plannerRestrictionContext, + true); if (error) { return error; diff --git a/src/backend/distributed/planner/merge_planner.c b/src/backend/distributed/planner/merge_planner.c index 6f3993794..b1c441f92 100644 --- a/src/backend/distributed/planner/merge_planner.c +++ b/src/backend/distributed/planner/merge_planner.c @@ -1149,7 +1149,8 @@ DeferErrorIfRoutableMergeNotSupported(Query *query, List *rangeTableList, { deferredError = DeferErrorIfUnsupportedSubqueryPushdown(query, - plannerRestrictionContext); + plannerRestrictionContext, + true); if (deferredError) { ereport(DEBUG1, (errmsg("Sub-query is not pushable, try repartitioning"))); diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index e0f4c4332..2c0a17d16 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -167,13 +167,16 @@ static uint32 HashPartitionCount(void); /* Local functions forward declarations for task list creation and helper functions */ static Job * BuildJobTreeTaskList(Job *jobTree, PlannerRestrictionContext *plannerRestrictionContext); -static bool IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction); +static bool IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction, + Bitmapset *distributedTables, + bool *outerPartHasDistributedTable); static void ErrorIfUnsupportedShardDistribution(Query *query); static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, RelationRestrictionContext *restrictionContext, uint32 taskId, TaskType taskType, bool modifyRequiresCoordinatorEvaluation, + bool updateQualsForOuterJoin, DeferredErrorMessage **planningError); static List * SqlTaskList(Job *job); static bool DependsOnHashPartitionJob(Job *job); @@ -2199,6 +2202,7 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, int minShardOffset = INT_MAX; int prevShardCount = 0; Bitmapset *taskRequiredForShardIndex = NULL; + Bitmapset *distributedTableIndex = NULL; /* error if shards are not co-partitioned */ ErrorIfUnsupportedShardDistribution(query); @@ -2215,8 +2219,12 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, RelationRestriction *relationRestriction = NULL; List *prunedShardList = NULL; - forboth_ptr(prunedShardList, prunedRelationShardList, - relationRestriction, relationRestrictionContext->relationRestrictionList) + /* First loop, gather the indexes of distributed tables + * this is required to decide whether we can skip shards + * from inner tables of outer joins + */ + foreach_declared_ptr(relationRestriction, + relationRestrictionContext->relationRestrictionList) { Oid relationId = relationRestriction->relationId; @@ -2237,6 +2245,24 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, } prevShardCount = cacheEntry->shardIntervalArrayLength; + distributedTableIndex = bms_add_member(distributedTableIndex, + relationRestriction->index); + } + + /* In the second loop, populate taskRequiredForShardIndex */ + bool updateQualsForOuterJoin = false; + bool outerPartHasDistributedTable = false; + forboth_ptr(prunedShardList, prunedRelationShardList, + relationRestriction, relationRestrictionContext->relationRestrictionList) + { + Oid relationId = relationRestriction->relationId; + + CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); + if (!HasDistributionKeyCacheEntry(cacheEntry)) + { + continue; + } + /* * For left joins we don't care about the shards pruned for the right hand side. * If the right hand side would prune to a smaller set we should still send it to @@ -2244,12 +2270,25 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, * the left hand side we don't have to send the query to any shard that is not * matching anything on the left hand side. * - * Instead we will simply skip any RelationRestriction if it is an OUTER join and - * the table is part of the non-outer side of the join. + * Instead we will simply skip any RelationRestriction if it is an OUTER join, + * the table is part of the non-outer side of the join and the outer side has a + * distributed table. */ - if (IsInnerTableOfOuterJoin(relationRestriction)) + if (IsInnerTableOfOuterJoin(relationRestriction, distributedTableIndex, + &outerPartHasDistributedTable)) { - continue; + if (outerPartHasDistributedTable) + { + /* we can skip the shards from this relation restriction */ + continue; + } + else + { + /* The outer part does not include distributed tables, we can not skip shards. + * Also, we will possibly update the quals of the outer relation for recurring join push down, mark here. + */ + updateQualsForOuterJoin = true; + } } ShardInterval *shardInterval = NULL; @@ -2263,6 +2302,22 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, } } + /* + * We might fail to find outer joins from the relationRestrictionContext + * when the original query has CTEs. In order to ensure that we always mark + * the outer joins correctly and compute additional quals when necessary, + * check the task query as well. + */ + if (!updateQualsForOuterJoin && FindNodeMatchingCheckFunction((Node *) query, + IsOuterJoinExpr)) + { + /* + * We have an outer join, so assume "might" need to update quals. + * See the usage of this flag in QueryPushdownTaskCreate(). + */ + updateQualsForOuterJoin = true; + } + /* * We keep track of minShardOffset to skip over a potentially big amount of pruned * shards. However, we need to start at minShardOffset - 1 to make sure we don't @@ -2282,6 +2337,7 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, taskIdIndex, taskType, modifyRequiresCoordinatorEvaluation, + updateQualsForOuterJoin, planningError); if (*planningError != NULL) { @@ -2315,10 +2371,13 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, * a) in an outer join * b) on the inner part of said join * - * The function returns true only if both conditions above hold true + * The function also sets outerPartHasDistributedTable if the outer part + * of the corresponding join has a distributed table. */ static bool -IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction) +IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction, + Bitmapset *distributedTables, + bool *outerPartHasDistributedTable) { RestrictInfo *joinInfo = NULL; foreach_declared_ptr(joinInfo, relationRestriction->relOptInfo->joininfo) @@ -2339,6 +2398,11 @@ IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction) if (!isInOuter) { /* this table is joined in the inner part of an outer join */ + /* set if the outer part has a distributed relation */ + *outerPartHasDistributedTable = bms_overlap(joinInfo->outer_relids, + distributedTables); + + /* this is an inner table of an outer join */ return true; } } @@ -2455,6 +2519,7 @@ static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, RelationRestrictionContext *restrictionContext, uint32 taskId, TaskType taskType, bool modifyRequiresCoordinatorEvaluation, + bool updateQualsForOuterJoin, DeferredErrorMessage **planningError) { Query *taskQuery = copyObject(originalQuery); @@ -2559,6 +2624,21 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, (List *) taskQuery->jointree->quals); } + if (updateQualsForOuterJoin) + { + /* + * QueryPushdownSqlTaskList() might set this when it detects an outer join, + * even if the outer join is not surely known to be happening between a + * recurring and a distributed rel. However, it's still safe to call + * UpdateWhereClauseToPushdownRecurringOuterJoinWalker() here as it only + * acts on the where clause if the join is happening between a + * recurring and a distributed rel. + */ + UpdateWhereClauseToPushdownRecurringOuterJoinWalker((Node *) taskQuery, + relationShardList); + } + + Task *subqueryTask = CreateBasicTask(jobId, taskId, taskType, NULL); if ((taskType == MODIFY_TASK && !modifyRequiresCoordinatorEvaluation) || diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 59124c5bf..43f79f30b 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -1321,7 +1321,8 @@ MultiShardUpdateDeleteSupported(Query *originalQuery, { errorMessage = DeferErrorIfUnsupportedSubqueryPushdown( originalQuery, - plannerRestrictionContext); + plannerRestrictionContext, + true); } return errorMessage; diff --git a/src/backend/distributed/planner/query_pushdown_planning.c b/src/backend/distributed/planner/query_pushdown_planning.c index 6150f4987..84f94dbb6 100644 --- a/src/backend/distributed/planner/query_pushdown_planning.c +++ b/src/backend/distributed/planner/query_pushdown_planning.c @@ -88,7 +88,7 @@ static bool WindowPartitionOnDistributionColumn(Query *query); static DeferredErrorMessage * DeferErrorIfFromClauseRecurs(Query *queryTree); static RecurringTuplesType FromClauseRecurringTupleType(Query *queryTree); static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin( - PlannerRestrictionContext *plannerRestrictionContext); + PlannerRestrictionContext *plannerRestrictionContext, bool plannerPhase); static DeferredErrorMessage * DeferErrorIfUnsupportedTableCombination(Query *queryTree); static DeferredErrorMessage * DeferErrorIfSubqueryRequiresMerge(Query *subqueryTree, bool lateral, @@ -109,6 +109,7 @@ static bool RelationInfoContainsOnlyRecurringTuples(PlannerInfo *plannerInfo, static char * RecurringTypeDescription(RecurringTuplesType recurType); static DeferredErrorMessage * DeferredErrorIfUnsupportedLateralSubquery( PlannerInfo *plannerInfo, Relids recurringRelIds, Relids nonRecurringRelIds); +static bool ContainsLateralSubquery(PlannerInfo *plannerInfo); static Var * PartitionColumnForPushedDownSubquery(Query *query); static bool ContainsReferencesToRelids(Query *query, Relids relids, int *foundRelid); static bool ContainsReferencesToRelidsWalker(Node *node, @@ -535,9 +536,16 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree, RaiseDeferredError(unsupportedQueryError, ERROR); } + /* + * We reach here at the third step of the planning, thus we already checked for pushed down + * feasibility of recurring outer joins, at this step the unsupported outer join check should + * only generate an error when there is a lateral subquery. + */ DeferredErrorMessage *subqueryPushdownError = DeferErrorIfUnsupportedSubqueryPushdown( originalQuery, - plannerRestrictionContext); + plannerRestrictionContext, + false); + if (subqueryPushdownError != NULL) { RaiseDeferredError(subqueryPushdownError, ERROR); @@ -560,7 +568,8 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree, DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery, PlannerRestrictionContext * - plannerRestrictionContext) + plannerRestrictionContext, + bool plannerPhase) { bool outerMostQueryHasLimit = false; ListCell *subqueryCell = NULL; @@ -612,7 +621,8 @@ DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery, return error; } - error = DeferredErrorIfUnsupportedRecurringTuplesJoin(plannerRestrictionContext); + error = DeferredErrorIfUnsupportedRecurringTuplesJoin(plannerRestrictionContext, + plannerPhase); if (error) { return error; @@ -770,12 +780,17 @@ FromClauseRecurringTupleType(Query *queryTree) * DeferredErrorIfUnsupportedRecurringTuplesJoin returns a DeferredError if * there exists a join between a recurring rel (such as reference tables * and intermediate_results) and a non-recurring rel (such as distributed tables - * and subqueries that we can push-down to worker nodes) that can return an - * incorrect result set due to recurring tuples coming from the recurring rel. + * and subqueries that we can push-down to worker nodes) when plannerPhase is + * true, so that we try to recursively plan these joins. + * During recursive planning phase, we either replace those with recursive plans + * or leave them if it is safe to push-down. + * During the logical planning phase (plannerPhase is false), we only check if + * such queries have lateral subqueries. */ static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin( - PlannerRestrictionContext *plannerRestrictionContext) + PlannerRestrictionContext *plannerRestrictionContext, + bool plannerPhase) { List *joinRestrictionList = plannerRestrictionContext->joinRestrictionContext->joinRestrictionList; @@ -828,14 +843,29 @@ DeferredErrorIfUnsupportedRecurringTuplesJoin( if (RelationInfoContainsOnlyRecurringTuples(plannerInfo, outerrelRelids)) { + if (plannerPhase) + { + /* + * We have not yet tried to recursively plan this join, we should + * defer an error. + */ + recurType = FetchFirstRecurType(plannerInfo, outerrelRelids); + break; + } + /* * Inner side contains distributed rels but the outer side only - * contains recurring rels, must be an unsupported lateral outer + * contains recurring rels, might be an unsupported lateral outer * join. + * Note that plannerInfo->hasLateralRTEs is not always set to + * true, so here we check rtes, see ContainsLateralSubquery for details. */ - recurType = FetchFirstRecurType(plannerInfo, outerrelRelids); - break; + if (ContainsLateralSubquery(plannerInfo)) + { + recurType = FetchFirstRecurType(plannerInfo, outerrelRelids); + break; + } } } else if (joinType == JOIN_FULL) @@ -1717,6 +1747,30 @@ DeferredErrorIfUnsupportedLateralSubquery(PlannerInfo *plannerInfo, } +/* + * ContainsLateralSubquery checks if the given plannerInfo contains any + * lateral subqueries in its rtable. If it does, it returns true, otherwise false. + */ +static bool +ContainsLateralSubquery(PlannerInfo *plannerInfo) +{ + ListCell *lc; + + foreach(lc, plannerInfo->parse->rtable) + { + RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc); + + /* We are only interested in subqueries that are lateral */ + if (rte->lateral && rte->rtekind == RTE_SUBQUERY) + { + return true; + } + } + + return false; +} + + /* * FetchFirstRecurType checks whether the relationInfo * contains any recurring table expression, namely a reference table, diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index 9db6481cb..856b09b3c 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -49,6 +49,7 @@ #include "postgres.h" #include "funcapi.h" +#include "miscadmin.h" #include "catalog/pg_class.h" #include "catalog/pg_type.h" @@ -73,8 +74,10 @@ #include "distributed/citus_nodes.h" #include "distributed/citus_ruleutils.h" +#include "distributed/combine_query_planner.h" #include "distributed/commands/multi_copy.h" #include "distributed/distributed_planner.h" +#include "distributed/distribution_column.h" #include "distributed/errormessage.h" #include "distributed/listutils.h" #include "distributed/local_distributed_join_planner.h" @@ -87,11 +90,14 @@ #include "distributed/multi_server_executor.h" #include "distributed/query_colocation_checker.h" #include "distributed/query_pushdown_planning.h" +#include "distributed/query_utils.h" #include "distributed/recursive_planning.h" #include "distributed/relation_restriction_equivalence.h" #include "distributed/shard_pruning.h" #include "distributed/version_compat.h" +bool EnableRecurringOuterJoinPushdown = true; + /* * RecursivePlanningContext is used to recursively plan subqueries * and CTEs, pull results to the coordinator, and push it back into @@ -104,6 +110,8 @@ struct RecursivePlanningContextInternal bool allDistributionKeysInQueryAreEqual; /* used for some optimizations */ List *subPlanList; PlannerRestrictionContext *plannerRestrictionContext; + bool restrictionEquivalenceCheck; + bool forceRecursivelyPlanRecurringOuterJoins; }; /* track depth of current recursive planner query */ @@ -152,7 +160,8 @@ static void RecursivelyPlanNonColocatedSubqueriesInWhere(Query *query, RecursivePlanningContext * recursivePlanningContext); static bool RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, - RecursivePlanningContext *context); + RecursivePlanningContext *context, + bool chainedJoin); static void RecursivelyPlanDistributedJoinNode(Node *node, Query *query, RecursivePlanningContext *context); static bool IsRTERefRecurring(RangeTblRef *rangeTableRef, Query *query); @@ -193,6 +202,9 @@ static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry, List *outerSubqueryTargetList); static List * GenerateRequiredColNamesFromTargetList(List *targetList); static char * GetRelationNameAndAliasName(RangeTblEntry *rangeTablentry); +static bool CanPushdownRecurringOuterJoinOnOuterRTE(RangeTblEntry *rte); +static bool CanPushdownRecurringOuterJoinOnInnerVar(Var *innervar, RangeTblEntry *rte); +static bool CanPushdownRecurringOuterJoin(JoinExpr *joinExpr, Query *query); #if PG_VERSION_NUM < PG_VERSION_17 static bool hasPseudoconstantQuals( RelationRestrictionContext *relationRestrictionContext); @@ -207,7 +219,8 @@ static bool hasPseudoconstantQuals( */ List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery, - PlannerRestrictionContext *plannerRestrictionContext) + PlannerRestrictionContext *plannerRestrictionContext, + RouterPlanType routerPlan) { RecursivePlanningContext context; @@ -221,6 +234,17 @@ GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery, context.planId = planId; context.subPlanList = NIL; context.plannerRestrictionContext = plannerRestrictionContext; + context.forceRecursivelyPlanRecurringOuterJoins = false; + + /* + * Force recursive planning of recurring outer joins for these queries + * since the planning error from the previous step is generated prior to + * the actual planning attempt. + */ + if (routerPlan == DML_QUERY) + { + context.forceRecursivelyPlanRecurringOuterJoins = true; + } /* * Calculating the distribution key equality upfront is a trade-off for us. @@ -363,7 +387,7 @@ RecursivelyPlanSubqueriesAndCTEs(Query *query, RecursivePlanningContext *context if (ShouldRecursivelyPlanOuterJoins(query, context)) { RecursivelyPlanRecurringTupleOuterJoinWalker((Node *) query->jointree, - query, context); + query, context, false); } /* @@ -691,7 +715,8 @@ RecursivelyPlanNonColocatedSubqueriesInWhere(Query *query, static bool RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, RecursivePlanningContext * - recursivePlanningContext) + recursivePlanningContext, + bool chainedJoin) { if (node == NULL) { @@ -708,7 +733,8 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, Node *fromElement = (Node *) lfirst(fromExprCell); RecursivelyPlanRecurringTupleOuterJoinWalker(fromElement, query, - recursivePlanningContext); + recursivePlanningContext, + false); } /* @@ -734,10 +760,12 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, */ bool leftNodeRecurs = RecursivelyPlanRecurringTupleOuterJoinWalker(leftNode, query, - recursivePlanningContext); + recursivePlanningContext, + true); bool rightNodeRecurs = RecursivelyPlanRecurringTupleOuterJoinWalker(rightNode, query, - recursivePlanningContext); + recursivePlanningContext, + true); switch (joinExpr->jointype) { case JOIN_LEFT: @@ -745,11 +773,23 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, /* left join */ if (leftNodeRecurs && !rightNodeRecurs) { - ereport(DEBUG1, (errmsg("recursively planning right side of " - "the left join since the outer side " - "is a recurring rel"))); - RecursivelyPlanDistributedJoinNode(rightNode, query, - recursivePlanningContext); + if (recursivePlanningContext->forceRecursivelyPlanRecurringOuterJoins + || + chainedJoin || !CanPushdownRecurringOuterJoin(joinExpr, + query)) + { + ereport(DEBUG1, (errmsg("recursively planning right side of " + "the left join since the outer side " + "is a recurring rel"))); + RecursivelyPlanDistributedJoinNode(rightNode, query, + recursivePlanningContext); + } + else + { + ereport(DEBUG3, (errmsg( + "a push down safe left join with recurring left side"))); + leftNodeRecurs = false; /* left node will be pushed down */ + } } /* @@ -766,11 +806,23 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, /* right join */ if (!leftNodeRecurs && rightNodeRecurs) { - ereport(DEBUG1, (errmsg("recursively planning left side of " - "the right join since the outer side " - "is a recurring rel"))); - RecursivelyPlanDistributedJoinNode(leftNode, query, - recursivePlanningContext); + if (recursivePlanningContext->forceRecursivelyPlanRecurringOuterJoins + || + chainedJoin || !CanPushdownRecurringOuterJoin(joinExpr, + query)) + { + ereport(DEBUG1, (errmsg("recursively planning left side of " + "the right join since the outer side " + "is a recurring rel"))); + RecursivelyPlanDistributedJoinNode(leftNode, query, + recursivePlanningContext); + } + else + { + ereport(DEBUG3, (errmsg( + "a push down safe right join with recurring left side"))); + rightNodeRecurs = false; /* right node will be pushed down */ + } } /* @@ -2642,3 +2694,335 @@ hasPseudoconstantQuals(RelationRestrictionContext *relationRestrictionContext) #endif + + +/* + * CanPushdownRecurringOuterJoinOnOuterRTE returns true if the given range table entry + * is safe for pushdown when it is the outer relation of a outer join when the + * inner relation is not recurring. + * Currently, we only allow reference tables. + */ +static bool +CanPushdownRecurringOuterJoinOnOuterRTE(RangeTblEntry *rte) +{ + if (IsCitusTable(rte->relid) && IsCitusTableType(rte->relid, REFERENCE_TABLE)) + { + return true; + } + else + { + ereport(DEBUG5, (errmsg("RTE type %d is not safe for pushdown", + rte->rtekind))); + return false; + } +} + + +/* + * ResolveBaseVarFromSubquery recursively resolves a Var from a subquery target list to + * the base Var and RTE + */ +bool +ResolveBaseVarFromSubquery(Var *var, Query *query, + Var **baseVar, RangeTblEntry **baseRte) +{ + TargetEntry *tle = get_tle_by_resno(query->targetList, var->varattno); + if (!tle || !IsA(tle->expr, Var)) + { + return false; + } + + Var *tleVar = (Var *) tle->expr; + RangeTblEntry *rte = rt_fetch(tleVar->varno, query->rtable); + + if (rte == NULL) + { + return false; + } + + if (rte->rtekind == RTE_RELATION || rte->rtekind == RTE_FUNCTION) + { + *baseVar = tleVar; + *baseRte = rte; + return true; + } + else if (rte->rtekind == RTE_SUBQUERY) + { + /* Prevent overflow, and allow query cancellation */ + check_stack_depth(); + CHECK_FOR_INTERRUPTS(); + return ResolveBaseVarFromSubquery(tleVar, rte->subquery, baseVar, baseRte); + } + + return false; +} + + +/* + * CanPushdownRecurringOuterJoinOnInnerVar checks if the inner variable + * from a join qual for a join pushdown. It returns true if it is valid, + * it is the partition column and hash distributed, otherwise it returns false. + */ +static bool +CanPushdownRecurringOuterJoinOnInnerVar(Var *innerVar, RangeTblEntry *rte) +{ + if (!innerVar || !rte) + { + return false; + } + + if (innerVar->varattno == InvalidAttrNumber) + { + return false; + } + + CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(rte->relid); + + if (!cacheEntry || GetCitusTableType(cacheEntry) != HASH_DISTRIBUTED) + { + return false; + } + + /* Check if the inner variable is part of the distribution column */ + if (cacheEntry->partitionColumn && innerVar->varattno == + cacheEntry->partitionColumn->varattno) + { + return true; + } + + return false; +} + + +/* + * JoinTreeContainsLateral checks if the given node contains a lateral + * join. It returns true if it does, otherwise false. + * + * It recursively traverses the join tree and checks each RangeTblRef and JoinExpr + * for lateral joins. + */ +static bool +JoinTreeContainsLateral(Node *node, List *rtable) +{ + if (node == NULL) + { + return false; + } + + /* Prevent overflow, and allow query cancellation */ + check_stack_depth(); + CHECK_FOR_INTERRUPTS(); + + if (IsA(node, RangeTblRef)) + { + RangeTblEntry *rte = rt_fetch(((RangeTblRef *) node)->rtindex, rtable); + if (rte == NULL) + { + return false; + } + + if (rte->lateral) + { + return true; + } + + if (rte->rtekind == RTE_SUBQUERY) + { + if (rte->subquery) + { + return JoinTreeContainsLateral((Node *) rte->subquery->jointree, + rte->subquery->rtable); + } + } + return false; + } + else if (IsA(node, JoinExpr)) + { + JoinExpr *join = (JoinExpr *) node; + return JoinTreeContainsLateral(join->larg, rtable) || + JoinTreeContainsLateral(join->rarg, rtable); + } + else if (IsA(node, FromExpr)) + { + FromExpr *fromExpr = (FromExpr *) node; + ListCell *lc = NULL; + foreach(lc, fromExpr->fromlist) + { + if (JoinTreeContainsLateral((Node *) lfirst(lc), rtable)) + { + return true; + } + } + } + return false; +} + + +/* + * CanPushdownRecurringOuterJoinExtended checks if the given join expression + * is an outer join between recurring rel -on outer part- and a distributed + * rel -on the inner side- and if it is feasible to push down the join. If feasible, + * it computes the outer relation's range table index, the outer relation's + * range table entry, the inner (distributed) relation's range table entry, and the + * attribute number of the partition column in the outer relation. + */ +bool +CanPushdownRecurringOuterJoinExtended(JoinExpr *joinExpr, Query *query, + int *outerRtIndex, RangeTblEntry **outerRte, + RangeTblEntry **distRte, int *attnum) +{ + if (!EnableRecurringOuterJoinPushdown) + { + return false; + } + + if (!IS_OUTER_JOIN(joinExpr->jointype)) + { + return false; + } + + if (joinExpr->jointype != JOIN_LEFT && joinExpr->jointype != JOIN_RIGHT) + { + return false; + } + + /* Push down for chained joins is not supported in this path. */ + if (IsA(joinExpr->rarg, JoinExpr) || IsA(joinExpr->larg, JoinExpr)) + { + ereport(DEBUG5, (errmsg( + "One side is a join expression, pushdown is not supported in this path."))); + return false; + } + + /* Push down for joins with fromExpr on one side is not supported in this path. */ + if (!IsA(joinExpr->larg, RangeTblRef) || !IsA(joinExpr->rarg, RangeTblRef)) + { + ereport(DEBUG5, (errmsg( + "One side is not a RangeTblRef, pushdown is not supported in this path."))); + return false; + } + + if (joinExpr->jointype == JOIN_LEFT) + { + *outerRtIndex = (((RangeTblRef *) joinExpr->larg)->rtindex); + } + else /* JOIN_RIGHT */ + { + *outerRtIndex = (((RangeTblRef *) joinExpr->rarg)->rtindex); + } + + *outerRte = rt_fetch(*outerRtIndex, query->rtable); + + if (!CanPushdownRecurringOuterJoinOnOuterRTE(*outerRte)) + { + return false; + } + + /* For now if we see any lateral join in the join tree, we return false. + * This check can be improved to support the cases where the lateral reference + * does not cause an error in the final planner checks. + */ + if (JoinTreeContainsLateral(joinExpr->rarg, query->rtable) || JoinTreeContainsLateral( + joinExpr->larg, query->rtable)) + { + ereport(DEBUG5, (errmsg( + "Lateral join is not supported for pushdown in this path."))); + return false; + } + + /* Check if the join is performed on the distribution column */ + List *joinClauseList = make_ands_implicit((Expr *) joinExpr->quals); + if (joinClauseList == NIL) + { + return false; + } + + Node *joinClause = NULL; + foreach_declared_ptr(joinClause, joinClauseList) + { + if (!NodeIsEqualsOpExpr(joinClause)) + { + continue; + } + OpExpr *joinClauseExpr = castNode(OpExpr, joinClause); + + Var *leftColumn = LeftColumnOrNULL(joinClauseExpr); + Var *rightColumn = RightColumnOrNULL(joinClauseExpr); + if (leftColumn == NULL || rightColumn == NULL) + { + continue; + } + + RangeTblEntry *rte; + Var *innerVar; + if (leftColumn->varno == *outerRtIndex) + { + /* left column is the outer table of the comparison, get right */ + rte = rt_fetch(rightColumn->varno, query->rtable); + innerVar = rightColumn; + + /* additional constraints will be introduced on outer relation variable */ + *attnum = leftColumn->varattno; + } + else if (rightColumn->varno == *outerRtIndex) + { + /* right column is the outer table of the comparison, get left*/ + rte = rt_fetch(leftColumn->varno, query->rtable); + innerVar = leftColumn; + + /* additional constraints will be introduced on outer relation variable */ + *attnum = rightColumn->varattno; + } + else + { + continue; + } + + /* the simple case, the inner table itself a Citus table */ + if (rte && IsCitusTable(rte->relid)) + { + if (CanPushdownRecurringOuterJoinOnInnerVar(innerVar, rte)) + { + *distRte = rte; + return true; + } + } + /* the inner table is a subquery, extract the base relation referred in the qual */ + else if (rte && rte->rtekind == RTE_SUBQUERY) + { + Var *baseVar = NULL; + RangeTblEntry *baseRte = NULL; + + if (ResolveBaseVarFromSubquery(innerVar, rte->subquery, &baseVar, &baseRte)) + { + if (baseRte && IsCitusTable(baseRte->relid)) + { + if (CanPushdownRecurringOuterJoinOnInnerVar(baseVar, baseRte)) + { + *distRte = baseRte; + return true; + } + } + } + } + } + + return false; +} + + +/* + * CanPushdownRecurringOuterJoin initializes input variables to call + * CanPushdownRecurringOuterJoinExtended. + * See CanPushdownRecurringOuterJoinExtended for more details. + */ +bool +CanPushdownRecurringOuterJoin(JoinExpr *joinExpr, Query *query) +{ + int outerRtIndex; + RangeTblEntry *outerRte = NULL; + RangeTblEntry *innerRte = NULL; + int attnum; + return CanPushdownRecurringOuterJoinExtended(joinExpr, query, &outerRtIndex, + &outerRte, &innerRte, &attnum); +} diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 165aea05f..e2d21d074 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -1480,6 +1480,21 @@ RegisterCitusConfigVariables(void) GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE, NULL, NULL, NULL); + DefineCustomBoolVariable( + "citus.enable_recurring_outer_join_pushdown", + gettext_noop("Enables outer join pushdown for recurring relations."), + gettext_noop("When enabled, Citus will try to push down outer joins " + "between recurring and non-recurring relations to workers " + "whenever feasible by introducing correctness constraints " + "to the where clause of the query. Note that if this is " + "disabled, or push down is not feasible, the result will " + "be computed via recursive planning."), + &EnableRecurringOuterJoinPushdown, + true, + PGC_USERSET, + GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE, + NULL, NULL, NULL); + DefineCustomBoolVariable( "citus.enable_repartition_joins", gettext_noop("Allows Citus to repartition data between nodes."), diff --git a/src/include/distributed/deparse_shard_query.h b/src/include/distributed/deparse_shard_query.h index efcdb3032..751f3bf50 100644 --- a/src/include/distributed/deparse_shard_query.h +++ b/src/include/distributed/deparse_shard_query.h @@ -20,10 +20,17 @@ #include "nodes/pg_list.h" #include "distributed/citus_custom_scan.h" +#include "distributed/query_utils.h" extern void RebuildQueryStrings(Job *workerJob); extern bool UpdateRelationToShardNames(Node *node, List *relationShardList); +extern void UpdateWhereClauseToPushdownRecurringOuterJoin(Query *query, + List *relationShardList); +extern bool UpdateWhereClauseToPushdownRecurringOuterJoinWalker(Node *node, + List *relationShardList); +Node * CreateQualsForShardInterval(RelationShard *relationShard, int attnum, + int outerRtIndex); extern void SetTaskQueryIfShouldLazyDeparse(Task *task, Query *query); extern void SetTaskQueryString(Task *task, char *queryString); extern void SetTaskQueryStringList(Task *task, List *queryStringList); diff --git a/src/include/distributed/distributed_planner.h b/src/include/distributed/distributed_planner.h index f416aa911..67637cd78 100644 --- a/src/include/distributed/distributed_planner.h +++ b/src/include/distributed/distributed_planner.h @@ -33,6 +33,18 @@ extern int PlannerLevel; +/* RouterPlanType is used to determine the router plan to invoke */ +typedef enum RouterPlanType +{ + INSERT_SELECT_INTO_CITUS_TABLE, + INSERT_SELECT_INTO_LOCAL_TABLE, + DML_QUERY, + SELECT_QUERY, + MERGE_QUERY, + REPLAN_WITH_BOUND_PARAMETERS +} RouterPlanType; + + typedef struct RelationRestrictionContext { bool allReferenceTables; diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index f1120497b..821a2c28d 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -146,6 +146,7 @@ extern bool IsCitusTableType(Oid relationId, CitusTableType tableType); extern CitusTableType GetCitusTableType(CitusTableCacheEntry *tableEntry); extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, CitusTableType tableType); +extern bool IsFirstShard(CitusTableCacheEntry *tableEntry, uint64 shardId); bool HasDistributionKey(Oid relationId); bool HasDistributionKeyCacheEntry(CitusTableCacheEntry *tableEntry); extern char * GetTableTypeName(Oid tableId); diff --git a/src/include/distributed/query_pushdown_planning.h b/src/include/distributed/query_pushdown_planning.h index 7035f5fc2..ba92a0462 100644 --- a/src/include/distributed/query_pushdown_planning.h +++ b/src/include/distributed/query_pushdown_planning.h @@ -37,11 +37,12 @@ extern MultiNode * SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree, PlannerRestrictionContext * plannerRestrictionContext); -extern DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown(Query * - originalQuery, - PlannerRestrictionContext - * - plannerRestrictionContext); +extern DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown( + Query *originalQuery, + PlannerRestrictionContext + * + plannerRestrictionContext, + bool plannerPhase); extern DeferredErrorMessage * DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLimit); diff --git a/src/include/distributed/recursive_planning.h b/src/include/distributed/recursive_planning.h index b4aaa4785..219e8b745 100644 --- a/src/include/distributed/recursive_planning.h +++ b/src/include/distributed/recursive_planning.h @@ -16,10 +16,12 @@ #include "pg_version_constants.h" +#include "distributed/distributed_planner.h" #include "distributed/errormessage.h" #include "distributed/log_utils.h" #include "distributed/relation_restriction_equivalence.h" +extern bool EnableRecurringOuterJoinPushdown; typedef struct RecursivePlanningContextInternal RecursivePlanningContext; typedef struct RangeTblEntryIndex @@ -32,7 +34,8 @@ extern PlannerRestrictionContext * GetPlannerRestrictionContext( RecursivePlanningContext *recursivePlanningContext); extern List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery, PlannerRestrictionContext * - plannerRestrictionContext); + plannerRestrictionContext, + RouterPlanType routerPlan); extern char * GenerateResultId(uint64 planId, uint32 subPlanId); extern Query * BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resultId); @@ -51,6 +54,11 @@ extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry); extern bool IsRelationLocalTableOrMatView(Oid relationId); extern bool ContainsReferencesToOuterQuery(Query *query); extern void UpdateVarNosInNode(Node *node, Index newVarNo); - - +extern bool CanPushdownRecurringOuterJoinExtended(JoinExpr *joinExpr, Query *query, + int *outerRtIndex, + RangeTblEntry **outerRte, + RangeTblEntry **distRte, + int *attnum); +bool ResolveBaseVarFromSubquery(Var *var, Query *query, Var **baseVar, + RangeTblEntry **baseRte); #endif /* RECURSIVE_PLANNING_H */ diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index 593c37bdd..507577c5f 100755 --- a/src/test/regress/citus_tests/run_test.py +++ b/src/test/regress/citus_tests/run_test.py @@ -235,6 +235,20 @@ DEPS = { "multi_subquery_in_where_clause": TestDeps( "minimal_schedule", ["multi_behavioral_analytics_create_table"] ), + "multi_limit_clause_approximate": TestDeps( + "minimal_schedule", + ["multi_create_table", "multi_create_users", "multi_load_data"], + ), + "multi_single_relation_subquery": TestDeps( + "minimal_schedule", + ["multi_create_table", "multi_create_users", "multi_load_data"], + ), + "multi_subquery_complex_reference_clause": TestDeps( + "minimal_schedule", ["multi_behavioral_analytics_create_table"] + ), + "multi_subquery_in_where_reference_clause": TestDeps( + "minimal_schedule", ["multi_behavioral_analytics_create_table"] + ), } diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 76538f22d..1d854704e 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -760,21 +760,28 @@ DEBUG: distributed statement: INSERT INTO multi_insert_select.agg_events_133000 DEBUG: distributed statement: INSERT INTO multi_insert_select.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (multi_insert_select.raw_events_first_13300001 raw_events_first LEFT JOIN multi_insert_select.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 IS NOT NULL) DEBUG: distributed statement: INSERT INTO multi_insert_select.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (multi_insert_select.raw_events_first_13300002 raw_events_first LEFT JOIN multi_insert_select.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 IS NOT NULL) DEBUG: distributed statement: INSERT INTO multi_insert_select.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (multi_insert_select.raw_events_first_13300003 raw_events_first LEFT JOIN multi_insert_select.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 IS NOT NULL) + SET client_min_messages to debug3; INSERT INTO agg_events (user_id) SELECT raw_events_second.user_id FROM reference_table LEFT JOIN raw_events_second ON reference_table.user_id = raw_events_second.user_id; +DEBUG: no shard pruning constraints on raw_events_second found +DEBUG: shard count after pruning for raw_events_second: 4 DEBUG: cannot perform a lateral outer join when a distributed subquery references a reference table +DEBUG: no shard pruning constraints on raw_events_second found +DEBUG: shard count after pruning for raw_events_second: 4 DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "raw_events_second" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "raw_events_second" to a subquery -DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM multi_insert_select.raw_events_second WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT raw_events_second.user_id FROM (multi_insert_select.reference_table LEFT JOIN (SELECT raw_events_second_1.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 FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) raw_events_second_1) raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) -DEBUG: Creating router plan -DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on raw_events_second found +DEBUG: shard count after pruning for raw_events_second: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: performing repartitioned INSERT ... SELECT +DEBUG: partitioning SELECT query by column index 0 with name 'user_id' + SET client_min_messages to debug2; INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id @@ -3378,8 +3385,8 @@ $$); Task Count: 1 (4 rows) --- verify that insert select cannot be pushed down when we have reference table in outside of outer join. -SELECT coordinator_plan($$ +-- verify that insert select cannot be pushed down when we have reference table in outside of outer join in a chained-join. + SELECT coordinator_plan($$ EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT a.id FROM dist_table_5 a LEFT JOIN ref_table_1 b ON (true) RIGHT JOIN ref_table_1 c ON (true); $$); coordinator_plan @@ -3392,9 +3399,21 @@ $$); Task Count: 4 (6 rows) --- verify that insert select cannot be pushed down when it has a recurring outer join in a subquery. +-- verify that insert select can be pushed down when we have reference table in outside of outer join. SELECT coordinator_plan($$ EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT id FROM ref_table_1 LEFT JOIN dist_table_5 USING(id); +$$); + coordinator_plan +--------------------------------------------------------------------- + Custom Scan (Citus INSERT ... SELECT) + INSERT/SELECT method: repartition + -> Custom Scan (Citus Adaptive) + Task Count: 4 +(4 rows) + +-- verify that insert select cannot be pushed down when we have reference table in outside of left join and joined on non-partition column. +SELECT coordinator_plan($$ + EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT ref_table_1.id FROM ref_table_1 LEFT JOIN dist_table_5 ON ref_table_1.id = dist_table_5.id2; $$); coordinator_plan --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_level_recursive_queries.out b/src/test/regress/expected/multi_level_recursive_queries.out index e0a4d44a9..2eaf0b53a 100644 --- a/src/test/regress/expected/multi_level_recursive_queries.out +++ b/src/test/regress/expected/multi_level_recursive_queries.out @@ -62,12 +62,9 @@ RIGHT JOIN ( RIGHT JOIN tbl_dist1 USING (id) ) AS table_4 USING (id); DEBUG: CTE cte_0 is going to be inlined via distributed planning -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "tbl_dist1" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "tbl_dist1" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT id FROM multi_recursive.tbl_dist1 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN multi_recursive.tbl_dist1 USING (id)) DEBUG: push down of limit count: 0 -DEBUG: generating subplan XXX_2 for subquery SELECT id FROM (SELECT tbl_dist1.id FROM multi_recursive.tbl_dist1 WHERE (tbl_dist1.id OPERATOR(pg_catalog.=) ANY (SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN (SELECT tbl_dist1_2.id FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) tbl_dist1_2) tbl_dist1_1 USING (id))))) cte_0 LIMIT 0 +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM (SELECT tbl_dist1.id FROM multi_recursive.tbl_dist1 WHERE (tbl_dist1.id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)))) cte_0 LIMIT 0 DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(table_4.id) AS count FROM (multi_recursive.tbl_dist1 RIGHT JOIN (SELECT table_5.id FROM ((SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) table_5 RIGHT JOIN multi_recursive.tbl_dist1 tbl_dist1_1 USING (id))) table_4 USING (id)) DEBUG: generating subplan XXX_1 for subquery SELECT table_5.id FROM ((SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) table_5 RIGHT JOIN multi_recursive.tbl_dist1 USING (id)) DEBUG: recursively planning left side of the right join since the outer side is a recurring rel @@ -76,11 +73,8 @@ DEBUG: Wrapping relation "tbl_dist1" to a subquery DEBUG: generating subplan XXX_2 for subquery SELECT id FROM multi_recursive.tbl_dist1 WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(table_4.id) AS count FROM ((SELECT tbl_dist1_1.id FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) tbl_dist1_1) tbl_dist1 RIGHT JOIN (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) table_4 USING (id)) DEBUG: generating subplan XXX_1 for CTE cte_0: SELECT id FROM multi_recursive.tbl_dist1 WHERE (id OPERATOR(pg_catalog.=) ANY (SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN multi_recursive.tbl_dist1 tbl_dist1_1 USING (id)))) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "tbl_dist1" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "tbl_dist1" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT id FROM multi_recursive.tbl_dist1 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id FROM multi_recursive.tbl_dist1 WHERE (id OPERATOR(pg_catalog.=) ANY (SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN (SELECT tbl_dist1_2.id FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) tbl_dist1_2) tbl_dist1_1 USING (id)))) +DEBUG: generating subplan XXX_1 for subquery SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN multi_recursive.tbl_dist1 USING (id)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id FROM multi_recursive.tbl_dist1 WHERE (id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer))) DEBUG: generating subplan XXX_2 for subquery SELECT id FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) cte_0 LIMIT 0 DEBUG: generating subplan XXX_3 for subquery SELECT table_5.id FROM ((SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) table_5 RIGHT JOIN multi_recursive.tbl_dist1 USING (id)) DEBUG: recursively planning left side of the right join since the outer side is a recurring rel diff --git a/src/test/regress/expected/multi_outer_join_reference.out b/src/test/regress/expected/multi_outer_join_reference.out index aca91bda7..0a474f9d9 100644 --- a/src/test/regress/expected/multi_outer_join_reference.out +++ b/src/test/regress/expected/multi_outer_join_reference.out @@ -208,12 +208,11 @@ FROM 20 | 0 (1 row) --- Right join is allowed as we recursively plan the distributed table (multi_outer_join_left_hash) +-- Right join is allowed (multi_outer_join_left_hash) SELECT min(r_custkey), max(r_custkey) FROM multi_outer_join_left_hash a RIGHT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey); -LOG: join order: [ "multi_outer_join_left_hash" ] min | max --------------------------------------------------------------------- 1 | 15 @@ -323,12 +322,11 @@ FROM 25 | 1 (1 row) --- Right join should be allowed in this case as we recursively plan the distributed table (multi_outer_join_left_hash +-- Right join should be allowed in this case (multi_outer_join_left_hash) SELECT min(r_custkey), max(r_custkey) FROM multi_outer_join_left_hash a RIGHT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey); -LOG: join order: [ "multi_outer_join_left_hash" ] min | max --------------------------------------------------------------------- 11 | 30 diff --git a/src/test/regress/expected/multi_subquery_complex_reference_clause.out b/src/test/regress/expected/multi_subquery_complex_reference_clause.out index 7db33834f..c49718e33 100644 --- a/src/test/regress/expected/multi_subquery_complex_reference_clause.out +++ b/src/test/regress/expected/multi_subquery_complex_reference_clause.out @@ -1,7 +1,7 @@ -- -- multi subquery complex queries aims to expand existing subquery pushdown -- regression tests to cover more caeses --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests -- SET citus.next_shard_id TO 1400000; @@ -539,15 +539,12 @@ SELECT * FROM SELECT user_id FROM user_buy_test_table) sub ORDER BY 1 DESC; DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "user_buy_test_table" "dis" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "user_buy_test_table" "dis" to a subquery 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 dis WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT dis.user_id FROM (public.users_ref_test_table ref LEFT JOIN public.user_buy_test_table dis ON ((ref.id OPERATOR(pg_catalog.=) dis.user_id))) DEBUG: Router planner cannot handle multi-shard select queries DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.user_buy_test_table DEBUG: Creating router plan -DEBUG: generating subplan XXX_3 for subquery SELECT dis.user_id FROM (public.users_ref_test_table ref LEFT JOIN (SELECT dis_1.user_id, NULL::integer AS item_id, NULL::integer AS buy_count FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) dis_1) dis ON ((ref.id OPERATOR(pg_catalog.=) dis.user_id))) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) +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) UNION 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)) sub ORDER BY user_id DESC DEBUG: Creating router plan user_id @@ -2452,5 +2449,4 @@ SELECT count(*) FROM (SELECT ref1.* FROM users_ref_test_table ref1 INNER JOIN us (1 row) DROP TABLE user_buy_test_table; -DROP TABLE users_ref_test_table; DROP TABLE users_return_test_table; diff --git a/src/test/regress/expected/multi_subquery_in_where_reference_clause.out b/src/test/regress/expected/multi_subquery_in_where_reference_clause.out index 0f656ee0b..eabc5e421 100644 --- a/src/test/regress/expected/multi_subquery_in_where_reference_clause.out +++ b/src/test/regress/expected/multi_subquery_in_where_reference_clause.out @@ -146,15 +146,15 @@ LIMIT 3; 3 (3 rows) --- outer join could still recur +-- outer join pushed down SELECT DISTINCT user_id FROM users_table RIGHT JOIN users_reference_table USING (user_id) WHERE - users_reference_table.value_2 IN + (users_reference_table.value_2, random()*0) IN (SELECT - value_2 + value_2, 0 FROM events_table WHERE @@ -162,7 +162,13 @@ WHERE ) ORDER BY user_id LIMIT 3; -ERROR: correlated subqueries are not supported when the FROM clause contains a CTE or subquery + user_id +--------------------------------------------------------------------- + 1 + 2 + 3 +(3 rows) + -- subqueries in WHERE with IN operator without equality SELECT users_table.user_id, count(*) diff --git a/src/test/regress/expected/pg17.out b/src/test/regress/expected/pg17.out index f70062eaa..a2a590a94 100644 --- a/src/test/regress/expected/pg17.out +++ b/src/test/regress/expected/pg17.out @@ -339,13 +339,46 @@ DEBUG: Router planner cannot handle multi-shard select queries 4 | 130 (2 rows) +SET client_min_messages TO DEBUG3; +CREATE TABLE users_ref(user_id int, dept int); +SELECT create_reference_table('users_ref'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO users_ref VALUES (1, 3), (2, 4), (3, 3), (4, 4); +DEBUG: Creating router plan +DEBUG: assigned task to node localhost:xxxxx +-- In PG17, the planner can pull up a correlated ANY subquery to a join, resulting +-- in a different query plan compared to PG16. Specifically, for the following query +-- the rewritten query has a lateral recurring outer join, which requires recursive +-- computation of the inner part. However, this join is not analyzed during the recursive +-- planning step, as it is performed on the original query structure. As a result, +-- the lateral join is not recursively planned, and a lateral join error is raised +-- at a later stage. +SELECT user_id FROM +users RIGHT JOIN users_ref USING (user_id) +WHERE users_ref.dept IN +( + SELECT events.event_type FROM events WHERE events.user_id = users.user_id +) ORDER BY 1 LIMIT 1; +DEBUG: no shard pruning constraints on users found +DEBUG: shard count after pruning for users: 2 +DEBUG: no shard pruning constraints on events found +DEBUG: shard count after pruning for events: 2 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe right join with recurring left side +ERROR: cannot perform a lateral outer join when a distributed subquery references a reference table RESET client_min_messages; RESET search_path; DROP SCHEMA pg17_corr_subq_folding CASCADE; -NOTICE: drop cascades to 3 other objects +NOTICE: drop cascades to 5 other objects DETAIL: drop cascades to table pg17_corr_subq_folding.test drop cascades to table pg17_corr_subq_folding.users drop cascades to table pg17_corr_subq_folding.events +drop cascades to table pg17_corr_subq_folding.users_ref +drop cascades to table pg17_corr_subq_folding.users_ref_20240023 -- Queries with outer joins with pseudoconstant quals work only in PG17 -- Relevant PG17 commit: -- https://github.com/postgres/postgres/commit/9e9931d2b diff --git a/src/test/regress/expected/pg17_0.out b/src/test/regress/expected/pg17_0.out index 697c97a15..c29cbdab8 100644 --- a/src/test/regress/expected/pg17_0.out +++ b/src/test/regress/expected/pg17_0.out @@ -283,13 +283,57 @@ DEBUG: Router planner cannot handle multi-shard select queries 4 | 130 (2 rows) +SET client_min_messages TO DEBUG3; +CREATE TABLE users_ref(user_id int, dept int); +SELECT create_reference_table('users_ref'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO users_ref VALUES (1, 3), (2, 4), (3, 3), (4, 4); +DEBUG: Creating router plan +DEBUG: assigned task to node localhost:xxxxx +-- In PG17, the planner can pull up a correlated ANY subquery to a join, resulting +-- in a different query plan compared to PG16. Specifically, for the following query +-- the rewritten query has a lateral recurring outer join, which requires recursive +-- computation of the inner part. However, this join is not analyzed during the recursive +-- planning step, as it is performed on the original query structure. As a result, +-- the lateral join is not recursively planned, and a lateral join error is raised +-- at a later stage. +SELECT user_id FROM +users RIGHT JOIN users_ref USING (user_id) +WHERE users_ref.dept IN +( + SELECT events.event_type FROM events WHERE events.user_id = users.user_id +) ORDER BY 1 LIMIT 1; +DEBUG: no shard pruning constraints on events found +DEBUG: shard count after pruning for events: 2 +DEBUG: no shard pruning constraints on users found +DEBUG: shard count after pruning for users: 2 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe right join with recurring left side +DEBUG: push down of limit count: 1 +DEBUG: no shard pruning constraints on events found +DEBUG: shard count after pruning for events: 2 +DEBUG: no shard pruning constraints on users found +DEBUG: shard count after pruning for users: 2 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + user_id +--------------------------------------------------------------------- + 1 +(1 row) + RESET client_min_messages; RESET search_path; DROP SCHEMA pg17_corr_subq_folding CASCADE; -NOTICE: drop cascades to 3 other objects +NOTICE: drop cascades to 5 other objects DETAIL: drop cascades to table pg17_corr_subq_folding.test drop cascades to table pg17_corr_subq_folding.users drop cascades to table pg17_corr_subq_folding.events +drop cascades to table pg17_corr_subq_folding.users_ref +drop cascades to table pg17_corr_subq_folding.users_ref_20240023 -- Queries with outer joins with pseudoconstant quals work only in PG17 -- Relevant PG17 commit: -- https://github.com/postgres/postgres/commit/9e9931d2b diff --git a/src/test/regress/expected/query_single_shard_table.out b/src/test/regress/expected/query_single_shard_table.out index fa1641a2b..5475e0c63 100644 --- a/src/test/regress/expected/query_single_shard_table.out +++ b/src/test/regress/expected/query_single_shard_table.out @@ -1985,11 +1985,8 @@ INSERT INTO raw_events_second (user_id, value_1) SELECT (a+5)*-1, b FROM cte; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: CTE cte is going to be inlined via distributed planning -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "distributed_table" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a))) cte) citus_insert_select_subquery("?column?", b) +DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT reference_table.a, 1 AS b FROM (query_single_shard_table.distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte) citus_insert_select_subquery("?column?", b) DEBUG: Collecting INSERT ... SELECT results on coordinator -- .. and via SELECT's cte list too INSERT INTO raw_events_second (user_id, value_1) @@ -2000,11 +1997,8 @@ WITH cte AS ( SELECT (a+5)*2, b FROM cte; DEBUG: CTE cte is going to be inlined via distributed planning DEBUG: cannot perform a lateral outer join when a distributed subquery references a reference table -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "distributed_table" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) 2) AS user_id, b AS value_1 FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a))) cte +DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT reference_table.a, 1 AS b FROM (query_single_shard_table.distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) 2) AS user_id, b AS value_1 FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte DEBUG: Collecting INSERT ... SELECT results on coordinator -- using set operations INSERT INTO @@ -2241,13 +2235,6 @@ UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.b IN (SELECT b FROM cte); DEBUG: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns DEBUG: generating subplan XXX_1 for CTE cte: SELECT reference_table.a, 1 AS b FROM (query_single_shard_table.distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a)) DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "distributed_table" to a subquery -DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a)) -DEBUG: Creating router plan DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE query_single_shard_table.nullkey_c1_t1 SET b = 5 WHERE (b OPERATOR(pg_catalog.=) ANY (SELECT cte.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte)) DEBUG: Creating router plan UPDATE nullkey_c1_t1 SET b = 5 FROM reference_table WHERE EXISTS ( @@ -2351,13 +2338,6 @@ DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b IN (SELECT b FROM cte); DEBUG: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns DEBUG: generating subplan XXX_1 for CTE cte: SELECT reference_table.a, 1 AS b FROM (query_single_shard_table.distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a)) DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "distributed_table" to a subquery -DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a)) -DEBUG: Creating router plan DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM query_single_shard_table.nullkey_c1_t1 WHERE (b OPERATOR(pg_catalog.=) ANY (SELECT cte.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte)) DEBUG: Creating router plan DELETE FROM nullkey_c1_t1 USING reference_table WHERE EXISTS ( @@ -3314,7 +3294,8 @@ DEBUG: Creating router plan DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.nullkey_c2_t1 table_3 ORDER BY a LIMIT 0 DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count, avg(a) AS avg FROM (SELECT table_0.a FROM (query_single_shard_table.nullkey_c1_t1 table_0 RIGHT JOIN (SELECT table_2.a FROM ((SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) table_2 JOIN query_single_shard_table.nullkey_c2_t1 table_4 USING (a)) WHERE (table_4.a OPERATOR(pg_catalog.<) 8)) table_1 USING (a))) avgsub DEBUG: router planner does not support queries that reference non-colocated distributed tables -ERROR: cannot perform a lateral outer join when a distributed subquery references complex subqueries, CTEs or local tables +ERROR: cannot push down this subquery +DETAIL: nullkey_c1_t1 and nullkey_c2_t1 are not colocated -- test nested exec CREATE FUNCTION dist_query_single_shard(p_key int) RETURNS bigint diff --git a/src/test/regress/expected/recurring_join_pushdown.out b/src/test/regress/expected/recurring_join_pushdown.out new file mode 100644 index 000000000..4bc19fcea --- /dev/null +++ b/src/test/regress/expected/recurring_join_pushdown.out @@ -0,0 +1,983 @@ +CREATE SCHEMA recurring_join_pushdown; +SET search_path TO recurring_join_pushdown; +SET citus.next_shard_id TO 1520000; +SET citus.shard_count TO 4; +CREATE TABLE r1(a int, b int); +SELECT create_reference_table('r1'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO r1 VALUES (1,10), (1,11), (1,20), (2,10), (2,12), (2, 20), (3, 20), (10, 1), (10, 2); +CREATE TABLE d1(a int, b int); +SELECT create_distributed_table('d1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO d1 VALUES (1,10), (1,11), (1,20), (2,10), (2,12), (2, 20), (4, 10); +CREATE TABLE d2(a int, c text); +SELECT create_distributed_table('d2', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO d2(a, c) VALUES (1,'a'), (1,'b'), (1,'c'), (2,'d'), (2,'e'), (2,'f'), (4,'g'); +SET citus.shard_count TO 2; +CREATE TABLE d3_not_colocated(like d1); +SELECT create_distributed_table('d3_not_colocated', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO DEBUG3; +-- Basic test cases +-- Test that the join is pushed down to the worker nodes, using "using" syntax +SELECT count(*) FROM r1 LEFT JOIN d1 using (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 21 +(1 row) + +SELECT * FROM r1 LEFT JOIN d1 using (a, b) ORDER BY 1, 2; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + a | b +--------------------------------------------------------------------- + 1 | 10 + 1 | 11 + 1 | 20 + 2 | 10 + 2 | 12 + 2 | 20 + 3 | 20 + 10 | 1 + 10 | 2 +(9 rows) + +-- Disable the pushdown and verify that the join is not pushed down +SET citus.enable_recurring_outer_join_pushdown TO off; +SELECT count(*) FROM r1 LEFT JOIN d1 using (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "d1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" to a subquery +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_join_pushdown.d1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT d1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) d1_1) d1 USING (a)) +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 21 +(1 row) + +SET citus.enable_recurring_outer_join_pushdown TO on; +SET client_min_messages TO DEBUG1; +-- Test that the join is not pushed down when joined on a non-distributed column +SELECT count(*) FROM r1 LEFT JOIN d1 USING (b); +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "d1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT b FROM recurring_join_pushdown.d1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT NULL::integer AS a, d1_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) d1_1) d1 USING (b)) + count +--------------------------------------------------------------------- + 16 +(1 row) + +-- Test that the join is not pushed down when we have non-colocated tables in the RHS +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d3_not_colocated.b FROM d3_not_colocated FULL JOIN d1 ON d3_not_colocated.a = d1.a) AS t1 USING (a); +ERROR: cannot push down this subquery +DETAIL: d3_not_colocated and d1 are not colocated +-- The same error with its RIGHT JOIN variant +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d3_not_colocated.b FROM d3_not_colocated JOIN d1 ON d3_not_colocated.a = d1.a) AS t1 USING (a); +ERROR: cannot push down this subquery +DETAIL: d3_not_colocated and d1 are not colocated +-- Basic test cases with ON syntax +-- Test that the join is pushed down to the worker nodes, using "on" syntax +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.a = d1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 21 +(1 row) + +SELECT * FROM r1 LEFT JOIN d1 ON r1.a = d1.a AND r1.b = d1.b ORDER BY 1, 2; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + a | b | a | b +--------------------------------------------------------------------- + 1 | 10 | 1 | 10 + 1 | 11 | 1 | 11 + 1 | 20 | 1 | 20 + 2 | 10 | 2 | 10 + 2 | 12 | 2 | 12 + 2 | 20 | 2 | 20 + 3 | 20 | | + 10 | 1 | | + 10 | 2 | | +(9 rows) + +-- Verfiy that the join is pushed via the execution plan. +EXPLAIN (COSTS OFF) SELECT * FROM r1 LEFT JOIN d1 ON r1.a = d1.a AND r1.b = d1.b ORDER BY 1, 2; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + QUERY PLAN +--------------------------------------------------------------------- + Sort + Sort Key: remote_scan.a, remote_scan.b + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Merge Left Join + Merge Cond: ((r1.a = d1.a) AND (r1.b = d1.b)) + -> Sort + Sort Key: r1.a, r1.b + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) + -> Sort + Sort Key: d1.a, d1.b + -> Seq Scan on d1_1520001 d1 +(16 rows) + +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.b = d1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 13 +(1 row) + +-- Test that the join is not pushed down when joined on a non-distributed column +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.b = d1.b; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "d1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" to a subquery +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT b FROM recurring_join_pushdown.d1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT NULL::integer AS a, d1_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) d1_1) d1 ON ((r1.b OPERATOR(pg_catalog.=) d1.b))) +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 16 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.a = d1.b; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "d1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" to a subquery +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT b FROM recurring_join_pushdown.d1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT NULL::integer AS a, d1_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) d1_1) d1 ON ((r1.a OPERATOR(pg_catalog.=) d1.b))) +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 13 +(1 row) + +SET client_min_messages TO DEBUG1; +-- Test that the join is not pushed down when joined on a distributed column with disjunctive conditions +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.a = d1.a OR r1.b = d1.b; +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "d1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_join_pushdown.d1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT d1_1.a, d1_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) d1_1) d1 ON (((r1.a OPERATOR(pg_catalog.=) d1.a) OR (r1.b OPERATOR(pg_catalog.=) d1.b)))) + count +--------------------------------------------------------------------- + 28 +(1 row) + +-- Test join pushdown behavior when the inner part of the join is a subquery +-- Using 'using' syntax +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1) AS t1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 21 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1 WHERE a > 1) AS t1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 15 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM (SELECT * FROM d1) AS t1 WHERE a > 1) AS t2 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 15 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1 JOIN d1 as d1_1 USING (a)) AS t1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 57 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (d1 LEFT JOIN d1 as d1_1 USING (a)) AS t1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "d1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" to a subquery +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_join_pushdown.d1 WHERE true +DEBUG: recursively planning distributed relation "d1" "d1_1" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "d1" "d1_1" to a subquery +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_2 for subquery SELECT a FROM recurring_join_pushdown.d1 d1_1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN ((SELECT d1_2.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) d1_2) d1 LEFT JOIN (SELECT d1_1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) d1_1_1) d1_1 USING (a)) t1(a, b, b_1) USING (a)) +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 57 +(1 row) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1) AS t1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + QUERY PLAN +--------------------------------------------------------------------- + Aggregate + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Hash Right Join + Hash Cond: (d1.a = r1.a) + -> Seq Scan on d1_1520001 d1 + -> Hash + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) +(13 rows) + +-- Using 'on' syntax +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1) AS d1 ON r1.a = d1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 21 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1 WHERE a > 1) AS d1 ON r1.a = d1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 15 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM (SELECT * FROM d1) AS d1 WHERE a > 1) AS d1 ON r1.a = d1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 15 +(1 row) + +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a as a, d1.b, d1_1.a AS a_1 FROM d1 LEFT JOIN d1 as d1_1 ON d1.a = d1_1.a) AS d1_2 ON r1.a = d1_2.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 57 +(1 row) + +-- Nested joins +-- It is safe to push the inner join to compute t1. However, as the var of the inner table for the top level join (t1.a) resolves to r1.a, the outer join cannot be pushed down. +SELECT count(*) FROM r1 LEFT JOIN (SELECT r1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT r1.a, d1.b FROM (recurring_join_pushdown.r1 LEFT JOIN recurring_join_pushdown.d1 ON ((r1.a OPERATOR(pg_catalog.=) d1.a))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t1 ON ((r1.a OPERATOR(pg_catalog.=) t1.a))) +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 59 +(1 row) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT r1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT r1.a, d1.b FROM (recurring_join_pushdown.r1 LEFT JOIN recurring_join_pushdown.d1 ON ((r1.a OPERATOR(pg_catalog.=) d1.a))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_join_pushdown.r1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t1 ON ((r1.a OPERATOR(pg_catalog.=) t1.a))) +DEBUG: Creating router plan + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Hash Right Join + Hash Cond: (d1.a = r1.a) + -> Seq Scan on d1_1520001 d1 + -> Hash + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Merge Right Join + Merge Cond: (intermediate_result.a = r1.a) + -> Sort + Sort Key: intermediate_result.a + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: r1.a + -> Seq Scan on r1_1520000 r1 +(26 rows) + +-- In the following case, it is safe to push down both joins as t1.a resolves to d1.a. +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 57 +(1 row) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + QUERY PLAN +--------------------------------------------------------------------- + Aggregate + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Hash Right Join + Hash Cond: (d1.a = r1.a) + -> Hash Join + Hash Cond: (d1.a = r1_1.a) + -> Seq Scan on d1_1520001 d1 + -> Hash + -> Seq Scan on r1_1520000 r1_1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) + -> Hash + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) +(18 rows) + +-- In the following case, the lower level joins will be pushed down, but as the top level join is chained, subquery pushdown will not be applied at the top level. +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON t1.a = r1.a LEFT JOIN (SELECT d2.a, d2.c FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: a push down safe left join with recurring left side +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT d1.a, d1.b FROM (recurring_join_pushdown.r1 LEFT JOIN recurring_join_pushdown.d1 ON ((r1.a OPERATOR(pg_catalog.=) d1.a))) +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_2 for subquery SELECT d2.a, d2.c FROM (recurring_join_pushdown.r1 LEFT JOIN recurring_join_pushdown.d2 ON ((r1.a OPERATOR(pg_catalog.=) d2.a))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((recurring_join_pushdown.r1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t1 ON ((t1.a OPERATOR(pg_catalog.=) r1.a))) LEFT JOIN (SELECT intermediate_result.a, intermediate_result.c FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, c text)) t2 ON ((t1.a OPERATOR(pg_catalog.=) t2.a))) +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 489 +(1 row) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON t1.a = r1.a LEFT JOIN (SELECT d2.a, d2.c FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: a push down safe left join with recurring left side +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT d1.a, d1.b FROM (recurring_join_pushdown.r1 LEFT JOIN recurring_join_pushdown.d1 ON ((r1.a OPERATOR(pg_catalog.=) d1.a))) +DEBUG: recursively planning right side of the left join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_2 for subquery SELECT d2.a, d2.c FROM (recurring_join_pushdown.r1 LEFT JOIN recurring_join_pushdown.d2 ON ((r1.a OPERATOR(pg_catalog.=) d2.a))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((recurring_join_pushdown.r1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t1 ON ((t1.a OPERATOR(pg_catalog.=) r1.a))) LEFT JOIN (SELECT intermediate_result.a, intermediate_result.c FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, c text)) t2 ON ((t1.a OPERATOR(pg_catalog.=) t2.a))) +DEBUG: Creating router plan + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Hash Right Join + Hash Cond: (d1.a = r1.a) + -> Seq Scan on d1_1520001 d1 + -> Hash + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) + -> Distributed Subplan XXX_2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Hash Right Join + Hash Cond: (d2.a = r1.a) + -> Seq Scan on d2_1520005 d2 + -> Hash + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Merge Left Join + Merge Cond: (r1.a = intermediate_result.a) + -> Sort + Sort Key: r1.a + -> Seq Scan on r1_1520000 r1 + -> Materialize + -> Merge Left Join + Merge Cond: (intermediate_result.a = intermediate_result_1.a) + -> Sort + Sort Key: intermediate_result.a + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: intermediate_result_1.a + -> Function Scan on read_intermediate_result intermediate_result_1 +(44 rows) + +--- As both subqueries are pushed and the top level join is over their results on distribution colums, the query is pushed down as a whole. +SELECT count(*) FROM (SELECT d1_1.a, r1.b FROM r1 LEFT JOIN d1 as d1_1 ON r1.a = d1_1.a) AS t1 LEFT JOIN + (SELECT d2.a, d2.c, r1.b FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 165 +(1 row) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM (SELECT d1_1.a, r1.b FROM r1 LEFT JOIN d1 as d1_1 ON r1.a = d1_1.a) AS t1 LEFT JOIN + (SELECT d2.a, d2.c, r1.b FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe left join with recurring left side +DEBUG: a push down safe left join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: no shard pruning constraints on d2 found +DEBUG: shard count after pruning for d2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + QUERY PLAN +--------------------------------------------------------------------- + Aggregate + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Merge Right Join + Merge Cond: (d2.a = d1_1.a) + -> Merge Join + Merge Cond: (r1_1.a = d2.a) + -> Sort + Sort Key: r1_1.a + -> Seq Scan on r1_1520000 r1_1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) + -> Sort + Sort Key: d2.a + -> Seq Scan on d2_1520005 d2 + -> Sort + Sort Key: d1_1.a + -> Hash Right Join + Hash Cond: (d1_1.a = r1.a) + -> Seq Scan on d1_1520001 d1_1 + -> Hash + -> Seq Scan on r1_1520000 r1 + Filter: ((a IS NULL) OR ((btint4cmp('-2147483648'::integer, hashint4(a)) < 0) AND (btint4cmp(hashint4(a), '-1073741825'::integer) <= 0))) +(26 rows) + + -- Basic cases with RIGHT JOIN +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM d1 RIGHT JOIN r1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe right join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 21 +(1 row) + +SELECT count(*) FROM (SELECT * FROM d1) AS t1 RIGHT JOIN r1 USING (a); +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe right join with recurring left side +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + count +--------------------------------------------------------------------- + 21 +(1 row) + +SET client_min_messages TO ERROR; +-- Ensure that even when CTEs are replaced, we insert push +-- down conditions for outer joins when necessary. +WITH cte_0 AS ( SELECT table_0.a FROM d1 AS table_0 WHERE table_0.a IN ( SELECT table_1.a FROM d1 AS table_1 ORDER BY a LIMIT 2 ) ORDER BY a ) SELECT count(*), avg(avgsub.a) FROM ( SELECT table_2.a FROM cte_0 AS table_2 RIGHT JOIN r1 AS table_3 USING (a)) AS avgsub; + count | avg +--------------------------------------------------------------------- + 15 | 1.00000000000000000000 +(1 row) + +-- The following queries trigger recursive computing, recurring outer-join push down +-- methods introduced in#7973 can be enhanced to cover these cases in the future. +CREATE TABLE r1_local AS SELECT * FROM r1; +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1_local LEFT JOIN d1 ON r1_local.a = d1.a; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Seq Scan on r1_local + -> Distributed Subplan XXX_2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on d1_1520001 d1 + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Merge Left Join + Merge Cond: (intermediate_result.a = intermediate_result_1.a) + -> Sort + Sort Key: intermediate_result.a + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: intermediate_result_1.a + -> Function Scan on read_intermediate_result intermediate_result_1 +(23 rows) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM (SELECT * FROM r1) sq LEFT JOIN d1 ON sq.a = d1.a; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on d1_1520001 d1 + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Merge Right Join + Merge Cond: (intermediate_result.a = r1.a) + -> Sort + Sort Key: intermediate_result.a + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: r1.a + -> Seq Scan on r1_1520000 r1 +(21 rows) + +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (d1 INNER JOIN d2 on d1.a = d2.a) on r1.a = d2.a; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on d1_1520001 d1 + -> Distributed Subplan XXX_2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on d2_1520005 d2 + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Aggregate + -> Merge Left Join + Merge Cond: (r1.a = intermediate_result_1.a) + -> Sort + Sort Key: r1.a + -> Seq Scan on r1_1520000 r1 + -> Materialize + -> Merge Join + Merge Cond: (intermediate_result.a = intermediate_result_1.a) + -> Sort + Sort Key: intermediate_result.a + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: intermediate_result_1.a + -> Function Scan on read_intermediate_result intermediate_result_1 +(34 rows) + +SET client_min_messages TO DEBUG3; +-- The following query is recursively computed due to the lateral join in the subquery. +-- Leaving the inner side of the query to query push down causes an error in deferred error +-- checks even if it is possible to push down the whole query. +SELECT DISTINCT sq.a FROM ( + SELECT d1.a FROM d1 JOIN LATERAL ( SELECT * FROM r1 WHERE r1.a = d1.a ) sq2 ON true ) AS sq RIGHT JOIN r1 USING (a) +ORDER BY sq.a +LIMIT 1; +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: recursively planning left side of the right join since the outer side is a recurring rel +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on d1 found +DEBUG: shard count after pruning for d1: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT d1.a FROM (recurring_join_pushdown.d1 JOIN LATERAL (SELECT r1.a, r1.b FROM recurring_join_pushdown.r1 WHERE (r1.a OPERATOR(pg_catalog.=) d1.a)) sq2 ON (true)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT DISTINCT sq.a FROM ((SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) sq RIGHT JOIN recurring_join_pushdown.r1 USING (a)) ORDER BY sq.a LIMIT 1 +DEBUG: Creating router plan + a +--------------------------------------------------------------------- + 1 +(1 row) + +SET client_min_messages TO ERROR; +DROP SCHEMA recurring_join_pushdown CASCADE; diff --git a/src/test/regress/expected/recurring_outer_join.out b/src/test/regress/expected/recurring_outer_join.out index e020c6814..c212a8f9f 100644 --- a/src/test/regress/expected/recurring_outer_join.out +++ b/src/test/regress/expected/recurring_outer_join.out @@ -142,33 +142,18 @@ DEBUG: verifying table "dist_5_with_pkey" -- basic cases -- SELECT COUNT(*) FROM ref_1 LEFT JOIN dist_1 USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 LEFT JOIN (SELECT dist_1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dist_1_1) dist_1 USING (a)) count --------------------------------------------------------------------- 28 (1 row) SELECT COUNT(*) FROM ref_1 LEFT JOIN dist_1 USING (a,b); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 LEFT JOIN (SELECT dist_1_1.a, dist_1_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) dist_1_1) dist_1 USING (a, b)) count --------------------------------------------------------------------- 15 (1 row) SELECT COUNT(*) FROM dist_1 RIGHT JOIN ref_1 USING (a); -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT dist_1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dist_1_1) dist_1 RIGHT JOIN recurring_outer_join.ref_1 USING (a)) count --------------------------------------------------------------------- 28 @@ -209,10 +194,6 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c -- distributed side is a subquery SELECT COUNT(*) FROM ref_1 LEFT JOIN (SELECT * FROM dist_1) q USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) q USING (a)) count --------------------------------------------------------------------- 28 @@ -356,7 +337,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c 43 (1 row) - -- subqury without FROM + -- subquery without FROM SELECT COUNT(*) FROM dist_1 t1 RIGHT JOIN (SELECT generate_series(1,10) AS a) t2 USING (a); DEBUG: recursively planning left side of the right join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t1" since it is part of a distributed join node that is outer joined with a recurring rel @@ -530,7 +511,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c 15 (1 row) - -- subqury without FROM + -- subquery without FROM SELECT COUNT(*) FROM (SELECT generate_series(1,10) AS a) t1 JOIN dist_1 t2 @@ -545,21 +526,11 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c SELECT COUNT(*) FROM ref_1 t1 LEFT JOIN dist_1 t2 USING (a,b) WHERE EXISTS (SELECT * FROM dist_1 t3 WHERE t1.a = t3.a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t2" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t2" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t2 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t2_1.a, t2_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t2_1) t2 USING (a, b)) WHERE (EXISTS (SELECT t3.a, t3.b FROM recurring_outer_join.dist_1 t3 WHERE (t1.a OPERATOR(pg_catalog.=) t3.a))) -ERROR: correlated subqueries are not supported when the FROM clause contains a reference table +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns SELECT COUNT(*) FROM dist_1 t1 RIGHT JOIN ref_1 t2 USING (a,b) WHERE EXISTS (SELECT * FROM dist_1 t3 WHERE t2.a = t3.a); -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t1" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t1" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t1 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT t1_1.a, t1_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t1_1) t1 RIGHT JOIN recurring_outer_join.ref_1 t2 USING (a, b)) WHERE (EXISTS (SELECT t3.a, t3.b FROM recurring_outer_join.dist_1 t3 WHERE (t2.a OPERATOR(pg_catalog.=) t3.a))) -ERROR: correlated subqueries are not supported when the FROM clause contains a CTE or subquery +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns -- "dist_1 t2" can't contribute to result set of the right join with -- a tuple having "(t2.a) a = NULL" because t2 is in the inner side of -- right join. For this reason, Postgres knows that can @@ -592,40 +563,69 @@ LATERAL --------------------------------------------------------------------- (0 rows) --- Qual is the same but top-level join is an anti-join. Right join --- stays as is and hence requires recursive planning. +-- Qual is the same but top-level join is an anti-join. +-- The right join between t2 and t3 is pushed down. +-- Citus determines that the whole query can be pushed down +-- due to the equality constraint between two distributed +-- tables t1 and t2. SELECT COUNT(*) FROM dist_1 t1 WHERE NOT EXISTS ( SELECT * FROM dist_1 t2 RIGHT JOIN ref_1 t3 USING (a) WHERE t2.a = t1.a ); -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t2" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t2" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t2 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM recurring_outer_join.dist_1 t1 WHERE (NOT (EXISTS (SELECT t3.a, t2.b, t3.b FROM ((SELECT t2_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t2_1) t2 RIGHT JOIN recurring_outer_join.ref_1 t3 USING (a)) WHERE (t2.a OPERATOR(pg_catalog.=) t1.a)))) count --------------------------------------------------------------------- 8 (1 row) +SET client_min_messages TO DEBUG3; -- This time the semi-join qual is (not <) -- where t3 is the outer rel of the right join. Hence Postgres can't --- replace right join with an inner join and so we recursively plan --- inner side of the right join since the outer side is a recurring --- rel. +-- replace right join with an inner join. +-- Citus pushes down the right join between t2 and t3 with constraints on +-- the recurring outer part (t3). However, it cannnot push down the whole +-- query as it can not establish an equivalence between the distribution +-- tables t1 and t2. Hence, Citus tries to recursively plan the subquery. +-- This attempt fails since the subquery has a reference to outer query. +-- See #8113 SELECT COUNT(*) FROM dist_1 t1 WHERE EXISTS ( SELECT * FROM dist_1 t2 RIGHT JOIN ref_1 t3 USING (a) WHERE t3.a = t1.a ); -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t2" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t2" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t2 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM recurring_outer_join.dist_1 t1 WHERE (EXISTS (SELECT t3.a, t2.b, t3.b FROM ((SELECT t2_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t2_1) t2 RIGHT JOIN recurring_outer_join.ref_1 t3 USING (a)) WHERE (t3.a OPERATOR(pg_catalog.=) t1.a))) +DEBUG: no shard pruning constraints on dist_1 found +DEBUG: shard count after pruning for dist_1: 32 +DEBUG: no shard pruning constraints on dist_1 found +DEBUG: shard count after pruning for dist_1: 32 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe right join with recurring left side +DEBUG: skipping recursive planning for the subquery since it contains references to outer queries +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +SELECT COUNT(*) FROM dist_1 t1 +WHERE NOT EXISTS ( + SELECT * FROM dist_1 t2 + RIGHT JOIN ref_1 t3 USING (a) + WHERE t3.a = t1.a +); +DEBUG: no shard pruning constraints on dist_1 found +DEBUG: shard count after pruning for dist_1: 32 +DEBUG: no shard pruning constraints on dist_1 found +DEBUG: shard count after pruning for dist_1: 32 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: a push down safe right join with recurring left side +DEBUG: skipping recursive planning for the subquery since it contains references to outer queries +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +SET client_min_messages TO DEBUG1; +-- Force recursive planning of the right join with offset +SELECT COUNT(*) FROM dist_1 t1 +WHERE EXISTS ( + SELECT * FROM dist_1 t2 + RIGHT JOIN ref_1 t3 USING (a) + WHERE t3.a = t1.a + OFFSET 0 +); count --------------------------------------------------------------------- 7 @@ -636,12 +636,8 @@ WHERE NOT EXISTS ( SELECT * FROM dist_1 t2 RIGHT JOIN ref_1 t3 USING (a) WHERE t3.a = t1.a + OFFSET 0 ); -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t2" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t2" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t2 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM recurring_outer_join.dist_1 t1 WHERE (NOT (EXISTS (SELECT t3.a, t2.b, t3.b FROM ((SELECT t2_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t2_1) t2 RIGHT JOIN recurring_outer_join.ref_1 t3 USING (a)) WHERE (t3.a OPERATOR(pg_catalog.=) t1.a)))) count --------------------------------------------------------------------- 8 @@ -685,7 +681,8 @@ LEFT JOIN ( dist_1 t4 JOIN - -- 1) t6 is recursively planned since the outer side is recurring + -- 1) t6 is not recursively planned since it is + -- safe to push down the recurring outer side with constraints (SELECT t6.a FROM dist_1 t6 RIGHT JOIN ref_1 t7 USING(a)) t5 USING(a) ) q @@ -696,19 +693,17 @@ LEFT JOIN dist_1 t8 USING (a) WHERE t8.b IS NULL; -DEBUG: recursively planning left side of the right join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t6" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t6" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t6 WHERE true DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t4" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t4" to a subquery -DEBUG: generating subplan XXX_2 for subquery SELECT a FROM recurring_outer_join.dist_1 t4 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t4 WHERE true +DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: generating subplan XXX_2 for subquery SELECT t6.a FROM (recurring_outer_join.dist_1 t6 RIGHT JOIN recurring_outer_join.ref_1 t7 USING (a)) DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t8" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t8" to a subquery DEBUG: generating subplan XXX_3 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t8 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((recurring_outer_join.ref_1 t1 LEFT JOIN ((SELECT t4_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t4_1) t4 JOIN (SELECT t6.a FROM ((SELECT t6_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t6_1) t6 RIGHT JOIN recurring_outer_join.ref_1 t7 USING (a))) t5 USING (a)) q USING (a)) LEFT JOIN (SELECT t8_1.a, t8_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t8_1) t8 USING (a)) WHERE (t8.b IS NULL) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((recurring_outer_join.ref_1 t1 LEFT JOIN ((SELECT t4_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t4_1) t4 JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t5 USING (a)) q USING (a)) LEFT JOIN (SELECT t8_1.a, t8_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t8_1) t8 USING (a)) WHERE (t8.b IS NULL) count --------------------------------------------------------------------- 10 @@ -790,13 +785,10 @@ LEFT JOIN USING(a) ) t6 USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t5" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t5" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t5 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN recurring_outer_join.dist_1 t5 USING (a)) DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_1 t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN (SELECT t5_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t5_1) t5 USING (a))) q(a, b, b_1) USING (a)) +DEBUG: generating subplan XXX_2 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_1 t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) q(a, b, b_1) USING (a)) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t6(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- @@ -834,13 +826,10 @@ LEFT JOIN USING(a) ) t6 USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t5" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t5" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t5 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN recurring_outer_join.dist_1 t5 USING (a)) DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_3_partitioned t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN (SELECT t5_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t5_1) t5 USING (a))) q(a, b, b_1) USING (a)) +DEBUG: generating subplan XXX_2 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_3_partitioned t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) q(a, b, b_1) USING (a)) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t6(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- @@ -864,13 +853,10 @@ USING (a) GROUP BY (t1.b) HAVING t1.b > 200 ORDER BY 1,2; -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t5" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t5" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t5 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN recurring_outer_join.dist_1 t5 USING (a)) DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_3_partitioned t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN (SELECT t5_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t5_1) t5 USING (a))) q(a, b, b_1) USING (a)) +DEBUG: generating subplan XXX_2 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_3_partitioned t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) q(a, b, b_1) USING (a)) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(t1.a) AS count, t1.b FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t6(a, b, b_1, b_2, b_3) USING (a)) GROUP BY t1.b HAVING (t1.b OPERATOR(pg_catalog.>) 200) ORDER BY (count(t1.a)), t1.b count | b --------------------------------------------------------------------- @@ -909,19 +895,12 @@ HAVING ( ) ) ORDER BY 1,2; -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t7" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t7" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t7 WHERE (b OPERATOR(pg_catalog.>) 10) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t5" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t5" to a subquery -DEBUG: generating subplan XXX_2 for subquery SELECT a FROM recurring_outer_join.dist_1 t5 WHERE true -DEBUG: generating subplan XXX_3 for subquery SELECT t6.a, t6.b, t7.b FROM (recurring_outer_join.ref_1 t6 LEFT JOIN (SELECT t7_1.a, t7_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t7_1) t7 USING (a)) WHERE (t7.b OPERATOR(pg_catalog.>) 10) +DEBUG: generating subplan XXX_1 for subquery SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN recurring_outer_join.dist_1 t5 USING (a)) +DEBUG: generating subplan XXX_2 for subquery SELECT t6.a, t6.b, t7.b FROM (recurring_outer_join.ref_1 t6 LEFT JOIN recurring_outer_join.dist_1 t7 USING (a)) WHERE (t7.b OPERATOR(pg_catalog.>) 10) DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_4 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_3_partitioned t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN (SELECT t5_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t5_1) t5 USING (a))) q(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(t1.a) AS count, t1.b FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t6(a, b, b_1, b_2, b_3) USING (a)) GROUP BY t1.b HAVING (EXISTS (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer))) ORDER BY (count(t1.a)), t1.b +DEBUG: generating subplan XXX_3 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_3_partitioned t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) q(a, b, b_1) USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(t1.a) AS count, t1.b FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t6(a, b, b_1, b_2, b_3) USING (a)) GROUP BY t1.b HAVING (EXISTS (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer))) ORDER BY (count(t1.a)), t1.b count | b --------------------------------------------------------------------- 0 | 102 @@ -955,15 +934,12 @@ LEFT JOIN USING(a) ) t6 USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t5" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t5" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t5 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN recurring_outer_join.dist_1 t5 USING (a)) DEBUG: Wrapping relation "citus_local_1" "t1" to a subquery DEBUG: generating subplan XXX_2 for subquery SELECT a FROM recurring_outer_join.citus_local_1 t1 WHERE true DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_3 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_1 t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT t4.a, t4.b, t5.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN (SELECT t5_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t5_1) t5 USING (a))) q(a, b, b_1) USING (a)) +DEBUG: generating subplan XXX_3 for subquery SELECT p.a, p.b, p.b_1 AS b, q.b, q.b_1 AS b FROM ((SELECT t2.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 JOIN recurring_outer_join.dist_1 t3 USING (a)) WHERE (t3.b IS NULL)) p(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) q(a, b, b_1) USING (a)) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t1_1) t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t6(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- @@ -1125,8 +1101,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c (1 row) -- cannot recursively plan because t3 (inner - distributed) --- references t1 (outer - recurring) -SELECT COUNT(*) FROM ref_1 t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (a); +-- references t1 (outer - recurring over non-distribution column) +SELECT COUNT(*) FROM ref_1 t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (b); DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel ERROR: cannot perform a lateral outer join when a distributed subquery references a reference table @@ -1179,10 +1155,6 @@ LEFT JOIN ) ) t3 USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t2 WHERE (EXISTS (SELECT t4.a, t4.b FROM recurring_outer_join.dist_1 t4 WHERE (t4.a OPERATOR(pg_catalog.=) t2.a))) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t3 USING (a)) count --------------------------------------------------------------------- 28 @@ -1197,10 +1169,6 @@ SELECT COUNT(*) FROM ref_1 t1 LEFT JOIN my_view_1 t3 USING (a); -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 table_name_for_view WHERE (EXISTS (SELECT t4.a, t4.b FROM recurring_outer_join.dist_1 t4 WHERE (t4.a OPERATOR(pg_catalog.=) table_name_for_view.a))) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t3 USING (a)) count --------------------------------------------------------------------- 28 @@ -1345,27 +1313,21 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c (1 row) SELECT COUNT(*) FROM ref_1 t1 --- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted --- to a recurring rel too. For this reason, subquery t8 is recursively --- planned because t7 is recurring already. +-- 2) It is also safe to push down this since the recurring outer side t1 and +-- distributed inner side t8 are joined on the distribution column. LEFT JOIN ( SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 JOIN - -- 1) subquery t6 is recursively planned because t5 is recurring + -- 1) it is safe to push down subquery t7 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < 150) t6 USING (a)) as t7 USING(a) ) t8 USING (a); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 150) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 150)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 514 @@ -1374,28 +1336,22 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c -- same test using a prepared statement PREPARE recurring_outer_join_p1 AS SELECT COUNT(*) FROM ref_1 t1 --- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted --- to a recurring rel too. For this reason, subquery t8 is recursively --- planned because t7 is recurring already. +-- 2) It is also safe to push down this since the recurring outer side t1 and +-- distributed inner side t8 are joined on the distribution column. LEFT JOIN ( SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 JOIN - -- 1) subquery t6 is recursively planned because t5 is recurring + -- 1) it is safe to push down subquery t7 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < $1) t6 USING (a)) as t7 USING(a) ) t8 USING (a); EXECUTE recurring_outer_join_p1(0); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 0) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 0)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 142 @@ -1403,14 +1359,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c EXECUTE recurring_outer_join_p1(100); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 100) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 100)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 514 @@ -1418,14 +1369,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c EXECUTE recurring_outer_join_p1(100); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 100) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 100)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 514 @@ -1433,14 +1379,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c EXECUTE recurring_outer_join_p1(10); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 10) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 10)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 142 @@ -1448,14 +1389,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c EXECUTE recurring_outer_join_p1(10); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 10) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 10)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 142 @@ -1464,14 +1400,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c EXECUTE recurring_outer_join_p1(1000); DEBUG: pathlist hook for columnar table am DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 1000) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 1000)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 514 @@ -1479,14 +1410,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c EXECUTE recurring_outer_join_p1(1000); DEBUG: pathlist hook for columnar table am -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: pathlist hook for columnar table am -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_2_columnar WHERE (b OPERATOR(pg_catalog.<) 1000) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_2 for subquery SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t6 USING (a))) t7(a, b, b_1) USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b, intermediate_result.b_2 AS b, intermediate_result.b_3 AS b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer, b_2 integer, b_3 integer)) t8(a, b, b_1, b_2, b_3) USING (a)) +DEBUG: generating subplan XXX_1 for subquery SELECT t5.a, t5.b, t6.b FROM (recurring_outer_join.ref_1 t5 LEFT JOIN (SELECT dist_2_columnar.a, dist_2_columnar.b FROM recurring_outer_join.dist_2_columnar WHERE (dist_2_columnar.b OPERATOR(pg_catalog.<) 1000)) t6 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t4.a, t4.b, t4.b_1 AS b, t7.b, t7.b_1 AS b FROM ((SELECT t3.a, t2.b, t3.b FROM (recurring_outer_join.ref_1 t2 RIGHT JOIN recurring_outer_join.dist_1 t3 USING (a))) t4(a, b, b_1) JOIN (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.b_1 AS b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, b_1 integer)) t7(a, b, b_1) USING (a))) t8(a, b, b_1, b_2, b_3) USING (a)) count --------------------------------------------------------------------- 514 @@ -1655,7 +1581,7 @@ SELECT * FROM ref_1 t36 WHERE (b,100,a) IN ( DISTINCT t31.b, -- 1) we first search for such joins in the target list and recursively plan t33 -- because t32 is recurring - (SELECT max(b) FROM ref_1 t32 LEFT JOIN dist_1 t33 USING(a,b) WHERE t31.a = t32.a), + (SELECT max(b) FROM ref_1 t32 LEFT JOIN dist_1 t33 USING(b) WHERE t31.a = t32.a), (SELECT t34.a) FROM ref_1 t35 LEFT JOIN dist_1 t31 USING (a,b) @@ -1706,16 +1632,13 @@ DEBUG: CTE cte_1_inner_cte is going to be inlined via distributed planning DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t33" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t33" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t33 WHERE true -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t12" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t12" to a subquery -DEBUG: generating subplan XXX_2 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t12 WHERE true +DEBUG: generating subplan XXX_1 for subquery SELECT b FROM recurring_outer_join.dist_1 t33 WHERE true +DEBUG: generating subplan XXX_2 for subquery SELECT t11.a, t11.b FROM (recurring_outer_join.ref_1 t11 LEFT JOIN recurring_outer_join.dist_1 t12 USING (a, b)) DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t14" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t14" to a subquery DEBUG: generating subplan XXX_3 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t14 WHERE true -DEBUG: generating subplan XXX_4 for subquery SELECT t13.a, t13.b FROM ((recurring_outer_join.ref_1 t13 LEFT JOIN (SELECT t14_1.a, t14_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t14_1) t14 USING (a, b)) JOIN (SELECT t11.a, t11.b FROM (recurring_outer_join.ref_1 t11 LEFT JOIN (SELECT t12_1.a, t12_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t12_1) t12 USING (a, b))) t15 USING (a, b)) OFFSET 0 +DEBUG: generating subplan XXX_4 for subquery SELECT t13.a, t13.b FROM ((recurring_outer_join.ref_1 t13 LEFT JOIN (SELECT t14_1.a, t14_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t14_1) t14 USING (a, b)) JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t15 USING (a, b)) OFFSET 0 DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "d" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "d" to a subquery @@ -1765,23 +1688,15 @@ DEBUG: recursively planning right side of the left join since the outer side is DEBUG: recursively planning distributed relation "dist_1" "t34" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t34" to a subquery DEBUG: generating subplan XXX_17 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t34 WHERE true -DEBUG: generating subplan XXX_18 for subquery SELECT DISTINCT t31.b, (SELECT max(t32.b) AS max FROM (recurring_outer_join.ref_1 t32 LEFT JOIN (SELECT t33_1.a, t33_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t33_1) t33 USING (a, b)) WHERE (t31.a OPERATOR(pg_catalog.=) t32.a)) AS max, (SELECT t34.a) AS a FROM ((((recurring_outer_join.ref_1 t35 LEFT JOIN (SELECT t31_1.a, t31_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_16'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t31_1) t31 USING (a, b)) LEFT JOIN (SELECT t34_1.a, t34_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_17'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t34_1) t34 USING (a, b)) LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte_1 USING (a, b)) LEFT JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_15'::text, 'binary'::citus_copy_format) intermediate_result(a bigint)) t30 ON ((t30.a OPERATOR(pg_catalog.=) cte_1.a))) ORDER BY t31.b, (SELECT max(t32.b) AS max FROM (recurring_outer_join.ref_1 t32 LEFT JOIN (SELECT t33_1.a, t33_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t33_1) t33 USING (a, b)) WHERE (t31.a OPERATOR(pg_catalog.=) t32.a)), (SELECT t34.a) -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t3" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t3" to a subquery -DEBUG: generating subplan XXX_19 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t3 WHERE true +DEBUG: generating subplan XXX_18 for subquery SELECT DISTINCT t31.b, (SELECT max(t32.b) AS max FROM (recurring_outer_join.ref_1 t32 LEFT JOIN (SELECT NULL::integer AS a, t33_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) t33_1) t33 USING (b)) WHERE (t31.a OPERATOR(pg_catalog.=) t32.a)) AS max, (SELECT t34.a) AS a FROM ((((recurring_outer_join.ref_1 t35 LEFT JOIN (SELECT t31_1.a, t31_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_16'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t31_1) t31 USING (a, b)) LEFT JOIN (SELECT t34_1.a, t34_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_17'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t34_1) t34 USING (a, b)) LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte_1 USING (a, b)) LEFT JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_15'::text, 'binary'::citus_copy_format) intermediate_result(a bigint)) t30 ON ((t30.a OPERATOR(pg_catalog.=) cte_1.a))) ORDER BY t31.b, (SELECT max(t32.b) AS max FROM (recurring_outer_join.ref_1 t32 LEFT JOIN (SELECT NULL::integer AS a, t33_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) t33_1) t33 USING (b)) WHERE (t31.a OPERATOR(pg_catalog.=) t32.a)), (SELECT t34.a) DEBUG: push down of limit count: 10 -DEBUG: generating subplan XXX_20 for subquery SELECT b, (SELECT t2.a FROM (recurring_outer_join.ref_1 t2 LEFT JOIN (SELECT t3_1.a, t3_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_19'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t3_1) t3 USING (a, b)) WHERE (t2.a OPERATOR(pg_catalog.=) t1.a) ORDER BY t2.a LIMIT 1) AS a FROM recurring_outer_join.dist_1 t1 ORDER BY b, (SELECT t2.a FROM (recurring_outer_join.ref_1 t2 LEFT JOIN (SELECT t3_1.a, t3_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_19'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t3_1) t3 USING (a, b)) WHERE (t2.a OPERATOR(pg_catalog.=) t1.a) ORDER BY t2.a LIMIT 1) LIMIT 10 -DEBUG: recursively planning right side of the left join since the outer side is a recurring rel -DEBUG: recursively planning distributed relation "dist_1" "t5" since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: Wrapping relation "dist_1" "t5" to a subquery -DEBUG: generating subplan XXX_21 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t5 WHERE true -DEBUG: generating subplan XXX_22 for subquery SELECT a, b FROM (SELECT t8.a, t8.b FROM (SELECT t7.a, t7.b FROM (SELECT t6.a, t6.b FROM (SELECT t4.a, t4.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN (SELECT t5_1.a, t5_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_21'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t5_1) t5 USING (a, b))) t6) t7) t8) t9 OFFSET 0 +DEBUG: generating subplan XXX_19 for subquery SELECT b, (SELECT t2.a FROM (recurring_outer_join.ref_1 t2 LEFT JOIN recurring_outer_join.dist_1 t3 USING (a, b)) WHERE (t2.a OPERATOR(pg_catalog.=) t1.a) ORDER BY t2.a LIMIT 1) AS a FROM recurring_outer_join.dist_1 t1 ORDER BY b, (SELECT t2.a FROM (recurring_outer_join.ref_1 t2 LEFT JOIN recurring_outer_join.dist_1 t3 USING (a, b)) WHERE (t2.a OPERATOR(pg_catalog.=) t1.a) ORDER BY t2.a LIMIT 1) LIMIT 10 +DEBUG: generating subplan XXX_20 for subquery SELECT a, b FROM (SELECT t8.a, t8.b FROM (SELECT t7.a, t7.b FROM (SELECT t6.a, t6.b FROM (SELECT t4.a, t4.b FROM (recurring_outer_join.ref_1 t4 LEFT JOIN recurring_outer_join.dist_1 t5 USING (a, b))) t6) t7) t8) t9 OFFSET 0 DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t11" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t11" to a subquery -DEBUG: generating subplan XXX_23 for subquery SELECT b FROM recurring_outer_join.dist_1 t11 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM recurring_outer_join.ref_1 t36 WHERE (((b, 100, a) OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.b, intermediate_result.max, intermediate_result.a FROM read_intermediate_result('XXX_18'::text, 'binary'::citus_copy_format) intermediate_result(b integer, max integer, a integer))) AND (NOT (a OPERATOR(pg_catalog.=) ANY (SELECT outer_cte_1.b FROM ((SELECT intermediate_result.b, intermediate_result.a FROM read_intermediate_result('XXX_20'::text, 'binary'::citus_copy_format) intermediate_result(b integer, a integer)) outer_cte_1 LEFT JOIN (SELECT t10.b, t10.a, t11.a FROM ((SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_22'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t10 LEFT JOIN (SELECT NULL::integer AS a, t11_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_23'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) t11_1) t11 USING (b))) outer_cte_2(b, a, a_1) USING (b)))))) +DEBUG: generating subplan XXX_21 for subquery SELECT b FROM recurring_outer_join.dist_1 t11 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM recurring_outer_join.ref_1 t36 WHERE (((b, 100, a) OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.b, intermediate_result.max, intermediate_result.a FROM read_intermediate_result('XXX_18'::text, 'binary'::citus_copy_format) intermediate_result(b integer, max integer, a integer))) AND (NOT (a OPERATOR(pg_catalog.=) ANY (SELECT outer_cte_1.b FROM ((SELECT intermediate_result.b, intermediate_result.a FROM read_intermediate_result('XXX_19'::text, 'binary'::citus_copy_format) intermediate_result(b integer, a integer)) outer_cte_1 LEFT JOIN (SELECT t10.b, t10.a, t11.a FROM ((SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_20'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t10 LEFT JOIN (SELECT NULL::integer AS a, t11_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_21'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) t11_1) t11 USING (b))) outer_cte_2(b, a, a_1) USING (b)))))) a | b --------------------------------------------------------------------- 1 | 11 @@ -1899,10 +1814,10 @@ BEGIN; SELECT t1.a, t1.b FROM ref_1 t1 LEFT JOIN ( - SELECT * FROM dist_1 t2 WHERE EXISTS ( + SELECT DISTINCT ON (a) * FROM dist_1 t2 WHERE EXISTS ( SELECT * FROM dist_1 t4 WHERE t4.a = t2.a - ) + ) ORDER BY a, b ) t3 USING (a) ) q @@ -1910,7 +1825,7 @@ BEGIN; RETURNING *; DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel -DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM recurring_outer_join.dist_1 t2 WHERE (EXISTS (SELECT t4.a, t4.b FROM recurring_outer_join.dist_1 t4 WHERE (t4.a OPERATOR(pg_catalog.=) t2.a))) +DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT ON (a) a, b FROM recurring_outer_join.dist_1 t2 WHERE (EXISTS (SELECT t4.a, t4.b FROM recurring_outer_join.dist_1 t4 WHERE (t4.a OPERATOR(pg_catalog.=) t2.a))) ORDER BY a, b DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM recurring_outer_join.dist_5 USING (SELECT t1.a, t1.b FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) t3 USING (a))) q WHERE (dist_5.a OPERATOR(pg_catalog.=) q.a) RETURNING dist_5.a, dist_5.b, q.a, q.b a | b | a | b --------------------------------------------------------------------- @@ -1955,7 +1870,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE recurring_out (7 rows) ROLLBACK; --- INSERT .. SELECT: pull to coordinator +-- INSERT .. SELECT: Repartitioned BEGIN; DELETE FROM ref_1 WHERE a IS NULL; INSERT INTO dist_1 @@ -1964,11 +1879,22 @@ BEGIN; LEFT JOIN dist_1 t2 ON (t1.a = t2.a); DEBUG: cannot perform a lateral outer join when a distributed subquery references a reference table +DEBUG: performing repartitioned INSERT ... SELECT +ROLLBACK; +-- INSERT .. SELECT: pull to coordinator +BEGIN; + DELETE FROM ref_1 WHERE a IS NULL; + INSERT INTO dist_1 + SELECT t1.* + FROM ref_1 t1 + LEFT JOIN dist_1 t2 + ON (t1.b = t2.b); +DEBUG: cannot perform a lateral outer join when a distributed subquery references a reference table DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "dist_1" "t2" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "dist_1" "t2" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t2 WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT t1.a, t1.b FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT t2_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t2_1) t2 ON ((t1.a OPERATOR(pg_catalog.=) t2.a))) +DEBUG: generating subplan XXX_1 for subquery SELECT b FROM recurring_outer_join.dist_1 t2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT t1.a, t1.b FROM (recurring_outer_join.ref_1 t1 LEFT JOIN (SELECT NULL::integer AS a, t2_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) t2_1) t2 ON ((t1.b OPERATOR(pg_catalog.=) t2.b))) DEBUG: Collecting INSERT ... SELECT results on coordinator ROLLBACK; -- INSERT .. SELECT: repartitioned (due to ) diff --git a/src/test/regress/expected/stat_counters.out b/src/test/regress/expected/stat_counters.out index 25327d4f7..e4f0351ea 100644 --- a/src/test/regress/expected/stat_counters.out +++ b/src/test/regress/expected/stat_counters.out @@ -701,7 +701,7 @@ RESET citus.enable_repartition_joins; -- ref_table, this increments query_execution_single_shard by 1 because it -- is a single shard query. CALL exec_query_and_check_query_counters($$ - SELECT * FROM ref_table LEFT JOIN dist_table ON dist_table.a = ref_table.a + SELECT * FROM ref_table LEFT JOIN dist_table ON dist_table.b = ref_table.b $$, 1, 1 ); diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 315e555eb..43527ff8a 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -235,6 +235,7 @@ test: local_table_join test: local_dist_join_mixed test: citus_local_dist_joins test: recurring_outer_join +test: recurring_join_pushdown test: query_single_shard_table test: insert_select_single_shard_table test: pg_dump diff --git a/src/test/regress/sql/multi_insert_select.sql b/src/test/regress/sql/multi_insert_select.sql index 19ae70abc..1f0679e34 100644 --- a/src/test/regress/sql/multi_insert_select.sql +++ b/src/test/regress/sql/multi_insert_select.sql @@ -577,12 +577,14 @@ FROM FROM raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id; + SET client_min_messages to debug3; INSERT INTO agg_events (user_id) SELECT raw_events_second.user_id FROM reference_table LEFT JOIN raw_events_second ON reference_table.user_id = raw_events_second.user_id; + SET client_min_messages to debug2; INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id @@ -2397,16 +2399,21 @@ SELECT coordinator_plan($$ EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT id, (SELECT id FROM ref_table_1 WHERE id = 1) FROM ref_table_1; $$); --- verify that insert select cannot be pushed down when we have reference table in outside of outer join. +-- verify that insert select cannot be pushed down when we have reference table in outside of outer join in a chained-join. SELECT coordinator_plan($$ EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT a.id FROM dist_table_5 a LEFT JOIN ref_table_1 b ON (true) RIGHT JOIN ref_table_1 c ON (true); $$); --- verify that insert select cannot be pushed down when it has a recurring outer join in a subquery. +-- verify that insert select can be pushed down when we have reference table in outside of outer join. SELECT coordinator_plan($$ EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT id FROM ref_table_1 LEFT JOIN dist_table_5 USING(id); $$); +-- verify that insert select cannot be pushed down when we have reference table in outside of left join and joined on non-partition column. +SELECT coordinator_plan($$ + EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT ref_table_1.id FROM ref_table_1 LEFT JOIN dist_table_5 ON ref_table_1.id = dist_table_5.id2; +$$); + CREATE TABLE loc_table_1(id int); -- verify that insert select cannot be pushed down when it contains join between local and distributed tables. diff --git a/src/test/regress/sql/multi_outer_join_reference.sql b/src/test/regress/sql/multi_outer_join_reference.sql index 04a9c23e2..4cede154f 100644 --- a/src/test/regress/sql/multi_outer_join_reference.sql +++ b/src/test/regress/sql/multi_outer_join_reference.sql @@ -159,7 +159,7 @@ FROM multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */); --- Right join is allowed as we recursively plan the distributed table (multi_outer_join_left_hash) +-- Right join is allowed (multi_outer_join_left_hash) SELECT min(r_custkey), max(r_custkey) FROM @@ -259,7 +259,7 @@ FROM ON (l_custkey = r_custkey AND r_custkey = 21); --- Right join should be allowed in this case as we recursively plan the distributed table (multi_outer_join_left_hash +-- Right join should be allowed in this case (multi_outer_join_left_hash) SELECT min(r_custkey), max(r_custkey) FROM diff --git a/src/test/regress/sql/multi_subquery_complex_reference_clause.sql b/src/test/regress/sql/multi_subquery_complex_reference_clause.sql index b722bd8f9..c4fa03237 100644 --- a/src/test/regress/sql/multi_subquery_complex_reference_clause.sql +++ b/src/test/regress/sql/multi_subquery_complex_reference_clause.sql @@ -1,7 +1,7 @@ -- -- multi subquery complex queries aims to expand existing subquery pushdown -- regression tests to cover more caeses --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests @@ -1531,5 +1531,4 @@ SELECT count(*) FROM (SELECT u1.*, random() FROM users_ref_test_table ref1 INNER SELECT count(*) FROM (SELECT ref1.* FROM users_ref_test_table ref1 INNER JOIN users_ref_test_table ref2 on ref1.id = ref2.id LIMIT 5) as foo FULL JOIN user_buy_test_table ON true; DROP TABLE user_buy_test_table; -DROP TABLE users_ref_test_table; DROP TABLE users_return_test_table; diff --git a/src/test/regress/sql/multi_subquery_in_where_reference_clause.sql b/src/test/regress/sql/multi_subquery_in_where_reference_clause.sql index fc1bb5c17..326ae5a91 100644 --- a/src/test/regress/sql/multi_subquery_in_where_reference_clause.sql +++ b/src/test/regress/sql/multi_subquery_in_where_reference_clause.sql @@ -126,15 +126,15 @@ WHERE ORDER BY user_id LIMIT 3; --- outer join could still recur +-- outer join pushed down SELECT DISTINCT user_id FROM users_table RIGHT JOIN users_reference_table USING (user_id) WHERE - users_reference_table.value_2 IN + (users_reference_table.value_2, random()*0) IN (SELECT - value_2 + value_2, 0 FROM events_table WHERE diff --git a/src/test/regress/sql/pg17.sql b/src/test/regress/sql/pg17.sql index 8d4c2097b..9a3979bee 100644 --- a/src/test/regress/sql/pg17.sql +++ b/src/test/regress/sql/pg17.sql @@ -169,8 +169,26 @@ WHERE d1.user_id = users.user_id AND users.dept IN (3,4) AND users.user_id = d2.user_id) dt GROUP BY dept; -RESET client_min_messages; +SET client_min_messages TO DEBUG3; +CREATE TABLE users_ref(user_id int, dept int); +SELECT create_reference_table('users_ref'); +INSERT INTO users_ref VALUES (1, 3), (2, 4), (3, 3), (4, 4); +-- In PG17, the planner can pull up a correlated ANY subquery to a join, resulting +-- in a different query plan compared to PG16. Specifically, for the following query +-- the rewritten query has a lateral recurring outer join, which requires recursive +-- computation of the inner part. However, this join is not analyzed during the recursive +-- planning step, as it is performed on the original query structure. As a result, +-- the lateral join is not recursively planned, and a lateral join error is raised +-- at a later stage. +SELECT user_id FROM +users RIGHT JOIN users_ref USING (user_id) +WHERE users_ref.dept IN +( + SELECT events.event_type FROM events WHERE events.user_id = users.user_id +) ORDER BY 1 LIMIT 1; + +RESET client_min_messages; RESET search_path; DROP SCHEMA pg17_corr_subq_folding CASCADE; diff --git a/src/test/regress/sql/recurring_join_pushdown.sql b/src/test/regress/sql/recurring_join_pushdown.sql new file mode 100644 index 000000000..792704abb --- /dev/null +++ b/src/test/regress/sql/recurring_join_pushdown.sql @@ -0,0 +1,142 @@ +CREATE SCHEMA recurring_join_pushdown; +SET search_path TO recurring_join_pushdown; + +SET citus.next_shard_id TO 1520000; +SET citus.shard_count TO 4; + +CREATE TABLE r1(a int, b int); +SELECT create_reference_table('r1'); +INSERT INTO r1 VALUES (1,10), (1,11), (1,20), (2,10), (2,12), (2, 20), (3, 20), (10, 1), (10, 2); + +CREATE TABLE d1(a int, b int); +SELECT create_distributed_table('d1', 'a'); +INSERT INTO d1 VALUES (1,10), (1,11), (1,20), (2,10), (2,12), (2, 20), (4, 10); + +CREATE TABLE d2(a int, c text); +SELECT create_distributed_table('d2', 'a'); +INSERT INTO d2(a, c) VALUES (1,'a'), (1,'b'), (1,'c'), (2,'d'), (2,'e'), (2,'f'), (4,'g'); + +SET citus.shard_count TO 2; +CREATE TABLE d3_not_colocated(like d1); +SELECT create_distributed_table('d3_not_colocated', 'a'); + + +SET client_min_messages TO DEBUG3; + +-- Basic test cases +-- Test that the join is pushed down to the worker nodes, using "using" syntax +SELECT count(*) FROM r1 LEFT JOIN d1 using (a); + +SELECT * FROM r1 LEFT JOIN d1 using (a, b) ORDER BY 1, 2; + +-- Disable the pushdown and verify that the join is not pushed down +SET citus.enable_recurring_outer_join_pushdown TO off; +SELECT count(*) FROM r1 LEFT JOIN d1 using (a); +SET citus.enable_recurring_outer_join_pushdown TO on; + +SET client_min_messages TO DEBUG1; +-- Test that the join is not pushed down when joined on a non-distributed column +SELECT count(*) FROM r1 LEFT JOIN d1 USING (b); + +-- Test that the join is not pushed down when we have non-colocated tables in the RHS +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d3_not_colocated.b FROM d3_not_colocated FULL JOIN d1 ON d3_not_colocated.a = d1.a) AS t1 USING (a); +-- The same error with its RIGHT JOIN variant +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d3_not_colocated.b FROM d3_not_colocated JOIN d1 ON d3_not_colocated.a = d1.a) AS t1 USING (a); + +-- Basic test cases with ON syntax +-- Test that the join is pushed down to the worker nodes, using "on" syntax +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.a = d1.a; +SELECT * FROM r1 LEFT JOIN d1 ON r1.a = d1.a AND r1.b = d1.b ORDER BY 1, 2; + +-- Verfiy that the join is pushed via the execution plan. +EXPLAIN (COSTS OFF) SELECT * FROM r1 LEFT JOIN d1 ON r1.a = d1.a AND r1.b = d1.b ORDER BY 1, 2; + +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.b = d1.a; +-- Test that the join is not pushed down when joined on a non-distributed column +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.b = d1.b; +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.a = d1.b; + +SET client_min_messages TO DEBUG1; +-- Test that the join is not pushed down when joined on a distributed column with disjunctive conditions +SELECT count(*) FROM r1 LEFT JOIN d1 ON r1.a = d1.a OR r1.b = d1.b; + +-- Test join pushdown behavior when the inner part of the join is a subquery +-- Using 'using' syntax +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1) AS t1 USING (a); + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1 WHERE a > 1) AS t1 USING (a); + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM (SELECT * FROM d1) AS t1 WHERE a > 1) AS t2 USING (a); + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1 JOIN d1 as d1_1 USING (a)) AS t1 USING (a); + +SELECT count(*) FROM r1 LEFT JOIN (d1 LEFT JOIN d1 as d1_1 USING (a)) AS t1 USING (a); + +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1) AS t1 USING (a); + + +-- Using 'on' syntax +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1) AS d1 ON r1.a = d1.a; + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM d1 WHERE a > 1) AS d1 ON r1.a = d1.a; + +SELECT count(*) FROM r1 LEFT JOIN (SELECT * FROM (SELECT * FROM d1) AS d1 WHERE a > 1) AS d1 ON r1.a = d1.a; + +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a as a, d1.b, d1_1.a AS a_1 FROM d1 LEFT JOIN d1 as d1_1 ON d1.a = d1_1.a) AS d1_2 ON r1.a = d1_2.a; + + +-- Nested joins +-- It is safe to push the inner join to compute t1. However, as the var of the inner table for the top level join (t1.a) resolves to r1.a, the outer join cannot be pushed down. +SELECT count(*) FROM r1 LEFT JOIN (SELECT r1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT r1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; + +-- In the following case, it is safe to push down both joins as t1.a resolves to d1.a. +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON r1.a = t1.a; + +-- In the following case, the lower level joins will be pushed down, but as the top level join is chained, subquery pushdown will not be applied at the top level. +SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON t1.a = r1.a LEFT JOIN (SELECT d2.a, d2.c FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (SELECT d1.a, d1.b FROM r1 LEFT JOIN d1 ON r1.a = d1.a) AS t1 ON t1.a = r1.a LEFT JOIN (SELECT d2.a, d2.c FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; + +--- As both subqueries are pushed and the top level join is over their results on distribution colums, the query is pushed down as a whole. +SELECT count(*) FROM (SELECT d1_1.a, r1.b FROM r1 LEFT JOIN d1 as d1_1 ON r1.a = d1_1.a) AS t1 LEFT JOIN + (SELECT d2.a, d2.c, r1.b FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; + +EXPLAIN (COSTS OFF) SELECT count(*) FROM (SELECT d1_1.a, r1.b FROM r1 LEFT JOIN d1 as d1_1 ON r1.a = d1_1.a) AS t1 LEFT JOIN + (SELECT d2.a, d2.c, r1.b FROM r1 LEFT JOIN d2 ON r1.a = d2.a) AS t2 ON t1.a = t2.a; + + + -- Basic cases with RIGHT JOIN +SET client_min_messages TO DEBUG3; +SELECT count(*) FROM d1 RIGHT JOIN r1 USING (a); + +SELECT count(*) FROM (SELECT * FROM d1) AS t1 RIGHT JOIN r1 USING (a); + +SET client_min_messages TO ERROR; + +-- Ensure that even when CTEs are replaced, we insert push +-- down conditions for outer joins when necessary. +WITH cte_0 AS ( SELECT table_0.a FROM d1 AS table_0 WHERE table_0.a IN ( SELECT table_1.a FROM d1 AS table_1 ORDER BY a LIMIT 2 ) ORDER BY a ) SELECT count(*), avg(avgsub.a) FROM ( SELECT table_2.a FROM cte_0 AS table_2 RIGHT JOIN r1 AS table_3 USING (a)) AS avgsub; + + +-- The following queries trigger recursive computing, recurring outer-join push down +-- methods introduced in#7973 can be enhanced to cover these cases in the future. +CREATE TABLE r1_local AS SELECT * FROM r1; +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1_local LEFT JOIN d1 ON r1_local.a = d1.a; +EXPLAIN (COSTS OFF) SELECT count(*) FROM (SELECT * FROM r1) sq LEFT JOIN d1 ON sq.a = d1.a; +EXPLAIN (COSTS OFF) SELECT count(*) FROM r1 LEFT JOIN (d1 INNER JOIN d2 on d1.a = d2.a) on r1.a = d2.a; + +SET client_min_messages TO DEBUG3; +-- The following query is recursively computed due to the lateral join in the subquery. +-- Leaving the inner side of the query to query push down causes an error in deferred error +-- checks even if it is possible to push down the whole query. +SELECT DISTINCT sq.a FROM ( + SELECT d1.a FROM d1 JOIN LATERAL ( SELECT * FROM r1 WHERE r1.a = d1.a ) sq2 ON true ) AS sq RIGHT JOIN r1 USING (a) +ORDER BY sq.a +LIMIT 1; + +SET client_min_messages TO ERROR; +DROP SCHEMA recurring_join_pushdown CASCADE; diff --git a/src/test/regress/sql/recurring_outer_join.sql b/src/test/regress/sql/recurring_outer_join.sql index 014a7e536..b89aadcc3 100644 --- a/src/test/regress/sql/recurring_outer_join.sql +++ b/src/test/regress/sql/recurring_outer_join.sql @@ -147,7 +147,7 @@ SELECT COUNT(*) FROM ref_1 LEFT JOIN (dist_1 t1 LEFT JOIN dist_1 t2 USING (a)) q SELECT COUNT(*) FROM dist_1 t1 FULL JOIN (dist_1 RIGHT JOIN citus_local_1 USING(a)) t2 USING (a); - -- subqury without FROM + -- subquery without FROM SELECT COUNT(*) FROM dist_1 t1 RIGHT JOIN (SELECT generate_series(1,10) AS a) t2 USING (a); -- such semi joins / anti joins are supported too @@ -253,7 +253,7 @@ SELECT COUNT(*) FROM ref_1 LEFT JOIN (dist_1 t1 LEFT JOIN dist_1 t2 USING (a)) q ON (t1.a = t2.a) WHERE t1.a IN (SELECT a FROM dist_1 t3); - -- subqury without FROM + -- subquery without FROM SELECT COUNT(*) FROM (SELECT generate_series(1,10) AS a) t1 JOIN dist_1 t2 @@ -296,8 +296,11 @@ LATERAL WHERE r1.a > dist_1.b ) as foo; --- Qual is the same but top-level join is an anti-join. Right join --- stays as is and hence requires recursive planning. +-- Qual is the same but top-level join is an anti-join. +-- The right join between t2 and t3 is pushed down. +-- Citus determines that the whole query can be pushed down +-- due to the equality constraint between two distributed +-- tables t1 and t2. SELECT COUNT(*) FROM dist_1 t1 WHERE NOT EXISTS ( SELECT * FROM dist_1 t2 @@ -305,11 +308,17 @@ WHERE NOT EXISTS ( WHERE t2.a = t1.a ); +SET client_min_messages TO DEBUG3; + -- This time the semi-join qual is (not <) -- where t3 is the outer rel of the right join. Hence Postgres can't --- replace right join with an inner join and so we recursively plan --- inner side of the right join since the outer side is a recurring --- rel. +-- replace right join with an inner join. +-- Citus pushes down the right join between t2 and t3 with constraints on +-- the recurring outer part (t3). However, it cannnot push down the whole +-- query as it can not establish an equivalence between the distribution +-- tables t1 and t2. Hence, Citus tries to recursively plan the subquery. +-- This attempt fails since the subquery has a reference to outer query. +-- See #8113 SELECT COUNT(*) FROM dist_1 t1 WHERE EXISTS ( SELECT * FROM dist_1 t2 @@ -324,6 +333,25 @@ WHERE NOT EXISTS ( WHERE t3.a = t1.a ); +SET client_min_messages TO DEBUG1; + +-- Force recursive planning of the right join with offset +SELECT COUNT(*) FROM dist_1 t1 +WHERE EXISTS ( + SELECT * FROM dist_1 t2 + RIGHT JOIN ref_1 t3 USING (a) + WHERE t3.a = t1.a + OFFSET 0 +); + +SELECT COUNT(*) FROM dist_1 t1 +WHERE NOT EXISTS ( + SELECT * FROM dist_1 t2 + RIGHT JOIN ref_1 t3 USING (a) + WHERE t3.a = t1.a + OFFSET 0 +); + -- -- more complex cases -- @@ -350,7 +378,8 @@ LEFT JOIN ( dist_1 t4 JOIN - -- 1) t6 is recursively planned since the outer side is recurring + -- 1) t6 is not recursively planned since it is + -- safe to push down the recurring outer side with constraints (SELECT t6.a FROM dist_1 t6 RIGHT JOIN ref_1 t7 USING(a)) t5 USING(a) ) q @@ -584,8 +613,8 @@ LEFT JOIN USING(a); -- cannot recursively plan because t3 (inner - distributed) --- references t1 (outer - recurring) -SELECT COUNT(*) FROM ref_1 t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (a); +-- references t1 (outer - recurring over non-distribution column) +SELECT COUNT(*) FROM ref_1 t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (b); SELECT COUNT(*) FROM (SELECT * FROM dist_1 OFFSET 100) t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (a); SELECT COUNT(*) FROM local_1 t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (a); SELECT COUNT(*) FROM (SELECT 1 a, generate_series(1,2) b) t1 LEFT JOIN LATERAL (SELECT * FROM dist_1 t2 WHERE t1.b < t2.b) t3 USING (a); @@ -712,14 +741,13 @@ LEFT JOIN USING (a); SELECT COUNT(*) FROM ref_1 t1 --- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted --- to a recurring rel too. For this reason, subquery t8 is recursively --- planned because t7 is recurring already. +-- 2) It is also safe to push down this since the recurring outer side t1 and +-- distributed inner side t8 are joined on the distribution column. LEFT JOIN ( SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 JOIN - -- 1) subquery t6 is recursively planned because t5 is recurring + -- 1) it is safe to push down subquery t7 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < 150) t6 USING (a)) as t7 USING(a) ) t8 @@ -728,14 +756,13 @@ USING (a); -- same test using a prepared statement PREPARE recurring_outer_join_p1 AS SELECT COUNT(*) FROM ref_1 t1 --- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted --- to a recurring rel too. For this reason, subquery t8 is recursively --- planned because t7 is recurring already. +-- 2) It is also safe to push down this since the recurring outer side t1 and +-- distributed inner side t8 are joined on the distribution column. LEFT JOIN ( SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 JOIN - -- 1) subquery t6 is recursively planned because t5 is recurring + -- 1) it is safe to push down subquery t7 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < $1) t6 USING (a)) as t7 USING(a) ) t8 @@ -845,7 +872,7 @@ SELECT * FROM ref_1 t36 WHERE (b,100,a) IN ( DISTINCT t31.b, -- 1) we first search for such joins in the target list and recursively plan t33 -- because t32 is recurring - (SELECT max(b) FROM ref_1 t32 LEFT JOIN dist_1 t33 USING(a,b) WHERE t31.a = t32.a), + (SELECT max(b) FROM ref_1 t32 LEFT JOIN dist_1 t33 USING(b) WHERE t31.a = t32.a), (SELECT t34.a) FROM ref_1 t35 LEFT JOIN dist_1 t31 USING (a,b) @@ -962,10 +989,10 @@ BEGIN; SELECT t1.a, t1.b FROM ref_1 t1 LEFT JOIN ( - SELECT * FROM dist_1 t2 WHERE EXISTS ( + SELECT DISTINCT ON (a) * FROM dist_1 t2 WHERE EXISTS ( SELECT * FROM dist_1 t4 WHERE t4.a = t2.a - ) + ) ORDER BY a, b ) t3 USING (a) ) q @@ -990,7 +1017,7 @@ BEGIN; RETURNING *; ROLLBACK; --- INSERT .. SELECT: pull to coordinator +-- INSERT .. SELECT: Repartitioned BEGIN; DELETE FROM ref_1 WHERE a IS NULL; @@ -1001,6 +1028,17 @@ BEGIN; ON (t1.a = t2.a); ROLLBACK; +-- INSERT .. SELECT: pull to coordinator +BEGIN; + DELETE FROM ref_1 WHERE a IS NULL; + + INSERT INTO dist_1 + SELECT t1.* + FROM ref_1 t1 + LEFT JOIN dist_1 t2 + ON (t1.b = t2.b); +ROLLBACK; + -- INSERT .. SELECT: repartitioned (due to ) BEGIN; INSERT INTO dist_1 diff --git a/src/test/regress/sql/stat_counters.sql b/src/test/regress/sql/stat_counters.sql index 18f4b8aac..bd07e32b0 100644 --- a/src/test/regress/sql/stat_counters.sql +++ b/src/test/regress/sql/stat_counters.sql @@ -453,7 +453,7 @@ RESET citus.enable_repartition_joins; -- ref_table, this increments query_execution_single_shard by 1 because it -- is a single shard query. CALL exec_query_and_check_query_counters($$ - SELECT * FROM ref_table LEFT JOIN dist_table ON dist_table.a = ref_table.a + SELECT * FROM ref_table LEFT JOIN dist_table ON dist_table.b = ref_table.b $$, 1, 1 );