Push down recurring outer joins when possible (#7973)

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., <reference table> LEFT JOIN <distributed table>)

Partially addresses #6546 

1) `<outer:reference>` LEFT JOIN `<inner:distributed>` 
2) `<inner:distributed>` RIGHT JOIN `<outer:reference>` 
 
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 <ebruaydingol@microsoft.com>
Co-authored-by: Onur Tirtir <onurcantirtir@gmail.com>
pull/8121/head^2
eaydingol 2025-08-18 14:03:44 +03:00 committed by GitHub
parent 87a1b631e8
commit 8d929d3bf8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
36 changed files with 2377 additions and 347 deletions

View File

@ -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. * HasDistributionKey returns true if given Citus table has a distribution key.
*/ */

View File

@ -16,6 +16,8 @@
#include "access/heapam.h" #include "access/heapam.h"
#include "access/htup_details.h" #include "access/htup_details.h"
#include "catalog/pg_constraint.h" #include "catalog/pg_constraint.h"
#include "catalog/pg_namespace.h"
#include "catalog/pg_operator.h"
#include "lib/stringinfo.h" #include "lib/stringinfo.h"
#include "nodes/makefuncs.h" #include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h" #include "nodes/nodeFuncs.h"
@ -38,6 +40,8 @@
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
#include "distributed/multi_router_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/shard_utils.h"
#include "distributed/stats/stat_tenants.h" #include "distributed/stats/stat_tenants.h"
#include "distributed/version_compat.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 * UpdateRelationToShardNames walks over the query tree and appends shard ids to
* relations. It uses unique identity value to establish connection between a * relations. It uses unique identity value to establish connection between a

View File

@ -75,17 +75,6 @@
#endif #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; static List *plannerRestrictionContextList = NIL;
int MultiTaskQueryLogLevel = CITUS_LOG_LEVEL_OFF; /* multi-task query log level */ int MultiTaskQueryLogLevel = CITUS_LOG_LEVEL_OFF; /* multi-task query log level */
static uint64 NextPlanId = 1; 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. * set_plan_references>add_rtes_to_flat_rtable>add_rte_to_flat_rtable.
*/ */
List *subPlanList = GenerateSubplansForSubqueriesAndCTEs(planId, originalQuery, List *subPlanList = GenerateSubplansForSubqueriesAndCTEs(planId, originalQuery,
plannerRestrictionContext); plannerRestrictionContext,
routerPlan);
/* /*
* If subqueries were recursively planned then we need to replan the query * If subqueries were recursively planned then we need to replan the query

View File

@ -766,7 +766,8 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte,
{ {
/* first apply toplevel pushdown checks to SELECT query */ /* first apply toplevel pushdown checks to SELECT query */
error = error =
DeferErrorIfUnsupportedSubqueryPushdown(subquery, plannerRestrictionContext); DeferErrorIfUnsupportedSubqueryPushdown(subquery, plannerRestrictionContext,
true);
if (error) if (error)
{ {
return error; return error;

View File

@ -1149,7 +1149,8 @@ DeferErrorIfRoutableMergeNotSupported(Query *query, List *rangeTableList,
{ {
deferredError = deferredError =
DeferErrorIfUnsupportedSubqueryPushdown(query, DeferErrorIfUnsupportedSubqueryPushdown(query,
plannerRestrictionContext); plannerRestrictionContext,
true);
if (deferredError) if (deferredError)
{ {
ereport(DEBUG1, (errmsg("Sub-query is not pushable, try repartitioning"))); ereport(DEBUG1, (errmsg("Sub-query is not pushable, try repartitioning")));

View File

@ -167,13 +167,16 @@ static uint32 HashPartitionCount(void);
/* Local functions forward declarations for task list creation and helper functions */ /* Local functions forward declarations for task list creation and helper functions */
static Job * BuildJobTreeTaskList(Job *jobTree, static Job * BuildJobTreeTaskList(Job *jobTree,
PlannerRestrictionContext *plannerRestrictionContext); PlannerRestrictionContext *plannerRestrictionContext);
static bool IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction); static bool IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction,
Bitmapset *distributedTables,
bool *outerPartHasDistributedTable);
static void ErrorIfUnsupportedShardDistribution(Query *query); static void ErrorIfUnsupportedShardDistribution(Query *query);
static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
RelationRestrictionContext *restrictionContext, RelationRestrictionContext *restrictionContext,
uint32 taskId, uint32 taskId,
TaskType taskType, TaskType taskType,
bool modifyRequiresCoordinatorEvaluation, bool modifyRequiresCoordinatorEvaluation,
bool updateQualsForOuterJoin,
DeferredErrorMessage **planningError); DeferredErrorMessage **planningError);
static List * SqlTaskList(Job *job); static List * SqlTaskList(Job *job);
static bool DependsOnHashPartitionJob(Job *job); static bool DependsOnHashPartitionJob(Job *job);
@ -2199,6 +2202,7 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
int minShardOffset = INT_MAX; int minShardOffset = INT_MAX;
int prevShardCount = 0; int prevShardCount = 0;
Bitmapset *taskRequiredForShardIndex = NULL; Bitmapset *taskRequiredForShardIndex = NULL;
Bitmapset *distributedTableIndex = NULL;
/* error if shards are not co-partitioned */ /* error if shards are not co-partitioned */
ErrorIfUnsupportedShardDistribution(query); ErrorIfUnsupportedShardDistribution(query);
@ -2215,8 +2219,12 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
RelationRestriction *relationRestriction = NULL; RelationRestriction *relationRestriction = NULL;
List *prunedShardList = NULL; List *prunedShardList = NULL;
forboth_ptr(prunedShardList, prunedRelationShardList, /* First loop, gather the indexes of distributed tables
relationRestriction, relationRestrictionContext->relationRestrictionList) * 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; Oid relationId = relationRestriction->relationId;
@ -2237,6 +2245,24 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
} }
prevShardCount = cacheEntry->shardIntervalArrayLength; 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. * 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 * If the right hand side would prune to a smaller set we should still send it to
@ -2244,13 +2270,26 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
* the left hand side we don't have to send the query to any shard that is not * 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. * matching anything on the left hand side.
* *
* Instead we will simply skip any RelationRestriction if it is an OUTER join and * 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. * 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))
{ {
if (outerPartHasDistributedTable)
{
/* we can skip the shards from this relation restriction */
continue; 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; ShardInterval *shardInterval = NULL;
foreach_declared_ptr(shardInterval, prunedShardList) foreach_declared_ptr(shardInterval, prunedShardList)
@ -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 * 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 * 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, taskIdIndex,
taskType, taskType,
modifyRequiresCoordinatorEvaluation, modifyRequiresCoordinatorEvaluation,
updateQualsForOuterJoin,
planningError); planningError);
if (*planningError != NULL) if (*planningError != NULL)
{ {
@ -2315,10 +2371,13 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
* a) in an outer join * a) in an outer join
* b) on the inner part of said 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 static bool
IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction) IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction,
Bitmapset *distributedTables,
bool *outerPartHasDistributedTable)
{ {
RestrictInfo *joinInfo = NULL; RestrictInfo *joinInfo = NULL;
foreach_declared_ptr(joinInfo, relationRestriction->relOptInfo->joininfo) foreach_declared_ptr(joinInfo, relationRestriction->relOptInfo->joininfo)
@ -2339,6 +2398,11 @@ IsInnerTableOfOuterJoin(RelationRestriction *relationRestriction)
if (!isInOuter) if (!isInOuter)
{ {
/* this table is joined in the inner part of an outer join */ /* 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; return true;
} }
} }
@ -2455,6 +2519,7 @@ static Task *
QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
RelationRestrictionContext *restrictionContext, uint32 taskId, RelationRestrictionContext *restrictionContext, uint32 taskId,
TaskType taskType, bool modifyRequiresCoordinatorEvaluation, TaskType taskType, bool modifyRequiresCoordinatorEvaluation,
bool updateQualsForOuterJoin,
DeferredErrorMessage **planningError) DeferredErrorMessage **planningError)
{ {
Query *taskQuery = copyObject(originalQuery); Query *taskQuery = copyObject(originalQuery);
@ -2559,6 +2624,21 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
(List *) taskQuery->jointree->quals); (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); Task *subqueryTask = CreateBasicTask(jobId, taskId, taskType, NULL);
if ((taskType == MODIFY_TASK && !modifyRequiresCoordinatorEvaluation) || if ((taskType == MODIFY_TASK && !modifyRequiresCoordinatorEvaluation) ||

View File

@ -1321,7 +1321,8 @@ MultiShardUpdateDeleteSupported(Query *originalQuery,
{ {
errorMessage = DeferErrorIfUnsupportedSubqueryPushdown( errorMessage = DeferErrorIfUnsupportedSubqueryPushdown(
originalQuery, originalQuery,
plannerRestrictionContext); plannerRestrictionContext,
true);
} }
return errorMessage; return errorMessage;

View File

@ -88,7 +88,7 @@ static bool WindowPartitionOnDistributionColumn(Query *query);
static DeferredErrorMessage * DeferErrorIfFromClauseRecurs(Query *queryTree); static DeferredErrorMessage * DeferErrorIfFromClauseRecurs(Query *queryTree);
static RecurringTuplesType FromClauseRecurringTupleType(Query *queryTree); static RecurringTuplesType FromClauseRecurringTupleType(Query *queryTree);
static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin( static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin(
PlannerRestrictionContext *plannerRestrictionContext); PlannerRestrictionContext *plannerRestrictionContext, bool plannerPhase);
static DeferredErrorMessage * DeferErrorIfUnsupportedTableCombination(Query *queryTree); static DeferredErrorMessage * DeferErrorIfUnsupportedTableCombination(Query *queryTree);
static DeferredErrorMessage * DeferErrorIfSubqueryRequiresMerge(Query *subqueryTree, bool static DeferredErrorMessage * DeferErrorIfSubqueryRequiresMerge(Query *subqueryTree, bool
lateral, lateral,
@ -109,6 +109,7 @@ static bool RelationInfoContainsOnlyRecurringTuples(PlannerInfo *plannerInfo,
static char * RecurringTypeDescription(RecurringTuplesType recurType); static char * RecurringTypeDescription(RecurringTuplesType recurType);
static DeferredErrorMessage * DeferredErrorIfUnsupportedLateralSubquery( static DeferredErrorMessage * DeferredErrorIfUnsupportedLateralSubquery(
PlannerInfo *plannerInfo, Relids recurringRelIds, Relids nonRecurringRelIds); PlannerInfo *plannerInfo, Relids recurringRelIds, Relids nonRecurringRelIds);
static bool ContainsLateralSubquery(PlannerInfo *plannerInfo);
static Var * PartitionColumnForPushedDownSubquery(Query *query); static Var * PartitionColumnForPushedDownSubquery(Query *query);
static bool ContainsReferencesToRelids(Query *query, Relids relids, int *foundRelid); static bool ContainsReferencesToRelids(Query *query, Relids relids, int *foundRelid);
static bool ContainsReferencesToRelidsWalker(Node *node, static bool ContainsReferencesToRelidsWalker(Node *node,
@ -535,9 +536,16 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree,
RaiseDeferredError(unsupportedQueryError, ERROR); 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( DeferredErrorMessage *subqueryPushdownError = DeferErrorIfUnsupportedSubqueryPushdown(
originalQuery, originalQuery,
plannerRestrictionContext); plannerRestrictionContext,
false);
if (subqueryPushdownError != NULL) if (subqueryPushdownError != NULL)
{ {
RaiseDeferredError(subqueryPushdownError, ERROR); RaiseDeferredError(subqueryPushdownError, ERROR);
@ -560,7 +568,8 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree,
DeferredErrorMessage * DeferredErrorMessage *
DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery, DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext) plannerRestrictionContext,
bool plannerPhase)
{ {
bool outerMostQueryHasLimit = false; bool outerMostQueryHasLimit = false;
ListCell *subqueryCell = NULL; ListCell *subqueryCell = NULL;
@ -612,7 +621,8 @@ DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery,
return error; return error;
} }
error = DeferredErrorIfUnsupportedRecurringTuplesJoin(plannerRestrictionContext); error = DeferredErrorIfUnsupportedRecurringTuplesJoin(plannerRestrictionContext,
plannerPhase);
if (error) if (error)
{ {
return error; return error;
@ -770,12 +780,17 @@ FromClauseRecurringTupleType(Query *queryTree)
* DeferredErrorIfUnsupportedRecurringTuplesJoin returns a DeferredError if * DeferredErrorIfUnsupportedRecurringTuplesJoin returns a DeferredError if
* there exists a join between a recurring rel (such as reference tables * 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 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 * and subqueries that we can push-down to worker nodes) when plannerPhase is
* incorrect result set due to recurring tuples coming from the recurring rel. * 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 * static DeferredErrorMessage *
DeferredErrorIfUnsupportedRecurringTuplesJoin( DeferredErrorIfUnsupportedRecurringTuplesJoin(
PlannerRestrictionContext *plannerRestrictionContext) PlannerRestrictionContext *plannerRestrictionContext,
bool plannerPhase)
{ {
List *joinRestrictionList = List *joinRestrictionList =
plannerRestrictionContext->joinRestrictionContext->joinRestrictionList; plannerRestrictionContext->joinRestrictionContext->joinRestrictionList;
@ -827,16 +842,31 @@ DeferredErrorIfUnsupportedRecurringTuplesJoin(
} }
if (RelationInfoContainsOnlyRecurringTuples(plannerInfo, outerrelRelids)) if (RelationInfoContainsOnlyRecurringTuples(plannerInfo, outerrelRelids))
{
if (plannerPhase)
{ {
/* /*
* Inner side contains distributed rels but the outer side only * We have not yet tried to recursively plan this join, we should
* contains recurring rels, must be an unsupported lateral outer * defer an error.
* join.
*/ */
recurType = FetchFirstRecurType(plannerInfo, outerrelRelids); recurType = FetchFirstRecurType(plannerInfo, outerrelRelids);
break; break;
} }
/*
* Inner side contains distributed rels but the outer side only
* 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.
*/
if (ContainsLateralSubquery(plannerInfo))
{
recurType = FetchFirstRecurType(plannerInfo, outerrelRelids);
break;
}
}
} }
else if (joinType == JOIN_FULL) 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 * FetchFirstRecurType checks whether the relationInfo
* contains any recurring table expression, namely a reference table, * contains any recurring table expression, namely a reference table,

View File

@ -49,6 +49,7 @@
#include "postgres.h" #include "postgres.h"
#include "funcapi.h" #include "funcapi.h"
#include "miscadmin.h"
#include "catalog/pg_class.h" #include "catalog/pg_class.h"
#include "catalog/pg_type.h" #include "catalog/pg_type.h"
@ -73,8 +74,10 @@
#include "distributed/citus_nodes.h" #include "distributed/citus_nodes.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/combine_query_planner.h"
#include "distributed/commands/multi_copy.h" #include "distributed/commands/multi_copy.h"
#include "distributed/distributed_planner.h" #include "distributed/distributed_planner.h"
#include "distributed/distribution_column.h"
#include "distributed/errormessage.h" #include "distributed/errormessage.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/local_distributed_join_planner.h" #include "distributed/local_distributed_join_planner.h"
@ -87,11 +90,14 @@
#include "distributed/multi_server_executor.h" #include "distributed/multi_server_executor.h"
#include "distributed/query_colocation_checker.h" #include "distributed/query_colocation_checker.h"
#include "distributed/query_pushdown_planning.h" #include "distributed/query_pushdown_planning.h"
#include "distributed/query_utils.h"
#include "distributed/recursive_planning.h" #include "distributed/recursive_planning.h"
#include "distributed/relation_restriction_equivalence.h" #include "distributed/relation_restriction_equivalence.h"
#include "distributed/shard_pruning.h" #include "distributed/shard_pruning.h"
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
bool EnableRecurringOuterJoinPushdown = true;
/* /*
* RecursivePlanningContext is used to recursively plan subqueries * RecursivePlanningContext is used to recursively plan subqueries
* and CTEs, pull results to the coordinator, and push it back into * and CTEs, pull results to the coordinator, and push it back into
@ -104,6 +110,8 @@ struct RecursivePlanningContextInternal
bool allDistributionKeysInQueryAreEqual; /* used for some optimizations */ bool allDistributionKeysInQueryAreEqual; /* used for some optimizations */
List *subPlanList; List *subPlanList;
PlannerRestrictionContext *plannerRestrictionContext; PlannerRestrictionContext *plannerRestrictionContext;
bool restrictionEquivalenceCheck;
bool forceRecursivelyPlanRecurringOuterJoins;
}; };
/* track depth of current recursive planner query */ /* track depth of current recursive planner query */
@ -152,7 +160,8 @@ static void RecursivelyPlanNonColocatedSubqueriesInWhere(Query *query,
RecursivePlanningContext * RecursivePlanningContext *
recursivePlanningContext); recursivePlanningContext);
static bool RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, static bool RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
RecursivePlanningContext *context); RecursivePlanningContext *context,
bool chainedJoin);
static void RecursivelyPlanDistributedJoinNode(Node *node, Query *query, static void RecursivelyPlanDistributedJoinNode(Node *node, Query *query,
RecursivePlanningContext *context); RecursivePlanningContext *context);
static bool IsRTERefRecurring(RangeTblRef *rangeTableRef, Query *query); static bool IsRTERefRecurring(RangeTblRef *rangeTableRef, Query *query);
@ -193,6 +202,9 @@ static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry,
List *outerSubqueryTargetList); List *outerSubqueryTargetList);
static List * GenerateRequiredColNamesFromTargetList(List *targetList); static List * GenerateRequiredColNamesFromTargetList(List *targetList);
static char * GetRelationNameAndAliasName(RangeTblEntry *rangeTablentry); 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 #if PG_VERSION_NUM < PG_VERSION_17
static bool hasPseudoconstantQuals( static bool hasPseudoconstantQuals(
RelationRestrictionContext *relationRestrictionContext); RelationRestrictionContext *relationRestrictionContext);
@ -207,7 +219,8 @@ static bool hasPseudoconstantQuals(
*/ */
List * List *
GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery, GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery,
PlannerRestrictionContext *plannerRestrictionContext) PlannerRestrictionContext *plannerRestrictionContext,
RouterPlanType routerPlan)
{ {
RecursivePlanningContext context; RecursivePlanningContext context;
@ -221,6 +234,17 @@ GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery,
context.planId = planId; context.planId = planId;
context.subPlanList = NIL; context.subPlanList = NIL;
context.plannerRestrictionContext = plannerRestrictionContext; 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. * 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)) if (ShouldRecursivelyPlanOuterJoins(query, context))
{ {
RecursivelyPlanRecurringTupleOuterJoinWalker((Node *) query->jointree, RecursivelyPlanRecurringTupleOuterJoinWalker((Node *) query->jointree,
query, context); query, context, false);
} }
/* /*
@ -691,7 +715,8 @@ RecursivelyPlanNonColocatedSubqueriesInWhere(Query *query,
static bool static bool
RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query, RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
RecursivePlanningContext * RecursivePlanningContext *
recursivePlanningContext) recursivePlanningContext,
bool chainedJoin)
{ {
if (node == NULL) if (node == NULL)
{ {
@ -708,7 +733,8 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
Node *fromElement = (Node *) lfirst(fromExprCell); Node *fromElement = (Node *) lfirst(fromExprCell);
RecursivelyPlanRecurringTupleOuterJoinWalker(fromElement, query, RecursivelyPlanRecurringTupleOuterJoinWalker(fromElement, query,
recursivePlanningContext); recursivePlanningContext,
false);
} }
/* /*
@ -734,16 +760,23 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
*/ */
bool leftNodeRecurs = bool leftNodeRecurs =
RecursivelyPlanRecurringTupleOuterJoinWalker(leftNode, query, RecursivelyPlanRecurringTupleOuterJoinWalker(leftNode, query,
recursivePlanningContext); recursivePlanningContext,
true);
bool rightNodeRecurs = bool rightNodeRecurs =
RecursivelyPlanRecurringTupleOuterJoinWalker(rightNode, query, RecursivelyPlanRecurringTupleOuterJoinWalker(rightNode, query,
recursivePlanningContext); recursivePlanningContext,
true);
switch (joinExpr->jointype) switch (joinExpr->jointype)
{ {
case JOIN_LEFT: case JOIN_LEFT:
{ {
/* <recurring> left join <distributed> */ /* <recurring> left join <distributed> */
if (leftNodeRecurs && !rightNodeRecurs) if (leftNodeRecurs && !rightNodeRecurs)
{
if (recursivePlanningContext->forceRecursivelyPlanRecurringOuterJoins
||
chainedJoin || !CanPushdownRecurringOuterJoin(joinExpr,
query))
{ {
ereport(DEBUG1, (errmsg("recursively planning right side of " ereport(DEBUG1, (errmsg("recursively planning right side of "
"the left join since the outer side " "the left join since the outer side "
@ -751,6 +784,13 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
RecursivelyPlanDistributedJoinNode(rightNode, query, RecursivelyPlanDistributedJoinNode(rightNode, query,
recursivePlanningContext); recursivePlanningContext);
} }
else
{
ereport(DEBUG3, (errmsg(
"a push down safe left join with recurring left side")));
leftNodeRecurs = false; /* left node will be pushed down */
}
}
/* /*
* A LEFT JOIN is recurring if the lhs is recurring. * A LEFT JOIN is recurring if the lhs is recurring.
@ -765,6 +805,11 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
{ {
/* <distributed> right join <recurring> */ /* <distributed> right join <recurring> */
if (!leftNodeRecurs && rightNodeRecurs) if (!leftNodeRecurs && rightNodeRecurs)
{
if (recursivePlanningContext->forceRecursivelyPlanRecurringOuterJoins
||
chainedJoin || !CanPushdownRecurringOuterJoin(joinExpr,
query))
{ {
ereport(DEBUG1, (errmsg("recursively planning left side of " ereport(DEBUG1, (errmsg("recursively planning left side of "
"the right join since the outer side " "the right join since the outer side "
@ -772,6 +817,13 @@ RecursivelyPlanRecurringTupleOuterJoinWalker(Node *node, Query *query,
RecursivelyPlanDistributedJoinNode(leftNode, query, RecursivelyPlanDistributedJoinNode(leftNode, query,
recursivePlanningContext); recursivePlanningContext);
} }
else
{
ereport(DEBUG3, (errmsg(
"a push down safe right join with recurring left side")));
rightNodeRecurs = false; /* right node will be pushed down */
}
}
/* /*
* Similar to LEFT JOINs, a RIGHT JOIN is recurring if the rhs * Similar to LEFT JOINs, a RIGHT JOIN is recurring if the rhs
@ -2642,3 +2694,335 @@ hasPseudoconstantQuals(RelationRestrictionContext *relationRestrictionContext)
#endif #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);
}

View File

@ -1480,6 +1480,21 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE, GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE,
NULL, NULL, NULL); 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( DefineCustomBoolVariable(
"citus.enable_repartition_joins", "citus.enable_repartition_joins",
gettext_noop("Allows Citus to repartition data between nodes."), gettext_noop("Allows Citus to repartition data between nodes."),

View File

@ -20,10 +20,17 @@
#include "nodes/pg_list.h" #include "nodes/pg_list.h"
#include "distributed/citus_custom_scan.h" #include "distributed/citus_custom_scan.h"
#include "distributed/query_utils.h"
extern void RebuildQueryStrings(Job *workerJob); extern void RebuildQueryStrings(Job *workerJob);
extern bool UpdateRelationToShardNames(Node *node, List *relationShardList); 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 SetTaskQueryIfShouldLazyDeparse(Task *task, Query *query);
extern void SetTaskQueryString(Task *task, char *queryString); extern void SetTaskQueryString(Task *task, char *queryString);
extern void SetTaskQueryStringList(Task *task, List *queryStringList); extern void SetTaskQueryStringList(Task *task, List *queryStringList);

View File

@ -33,6 +33,18 @@
extern int PlannerLevel; 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 typedef struct RelationRestrictionContext
{ {
bool allReferenceTables; bool allReferenceTables;

View File

@ -146,6 +146,7 @@ extern bool IsCitusTableType(Oid relationId, CitusTableType tableType);
extern CitusTableType GetCitusTableType(CitusTableCacheEntry *tableEntry); extern CitusTableType GetCitusTableType(CitusTableCacheEntry *tableEntry);
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
CitusTableType tableType); CitusTableType tableType);
extern bool IsFirstShard(CitusTableCacheEntry *tableEntry, uint64 shardId);
bool HasDistributionKey(Oid relationId); bool HasDistributionKey(Oid relationId);
bool HasDistributionKeyCacheEntry(CitusTableCacheEntry *tableEntry); bool HasDistributionKeyCacheEntry(CitusTableCacheEntry *tableEntry);
extern char * GetTableTypeName(Oid tableId); extern char * GetTableTypeName(Oid tableId);

View File

@ -37,11 +37,12 @@ extern MultiNode * SubqueryMultiNodeTree(Query *originalQuery,
Query *queryTree, Query *queryTree,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext);
extern DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown(Query * extern DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown(
originalQuery, Query *originalQuery,
PlannerRestrictionContext PlannerRestrictionContext
* *
plannerRestrictionContext); plannerRestrictionContext,
bool plannerPhase);
extern DeferredErrorMessage * DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, extern DeferredErrorMessage * DeferErrorIfCannotPushdownSubquery(Query *subqueryTree,
bool bool
outerMostQueryHasLimit); outerMostQueryHasLimit);

View File

@ -16,10 +16,12 @@
#include "pg_version_constants.h" #include "pg_version_constants.h"
#include "distributed/distributed_planner.h"
#include "distributed/errormessage.h" #include "distributed/errormessage.h"
#include "distributed/log_utils.h" #include "distributed/log_utils.h"
#include "distributed/relation_restriction_equivalence.h" #include "distributed/relation_restriction_equivalence.h"
extern bool EnableRecurringOuterJoinPushdown;
typedef struct RecursivePlanningContextInternal RecursivePlanningContext; typedef struct RecursivePlanningContextInternal RecursivePlanningContext;
typedef struct RangeTblEntryIndex typedef struct RangeTblEntryIndex
@ -32,7 +34,8 @@ extern PlannerRestrictionContext * GetPlannerRestrictionContext(
RecursivePlanningContext *recursivePlanningContext); RecursivePlanningContext *recursivePlanningContext);
extern List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery, extern List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext,
RouterPlanType routerPlan);
extern char * GenerateResultId(uint64 planId, uint32 subPlanId); extern char * GenerateResultId(uint64 planId, uint32 subPlanId);
extern Query * BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, extern Query * BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList,
char *resultId); char *resultId);
@ -51,6 +54,11 @@ extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry);
extern bool IsRelationLocalTableOrMatView(Oid relationId); extern bool IsRelationLocalTableOrMatView(Oid relationId);
extern bool ContainsReferencesToOuterQuery(Query *query); extern bool ContainsReferencesToOuterQuery(Query *query);
extern void UpdateVarNosInNode(Node *node, Index newVarNo); 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 */ #endif /* RECURSIVE_PLANNING_H */

View File

@ -235,6 +235,20 @@ DEPS = {
"multi_subquery_in_where_clause": TestDeps( "multi_subquery_in_where_clause": TestDeps(
"minimal_schedule", ["multi_behavioral_analytics_create_table"] "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"]
),
} }

View File

@ -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_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_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) 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) INSERT INTO agg_events (user_id)
SELECT SELECT
raw_events_second.user_id raw_events_second.user_id
FROM FROM
reference_table LEFT JOIN raw_events_second ON reference_table.user_id = raw_events_second.user_id; 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: 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: 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: a push down safe left join with recurring left side
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: no shard pruning constraints on raw_events_second found
DEBUG: Wrapping relation "raw_events_second" to a subquery DEBUG: shard count after pruning for raw_events_second: 4
DEBUG: Router planner cannot handle multi-shard select queries DEBUG: assigned task to node localhost:xxxxx
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM multi_insert_select.raw_events_second WHERE true DEBUG: assigned task to node localhost:xxxxx
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: assigned task to node localhost:xxxxx
DEBUG: Creating router plan DEBUG: assigned task to node localhost:xxxxx
DEBUG: Collecting INSERT ... SELECT results on coordinator 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) INSERT INTO agg_events (user_id)
SELECT SELECT
raw_events_first.user_id raw_events_first.user_id
@ -3378,7 +3385,7 @@ $$);
Task Count: 1 Task Count: 1
(4 rows) (4 rows)
-- 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($$ 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); 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);
$$); $$);
@ -3392,11 +3399,23 @@ $$);
Task Count: 4 Task Count: 4
(6 rows) (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($$ SELECT coordinator_plan($$
EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT id FROM ref_table_1 LEFT JOIN dist_table_5 USING(id); EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT id FROM ref_table_1 LEFT JOIN dist_table_5 USING(id);
$$); $$);
coordinator_plan 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
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT) Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator INSERT/SELECT method: pull to coordinator

View File

@ -62,12 +62,9 @@ RIGHT JOIN (
RIGHT JOIN tbl_dist1 USING (id) RIGHT JOIN tbl_dist1 USING (id)
) AS table_4 USING (id); ) AS table_4 USING (id);
DEBUG: CTE cte_0 is going to be inlined via distributed planning 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: generating subplan XXX_1 for subquery SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN multi_recursive.tbl_dist1 USING (id))
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: push down of limit count: 0 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: 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: 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 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: 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: 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: 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: generating subplan XXX_1 for subquery SELECT tbl_ref1.id FROM (multi_recursive.tbl_ref1 LEFT JOIN multi_recursive.tbl_dist1 USING (id))
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: 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: 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_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_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: 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 DEBUG: recursively planning left side of the right join since the outer side is a recurring rel

View File

@ -208,12 +208,11 @@ FROM
20 | 0 20 | 0
(1 row) (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 SELECT
min(r_custkey), max(r_custkey) min(r_custkey), max(r_custkey)
FROM FROM
multi_outer_join_left_hash a RIGHT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey); 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 min | max
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 | 15 1 | 15
@ -323,12 +322,11 @@ FROM
25 | 1 25 | 1
(1 row) (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 SELECT
min(r_custkey), max(r_custkey) min(r_custkey), max(r_custkey)
FROM FROM
multi_outer_join_left_hash a RIGHT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey); 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 min | max
--------------------------------------------------------------------- ---------------------------------------------------------------------
11 | 30 11 | 30

View File

@ -1,7 +1,7 @@
-- --
-- multi subquery complex queries aims to expand existing subquery pushdown -- multi subquery complex queries aims to expand existing subquery pushdown
-- regression tests to cover more caeses -- 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 -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
-- SET citus.next_shard_id TO 1400000; -- SET citus.next_shard_id TO 1400000;
@ -539,15 +539,12 @@ SELECT * FROM
SELECT user_id FROM user_buy_test_table) sub SELECT user_id FROM user_buy_test_table) sub
ORDER BY 1 DESC; ORDER BY 1 DESC;
DEBUG: Router planner cannot handle multi-shard select queries 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: 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: 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: generating subplan XXX_2 for subquery SELECT user_id FROM public.user_buy_test_table
DEBUG: Creating router plan 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: 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 DEBUG: Creating router plan
user_id user_id
@ -2452,5 +2449,4 @@ SELECT count(*) FROM (SELECT ref1.* FROM users_ref_test_table ref1 INNER JOIN us
(1 row) (1 row)
DROP TABLE user_buy_test_table; DROP TABLE user_buy_test_table;
DROP TABLE users_ref_test_table;
DROP TABLE users_return_test_table; DROP TABLE users_return_test_table;

View File

@ -146,15 +146,15 @@ LIMIT 3;
3 3
(3 rows) (3 rows)
-- outer join could still recur -- outer join pushed down
SELECT SELECT
DISTINCT user_id DISTINCT user_id
FROM FROM
users_table RIGHT JOIN users_reference_table USING (user_id) users_table RIGHT JOIN users_reference_table USING (user_id)
WHERE WHERE
users_reference_table.value_2 IN (users_reference_table.value_2, random()*0) IN
(SELECT (SELECT
value_2 value_2, 0
FROM FROM
events_table events_table
WHERE WHERE
@ -162,7 +162,13 @@ WHERE
) )
ORDER BY user_id ORDER BY user_id
LIMIT 3; 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 -- subqueries in WHERE with IN operator without equality
SELECT SELECT
users_table.user_id, count(*) users_table.user_id, count(*)

View File

@ -339,13 +339,46 @@ DEBUG: Router planner cannot handle multi-shard select queries
4 | 130 4 | 130
(2 rows) (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 client_min_messages;
RESET search_path; RESET search_path;
DROP SCHEMA pg17_corr_subq_folding CASCADE; 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 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.users
drop cascades to table pg17_corr_subq_folding.events 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 -- Queries with outer joins with pseudoconstant quals work only in PG17
-- Relevant PG17 commit: -- Relevant PG17 commit:
-- https://github.com/postgres/postgres/commit/9e9931d2b -- https://github.com/postgres/postgres/commit/9e9931d2b

View File

@ -283,13 +283,57 @@ DEBUG: Router planner cannot handle multi-shard select queries
4 | 130 4 | 130
(2 rows) (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 client_min_messages;
RESET search_path; RESET search_path;
DROP SCHEMA pg17_corr_subq_folding CASCADE; 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 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.users
drop cascades to table pg17_corr_subq_folding.events 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 -- Queries with outer joins with pseudoconstant quals work only in PG17
-- Relevant PG17 commit: -- Relevant PG17 commit:
-- https://github.com/postgres/postgres/commit/9e9931d2b -- https://github.com/postgres/postgres/commit/9e9931d2b

View File

@ -1985,11 +1985,8 @@ INSERT INTO raw_events_second (user_id, value_1)
SELECT (a+5)*-1, b FROM cte; SELECT (a+5)*-1, b FROM cte;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: CTE cte is going to be inlined via distributed planning 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: 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: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel 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: 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: Collecting INSERT ... SELECT results on coordinator DEBUG: Collecting INSERT ... SELECT results on coordinator
-- .. and via SELECT's cte list too -- .. and via SELECT's cte list too
INSERT INTO raw_events_second (user_id, value_1) INSERT INTO raw_events_second (user_id, value_1)
@ -2000,11 +1997,8 @@ WITH cte AS (
SELECT (a+5)*2, b FROM cte; SELECT (a+5)*2, b FROM cte;
DEBUG: CTE cte is going to be inlined via distributed planning 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: 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: 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: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel 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: 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: Collecting INSERT ... SELECT results on coordinator DEBUG: Collecting INSERT ... SELECT results on coordinator
-- using set operations -- using set operations
INSERT INTO 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: 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: 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: 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: 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 DEBUG: Creating router plan
UPDATE nullkey_c1_t1 SET b = 5 FROM reference_table WHERE EXISTS ( 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: 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: 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: 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: 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 DEBUG: Creating router plan
DELETE FROM nullkey_c1_t1 USING reference_table WHERE EXISTS ( 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: 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: 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 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 -- test nested exec
CREATE FUNCTION dist_query_single_shard(p_key int) CREATE FUNCTION dist_query_single_shard(p_key int)
RETURNS bigint RETURNS bigint

View File

@ -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;

View File

@ -142,33 +142,18 @@ DEBUG: verifying table "dist_5_with_pkey"
-- basic cases -- basic cases
-- --
SELECT COUNT(*) FROM ref_1 LEFT JOIN dist_1 USING (a); 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
28 28
(1 row) (1 row)
SELECT COUNT(*) FROM ref_1 LEFT JOIN dist_1 USING (a,b); 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
15 15
(1 row) (1 row)
SELECT COUNT(*) FROM dist_1 RIGHT JOIN ref_1 USING (a); 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
28 28
@ -209,10 +194,6 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
-- distributed side is a subquery -- distributed side is a subquery
SELECT COUNT(*) FROM ref_1 LEFT JOIN (SELECT * FROM dist_1) q USING (a); 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
28 28
@ -356,7 +337,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
43 43
(1 row) (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); 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 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: 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 15
(1 row) (1 row)
-- subqury without FROM -- subquery without FROM
SELECT COUNT(*) FROM SELECT COUNT(*) FROM
(SELECT generate_series(1,10) AS a) t1 (SELECT generate_series(1,10) AS a) t1
JOIN dist_1 t2 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 SELECT COUNT(*) FROM ref_1 t1
LEFT JOIN dist_1 t2 USING (a,b) LEFT JOIN dist_1 t2 USING (a,b)
WHERE EXISTS (SELECT * FROM dist_1 t3 WHERE t1.a = t3.a); 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 ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
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
SELECT COUNT(*) FROM dist_1 t1 SELECT COUNT(*) FROM dist_1 t1
RIGHT JOIN ref_1 t2 USING (a,b) RIGHT JOIN ref_1 t2 USING (a,b)
WHERE EXISTS (SELECT * FROM dist_1 t3 WHERE t2.a = t3.a); 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 ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
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
-- "dist_1 t2" can't contribute to result set of the right join with -- "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 -- a tuple having "(t2.a) a = NULL" because t2 is in the inner side of
-- right join. For this reason, Postgres knows that <t2.a = t1.a> can -- right join. For this reason, Postgres knows that <t2.a = t1.a> can
@ -592,40 +563,69 @@ LATERAL
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
-- Qual is the same but top-level join is an anti-join. Right join -- Qual is the same but top-level join is an anti-join.
-- stays as is and hence requires recursive planning. -- 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 SELECT COUNT(*) FROM dist_1 t1
WHERE NOT EXISTS ( WHERE NOT EXISTS (
SELECT * FROM dist_1 t2 SELECT * FROM dist_1 t2
RIGHT JOIN ref_1 t3 USING (a) RIGHT JOIN ref_1 t3 USING (a)
WHERE t2.a = t1.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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
8 8
(1 row) (1 row)
SET client_min_messages TO DEBUG3;
-- This time the semi-join qual is <t3.a = t1.a> (not <<t2.a = t1.a>) -- This time the semi-join qual is <t3.a = t1.a> (not <<t2.a = t1.a>)
-- where t3 is the outer rel of the right join. Hence Postgres can't -- 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 -- replace right join with an inner join.
-- inner side of the right join since the outer side is a recurring -- Citus pushes down the right join between t2 and t3 with constraints on
-- rel. -- 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 SELECT COUNT(*) FROM dist_1 t1
WHERE EXISTS ( WHERE EXISTS (
SELECT * FROM dist_1 t2 SELECT * FROM dist_1 t2
RIGHT JOIN ref_1 t3 USING (a) RIGHT JOIN ref_1 t3 USING (a)
WHERE t3.a = t1.a WHERE t3.a = t1.a
); );
DEBUG: recursively planning left side of the right join since the outer side is a recurring rel DEBUG: no shard pruning constraints on dist_1 found
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: shard count after pruning for dist_1: 32
DEBUG: Wrapping relation "dist_1" "t2" to a subquery DEBUG: no shard pruning constraints on dist_1 found
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM recurring_outer_join.dist_1 t2 WHERE true DEBUG: shard count after pruning for dist_1: 32
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: 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
7 7
@ -636,12 +636,8 @@ WHERE NOT EXISTS (
SELECT * FROM dist_1 t2 SELECT * FROM dist_1 t2
RIGHT JOIN ref_1 t3 USING (a) RIGHT JOIN ref_1 t3 USING (a)
WHERE t3.a = t1.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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
8 8
@ -685,7 +681,8 @@ LEFT JOIN
( (
dist_1 t4 dist_1 t4
JOIN 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 (SELECT t6.a FROM dist_1 t6 RIGHT JOIN ref_1 t7 USING(a)) t5
USING(a) USING(a)
) q ) q
@ -696,19 +693,17 @@ LEFT JOIN
dist_1 t8 dist_1 t8
USING (a) USING (a)
WHERE t8.b IS NULL; 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 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: 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: 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 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: 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: 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: 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
10 10
@ -790,13 +785,10 @@ LEFT JOIN
USING(a) USING(a)
) t6 ) t6
USING (a); USING (a);
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel 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 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: recursively planning right side of the left join since the outer side is a recurring rel 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: 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)) 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -834,13 +826,10 @@ LEFT JOIN
USING(a) USING(a)
) t6 ) t6
USING (a); USING (a);
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel 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 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: recursively planning right side of the left join since the outer side is a recurring rel 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: 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)) 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -864,13 +853,10 @@ USING (a)
GROUP BY (t1.b) GROUP BY (t1.b)
HAVING t1.b > 200 HAVING t1.b > 200
ORDER BY 1,2; ORDER BY 1,2;
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel 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 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: recursively planning right side of the left join since the outer side is a recurring rel 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: 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 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 count | b
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -909,19 +895,12 @@ HAVING (
) )
) )
ORDER BY 1,2; ORDER BY 1,2;
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel 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 distributed relation "dist_1" "t7" 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, 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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: 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: 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_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: 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 count | b
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 | 102 0 | 102
@ -955,15 +934,12 @@ LEFT JOIN
USING(a) USING(a)
) t6 ) t6
USING (a); USING (a);
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel 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 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: Wrapping relation "citus_local_1" "t1" to a subquery 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: 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 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: 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)) 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1125,8 +1101,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
(1 row) (1 row)
-- cannot recursively plan because t3 (inner - distributed) -- cannot recursively plan because t3 (inner - distributed)
-- references t1 (outer - recurring) -- 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 (a); 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 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: 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 ERROR: cannot perform a lateral outer join when a distributed subquery references a reference table
@ -1179,10 +1155,6 @@ LEFT JOIN
) )
) t3 ) t3
USING (a); 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
28 28
@ -1197,10 +1169,6 @@ SELECT COUNT(*) FROM ref_1 t1
LEFT JOIN LEFT JOIN
my_view_1 t3 my_view_1 t3
USING (a); 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
28 28
@ -1345,27 +1313,21 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
(1 row) (1 row)
SELECT COUNT(*) FROM ref_1 t1 SELECT COUNT(*) FROM ref_1 t1
-- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted -- 2) It is also safe to push down this since the recurring outer side t1 and
-- to a recurring rel too. For this reason, subquery t8 is recursively -- distributed inner side t8 are joined on the distribution column.
-- planned because t7 is recurring already.
LEFT JOIN LEFT JOIN
( (
SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4
JOIN 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 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < 150) t6 USING (a)) as t7
USING(a) USING(a)
) t8 ) t8
USING (a); USING (a);
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
514 514
@ -1374,28 +1336,22 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
-- same test using a prepared statement -- same test using a prepared statement
PREPARE recurring_outer_join_p1 AS PREPARE recurring_outer_join_p1 AS
SELECT COUNT(*) FROM ref_1 t1 SELECT COUNT(*) FROM ref_1 t1
-- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted -- 2) It is also safe to push down this since the recurring outer side t1 and
-- to a recurring rel too. For this reason, subquery t8 is recursively -- distributed inner side t8 are joined on the distribution column.
-- planned because t7 is recurring already.
LEFT JOIN LEFT JOIN
( (
SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4
JOIN 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 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < $1) t6 USING (a)) as t7
USING(a) USING(a)
) t8 ) t8
USING (a); USING (a);
EXECUTE recurring_outer_join_p1(0); EXECUTE recurring_outer_join_p1(0);
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
142 142
@ -1403,14 +1359,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
EXECUTE recurring_outer_join_p1(100); EXECUTE recurring_outer_join_p1(100);
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
514 514
@ -1418,14 +1369,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
EXECUTE recurring_outer_join_p1(100); EXECUTE recurring_outer_join_p1(100);
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
514 514
@ -1433,14 +1379,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
EXECUTE recurring_outer_join_p1(10); EXECUTE recurring_outer_join_p1(10);
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
142 142
@ -1448,14 +1389,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
EXECUTE recurring_outer_join_p1(10); EXECUTE recurring_outer_join_p1(10);
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
142 142
@ -1464,14 +1400,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
EXECUTE recurring_outer_join_p1(1000); EXECUTE recurring_outer_join_p1(1000);
DEBUG: pathlist hook for columnar table am DEBUG: pathlist hook for columnar table am
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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
514 514
@ -1479,14 +1410,9 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c
EXECUTE recurring_outer_join_p1(1000); 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: 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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))
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))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
514 514
@ -1655,7 +1581,7 @@ SELECT * FROM ref_1 t36 WHERE (b,100,a) IN (
DISTINCT t31.b, DISTINCT t31.b,
-- 1) we first search for such joins in the target list and recursively plan t33 -- 1) we first search for such joins in the target list and recursively plan t33
-- because t32 is recurring -- 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) (SELECT t34.a)
FROM ref_1 t35 FROM ref_1 t35
LEFT JOIN dist_1 t31 USING (a,b) 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 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: 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: 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: generating subplan XXX_1 for subquery SELECT 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: 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 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: recursively planning right side of the left join since the outer side is a recurring rel 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: 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: 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_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 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: 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 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: 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: 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_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: 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: 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: push down of limit count: 10 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: 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: recursively planning right side of the left join since the outer side is a recurring rel 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 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: recursively planning right side of the left join since the outer side is a recurring rel 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: 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: 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: 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_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: 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 a | b
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 | 11 1 | 11
@ -1899,10 +1814,10 @@ BEGIN;
SELECT t1.a, t1.b FROM ref_1 t1 SELECT t1.a, t1.b FROM ref_1 t1
LEFT JOIN LEFT JOIN
( (
SELECT * FROM dist_1 t2 WHERE EXISTS ( SELECT DISTINCT ON (a) * FROM dist_1 t2 WHERE EXISTS (
SELECT * FROM dist_1 t4 SELECT * FROM dist_1 t4
WHERE t4.a = t2.a WHERE t4.a = t2.a
) ) ORDER BY a, b
) t3 ) t3
USING (a) USING (a)
) q ) q
@ -1910,7 +1825,7 @@ BEGIN;
RETURNING *; RETURNING *;
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel 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: 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 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 a | b | a | b
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1955,7 +1870,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE recurring_out
(7 rows) (7 rows)
ROLLBACK; ROLLBACK;
-- INSERT .. SELECT: pull to coordinator -- INSERT .. SELECT: Repartitioned
BEGIN; BEGIN;
DELETE FROM ref_1 WHERE a IS NULL; DELETE FROM ref_1 WHERE a IS NULL;
INSERT INTO dist_1 INSERT INTO dist_1
@ -1964,11 +1879,22 @@ BEGIN;
LEFT JOIN dist_1 t2 LEFT JOIN dist_1 t2
ON (t1.a = t2.a); ON (t1.a = t2.a);
DEBUG: cannot perform a lateral outer join when a distributed subquery references a reference table 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 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: 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: 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: 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 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: 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 DEBUG: Collecting INSERT ... SELECT results on coordinator
ROLLBACK; ROLLBACK;
-- INSERT .. SELECT: repartitioned (due to <t1.a*3>) -- INSERT .. SELECT: repartitioned (due to <t1.a*3>)

View File

@ -701,7 +701,7 @@ RESET citus.enable_repartition_joins;
-- ref_table, this increments query_execution_single_shard by 1 because it -- ref_table, this increments query_execution_single_shard by 1 because it
-- is a single shard query. -- is a single shard query.
CALL exec_query_and_check_query_counters($$ 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 1, 1
); );

View File

@ -235,6 +235,7 @@ test: local_table_join
test: local_dist_join_mixed test: local_dist_join_mixed
test: citus_local_dist_joins test: citus_local_dist_joins
test: recurring_outer_join test: recurring_outer_join
test: recurring_join_pushdown
test: query_single_shard_table test: query_single_shard_table
test: insert_select_single_shard_table test: insert_select_single_shard_table
test: pg_dump test: pg_dump

View File

@ -577,12 +577,14 @@ FROM
FROM FROM
raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id; 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) INSERT INTO agg_events (user_id)
SELECT SELECT
raw_events_second.user_id raw_events_second.user_id
FROM FROM
reference_table LEFT JOIN raw_events_second ON reference_table.user_id = raw_events_second.user_id; 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) INSERT INTO agg_events (user_id)
SELECT SELECT
raw_events_first.user_id 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; 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($$ 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); 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($$ SELECT coordinator_plan($$
EXPLAIN (COSTS FALSE) INSERT INTO dist_table_5 SELECT id FROM ref_table_1 LEFT JOIN dist_table_5 USING(id); 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); CREATE TABLE loc_table_1(id int);
-- verify that insert select cannot be pushed down when it contains join between local and distributed tables. -- verify that insert select cannot be pushed down when it contains join between local and distributed tables.

View File

@ -159,7 +159,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */); 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 SELECT
min(r_custkey), max(r_custkey) min(r_custkey), max(r_custkey)
FROM FROM
@ -259,7 +259,7 @@ FROM
ON (l_custkey = r_custkey AND r_custkey = 21); 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 SELECT
min(r_custkey), max(r_custkey) min(r_custkey), max(r_custkey)
FROM FROM

View File

@ -1,7 +1,7 @@
-- --
-- multi subquery complex queries aims to expand existing subquery pushdown -- multi subquery complex queries aims to expand existing subquery pushdown
-- regression tests to cover more caeses -- 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 -- 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; 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 user_buy_test_table;
DROP TABLE users_ref_test_table;
DROP TABLE users_return_test_table; DROP TABLE users_return_test_table;

View File

@ -126,15 +126,15 @@ WHERE
ORDER BY user_id ORDER BY user_id
LIMIT 3; LIMIT 3;
-- outer join could still recur -- outer join pushed down
SELECT SELECT
DISTINCT user_id DISTINCT user_id
FROM FROM
users_table RIGHT JOIN users_reference_table USING (user_id) users_table RIGHT JOIN users_reference_table USING (user_id)
WHERE WHERE
users_reference_table.value_2 IN (users_reference_table.value_2, random()*0) IN
(SELECT (SELECT
value_2 value_2, 0
FROM FROM
events_table events_table
WHERE WHERE

View File

@ -169,8 +169,26 @@ WHERE d1.user_id = users.user_id
AND users.dept IN (3,4) AND users.dept IN (3,4)
AND users.user_id = d2.user_id) dt AND users.user_id = d2.user_id) dt
GROUP BY dept; 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; RESET search_path;
DROP SCHEMA pg17_corr_subq_folding CASCADE; DROP SCHEMA pg17_corr_subq_folding CASCADE;

View File

@ -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;

View File

@ -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); 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); 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 -- 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) ON (t1.a = t2.a)
WHERE t1.a IN (SELECT a FROM dist_1 t3); WHERE t1.a IN (SELECT a FROM dist_1 t3);
-- subqury without FROM -- subquery without FROM
SELECT COUNT(*) FROM SELECT COUNT(*) FROM
(SELECT generate_series(1,10) AS a) t1 (SELECT generate_series(1,10) AS a) t1
JOIN dist_1 t2 JOIN dist_1 t2
@ -296,8 +296,11 @@ LATERAL
WHERE r1.a > dist_1.b WHERE r1.a > dist_1.b
) as foo; ) as foo;
-- Qual is the same but top-level join is an anti-join. Right join -- Qual is the same but top-level join is an anti-join.
-- stays as is and hence requires recursive planning. -- 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 SELECT COUNT(*) FROM dist_1 t1
WHERE NOT EXISTS ( WHERE NOT EXISTS (
SELECT * FROM dist_1 t2 SELECT * FROM dist_1 t2
@ -305,11 +308,17 @@ WHERE NOT EXISTS (
WHERE t2.a = t1.a WHERE t2.a = t1.a
); );
SET client_min_messages TO DEBUG3;
-- This time the semi-join qual is <t3.a = t1.a> (not <<t2.a = t1.a>) -- This time the semi-join qual is <t3.a = t1.a> (not <<t2.a = t1.a>)
-- where t3 is the outer rel of the right join. Hence Postgres can't -- 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 -- replace right join with an inner join.
-- inner side of the right join since the outer side is a recurring -- Citus pushes down the right join between t2 and t3 with constraints on
-- rel. -- 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 SELECT COUNT(*) FROM dist_1 t1
WHERE EXISTS ( WHERE EXISTS (
SELECT * FROM dist_1 t2 SELECT * FROM dist_1 t2
@ -324,6 +333,25 @@ WHERE NOT EXISTS (
WHERE t3.a = t1.a 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 -- more complex cases
-- --
@ -350,7 +378,8 @@ LEFT JOIN
( (
dist_1 t4 dist_1 t4
JOIN 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 (SELECT t6.a FROM dist_1 t6 RIGHT JOIN ref_1 t7 USING(a)) t5
USING(a) USING(a)
) q ) q
@ -584,8 +613,8 @@ LEFT JOIN
USING(a); USING(a);
-- cannot recursively plan because t3 (inner - distributed) -- cannot recursively plan because t3 (inner - distributed)
-- references t1 (outer - recurring) -- 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 (a); 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 (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 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); 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); USING (a);
SELECT COUNT(*) FROM ref_1 t1 SELECT COUNT(*) FROM ref_1 t1
-- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted -- 2) It is also safe to push down this since the recurring outer side t1 and
-- to a recurring rel too. For this reason, subquery t8 is recursively -- distributed inner side t8 are joined on the distribution column.
-- planned because t7 is recurring already.
LEFT JOIN LEFT JOIN
( (
SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4
JOIN 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 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < 150) t6 USING (a)) as t7
USING(a) USING(a)
) t8 ) t8
@ -728,14 +756,13 @@ USING (a);
-- same test using a prepared statement -- same test using a prepared statement
PREPARE recurring_outer_join_p1 AS PREPARE recurring_outer_join_p1 AS
SELECT COUNT(*) FROM ref_1 t1 SELECT COUNT(*) FROM ref_1 t1
-- 2) Since t8 is distributed and t1 is recurring, t8 needs be converted -- 2) It is also safe to push down this since the recurring outer side t1 and
-- to a recurring rel too. For this reason, subquery t8 is recursively -- distributed inner side t8 are joined on the distribution column.
-- planned because t7 is recurring already.
LEFT JOIN LEFT JOIN
( (
SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4 SELECT * FROM (SELECT * FROM ref_1 t2 RIGHT JOIN dist_1 t3 USING (a)) AS t4
JOIN 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 (SELECT * FROM ref_1 t5 LEFT JOIN (SELECT * FROM dist_2_columnar WHERE b < $1) t6 USING (a)) as t7
USING(a) USING(a)
) t8 ) t8
@ -845,7 +872,7 @@ SELECT * FROM ref_1 t36 WHERE (b,100,a) IN (
DISTINCT t31.b, DISTINCT t31.b,
-- 1) we first search for such joins in the target list and recursively plan t33 -- 1) we first search for such joins in the target list and recursively plan t33
-- because t32 is recurring -- 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) (SELECT t34.a)
FROM ref_1 t35 FROM ref_1 t35
LEFT JOIN dist_1 t31 USING (a,b) LEFT JOIN dist_1 t31 USING (a,b)
@ -962,10 +989,10 @@ BEGIN;
SELECT t1.a, t1.b FROM ref_1 t1 SELECT t1.a, t1.b FROM ref_1 t1
LEFT JOIN LEFT JOIN
( (
SELECT * FROM dist_1 t2 WHERE EXISTS ( SELECT DISTINCT ON (a) * FROM dist_1 t2 WHERE EXISTS (
SELECT * FROM dist_1 t4 SELECT * FROM dist_1 t4
WHERE t4.a = t2.a WHERE t4.a = t2.a
) ) ORDER BY a, b
) t3 ) t3
USING (a) USING (a)
) q ) q
@ -990,7 +1017,7 @@ BEGIN;
RETURNING *; RETURNING *;
ROLLBACK; ROLLBACK;
-- INSERT .. SELECT: pull to coordinator -- INSERT .. SELECT: Repartitioned
BEGIN; BEGIN;
DELETE FROM ref_1 WHERE a IS NULL; DELETE FROM ref_1 WHERE a IS NULL;
@ -1001,6 +1028,17 @@ BEGIN;
ON (t1.a = t2.a); ON (t1.a = t2.a);
ROLLBACK; 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 <t1.a*3>) -- INSERT .. SELECT: repartitioned (due to <t1.a*3>)
BEGIN; BEGIN;
INSERT INTO dist_1 INSERT INTO dist_1

View File

@ -453,7 +453,7 @@ RESET citus.enable_repartition_joins;
-- ref_table, this increments query_execution_single_shard by 1 because it -- ref_table, this increments query_execution_single_shard by 1 because it
-- is a single shard query. -- is a single shard query.
CALL exec_query_and_check_query_counters($$ 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 1, 1
); );