Plan outer joins through pushdown planning

pull/2481/head
Marco Slot 2018-11-15 23:05:11 +01:00
parent ad05634444
commit 1656b519c4
17 changed files with 521 additions and 479 deletions

View File

@ -48,8 +48,6 @@ static RuleEvalFunction RuleEvalFunctionArray[JOIN_RULE_LAST] = { 0 }; /* join r
/* Local functions forward declarations */
static JoinOrderNode * CreateFirstJoinOrderNode(FromExpr *fromExpr,
List *tableEntryList);
static bool JoinExprListWalker(Node *node, List **joinList);
static bool ExtractLeftMostRangeTableIndex(Node *node, int *rangeTableIndex);
static List * JoinOrderForTable(TableEntry *firstTable, List *tableEntryList,
@ -61,7 +59,6 @@ static List * LatestLargeDataTransfer(List *candidateJoinOrders);
static void PrintJoinOrderList(List *joinOrder);
static uint32 LargeDataTransferLocation(List *joinOrder);
static List * TableEntryListDifference(List *lhsTableList, List *rhsTableList);
static TableEntry * FindTableEntry(List *tableEntryList, uint32 tableId);
/* Local functions forward declarations for join evaluations */
static JoinOrderNode * EvaluateJoinRules(List *joinedTableList,
@ -95,152 +92,6 @@ static JoinOrderNode * MakeJoinOrderNode(TableEntry *tableEntry, JoinRuleType
TableEntry *anchorTable);
/*
* FixedJoinOrderList returns a list of join order nodes for the query in the order
* specified by the user. This is used to handle join trees that contain OUTER joins.
* The regular JoinOrderList currently assumes that all joins are inner-joins and can
* thus be arbitrarily reordered, which is not the case for OUTER joins. At some point
* we should merge these two functions.
*/
List *
FixedJoinOrderList(FromExpr *fromExpr, List *tableEntryList)
{
List *joinList = NIL;
ListCell *joinCell = NULL;
List *joinWhereClauseList = NIL;
List *joinOrderList = NIL;
List *joinedTableList = NIL;
JoinOrderNode *firstJoinNode = NULL;
JoinOrderNode *currentJoinNode = NULL;
ListCell *tableEntryCell = NULL;
foreach(tableEntryCell, tableEntryList)
{
TableEntry *rangeTableEntry = (TableEntry *) lfirst(tableEntryCell);
Oid relationId = rangeTableEntry->relationId;
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
if (cacheEntry->partitionMethod != DISTRIBUTE_BY_NONE &&
cacheEntry->hasUninitializedShardInterval)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning on this query"),
errdetail("Shards of relations in outer join queries must "
"have shard min/max values.")));
}
}
/* get the FROM section as a flattened list of JoinExpr nodes */
joinList = JoinExprList(fromExpr);
/* get the join clauses in the WHERE section for implicit joins */
joinWhereClauseList = JoinClauseList((List *) fromExpr->quals);
/* create join node for the first table */
firstJoinNode = CreateFirstJoinOrderNode(fromExpr, tableEntryList);
/* add first node to the join order */
joinOrderList = list_make1(firstJoinNode);
joinedTableList = list_make1(firstJoinNode->tableEntry);
currentJoinNode = firstJoinNode;
foreach(joinCell, joinList)
{
JoinExpr *joinExpr = (JoinExpr *) lfirst(joinCell);
List *onClauseList = list_copy((List *) joinExpr->quals);
List *joinClauseList = list_copy((List *) joinExpr->quals);
JoinType joinType = joinExpr->jointype;
RangeTblRef *nextRangeTableRef = NULL;
TableEntry *nextTable = NULL;
JoinOrderNode *nextJoinNode = NULL;
Node *rightArg = joinExpr->rarg;
/* get the table on the right hand side of the join */
if (IsA(rightArg, RangeTblRef))
{
nextRangeTableRef = (RangeTblRef *) rightArg;
}
else
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning on this query"),
errdetail("Subqueries in outer joins are not supported")));
}
nextTable = FindTableEntry(tableEntryList, nextRangeTableRef->rtindex);
if (joinType == JOIN_INNER)
{
/* also consider WHERE clauses for INNER joins */
joinClauseList = list_concat(joinClauseList, joinWhereClauseList);
}
/* find the best join rule type */
nextJoinNode = EvaluateJoinRules(joinedTableList, currentJoinNode,
nextTable, joinClauseList, joinType);
if (nextJoinNode->joinRuleType >= SINGLE_HASH_PARTITION_JOIN)
{
/* re-partitioning for OUTER joins is not implemented */
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot run outer join query if join is not on the "
"partition column"),
errdetail("Outer joins requiring repartitioning are not "
"supported.")));
}
if (joinType != JOIN_INNER)
{
/* preserve non-join clauses for OUTER joins */
nextJoinNode->joinClauseList = onClauseList;
}
/* add next node to the join order */
joinOrderList = lappend(joinOrderList, nextJoinNode);
joinedTableList = lappend(joinedTableList, nextTable);
currentJoinNode = nextJoinNode;
}
if (LogMultiJoinOrder)
{
PrintJoinOrderList(joinOrderList);
}
return joinOrderList;
}
/*
* CreateFirstJoinOrderNode creates the join order node for the left-most table in the
* join tree.
*/
static JoinOrderNode *
CreateFirstJoinOrderNode(FromExpr *fromExpr, List *tableEntryList)
{
JoinOrderNode *firstJoinNode = NULL;
TableEntry *firstTable = NULL;
JoinRuleType firstJoinRule = JOIN_RULE_INVALID_FIRST;
Var *firstPartitionColumn = NULL;
char firstPartitionMethod = '\0';
int rangeTableIndex = 0;
ExtractLeftMostRangeTableIndex((Node *) fromExpr, &rangeTableIndex);
firstTable = FindTableEntry(tableEntryList, rangeTableIndex);
firstPartitionColumn = PartitionColumn(firstTable->relationId,
firstTable->rangeTableId);
firstPartitionMethod = PartitionMethod(firstTable->relationId);
firstJoinNode = MakeJoinOrderNode(firstTable, firstJoinRule,
firstPartitionColumn,
firstPartitionMethod,
firstTable);
return firstJoinNode;
}
/*
* JoinExprList flattens the JoinExpr nodes in the FROM expression and translate implicit
* joins to inner joins. This function does not consider (right-)nested joins.
@ -274,6 +125,8 @@ JoinExprList(FromExpr *fromExpr)
newJoinExpr->jointype = JOIN_INNER;
newJoinExpr->rarg = (Node *) nextRangeTableRef;
newJoinExpr->quals = NULL;
joinList = lappend(joinList, newJoinExpr);
}
JoinExprListWalker(nextNode, &joinList);
@ -756,27 +609,6 @@ TableEntryListDifference(List *lhsTableList, List *rhsTableList)
}
/*
* Finds the table entry in tableEntryList with the given range table id.
*/
static TableEntry *
FindTableEntry(List *tableEntryList, uint32 tableId)
{
ListCell *tableEntryCell = NULL;
foreach(tableEntryCell, tableEntryList)
{
TableEntry *tableEntry = (TableEntry *) lfirst(tableEntryCell);
if (tableEntry->rangeTableId == tableId)
{
return tableEntry;
}
}
return NULL;
}
/*
* EvaluateJoinRules takes in a list of already joined tables and a candidate
* next table, evaluates different join rules between the two tables, and finds

View File

@ -68,9 +68,6 @@ static bool FullCompositeFieldList(List *compositeFieldList);
static bool HasUnsupportedJoinWalker(Node *node, void *context);
static bool ErrorHintRequired(const char *errorHint, Query *queryTree);
static bool HasTablesample(Query *queryTree);
static bool HasOuterJoin(Query *queryTree);
static bool HasOuterJoinWalker(Node *node, void *maxJoinLevel);
static bool HasComplexJoinOrder(Query *queryTree);
static bool HasComplexRangeTableType(Query *queryTree);
static bool IsReadIntermediateResultFunction(Node *node);
static bool ExtractFromExpressionWalker(Node *node,
@ -717,8 +714,6 @@ MultiNodeTree(Query *queryTree)
}
else
{
bool hasOuterJoin = false;
/*
* We calculate the join order using the list of tables in the query and
* the join clauses between them. Note that this function owns the table
@ -734,17 +729,8 @@ MultiNodeTree(Query *queryTree)
/* add collect nodes on top of the multi table nodes */
collectTableList = AddMultiCollectNodes(tableNodeList);
hasOuterJoin = HasOuterJoin(queryTree);
if (hasOuterJoin)
{
/* use the user-defined join order when there are outer joins */
joinOrderList = FixedJoinOrderList(queryTree->jointree, tableEntryList);
}
else
{
/* find best join order for commutative inner joins */
joinOrderList = JoinOrderList(tableEntryList, joinClauseList);
}
/* find best join order for commutative inner joins */
joinOrderList = JoinOrderList(tableEntryList, joinClauseList);
/* build join tree using the join order and collected tables */
joinTreeNode = MultiJoinTree(joinOrderList, collectTableList, joinClauseList);
@ -824,7 +810,6 @@ DeferErrorIfQueryNotSupported(Query *queryTree)
char *errorMessage = NULL;
bool hasTablesample = false;
bool hasUnsupportedJoin = false;
bool hasComplexJoinOrder = false;
bool hasComplexRangeTableType = false;
bool preconditionsSatisfied = true;
StringInfo errorInfo = NULL;
@ -912,14 +897,6 @@ DeferErrorIfQueryNotSupported(Query *queryTree)
errorHint = joinHint;
}
hasComplexJoinOrder = HasComplexJoinOrder(queryTree);
if (hasComplexJoinOrder)
{
preconditionsSatisfied = false;
errorMessage = "could not run distributed query with complex join orders";
errorHint = joinHint;
}
hasComplexRangeTableType = HasComplexRangeTableType(queryTree);
if (hasComplexRangeTableType)
{
@ -1134,76 +1111,6 @@ DeferErrorIfUnsupportedSubqueryRepartition(Query *subqueryTree)
}
/*
* HasOuterJoin returns true if query has a outer join.
*/
static bool
HasOuterJoin(Query *queryTree)
{
bool hasOuterJoin = HasOuterJoinWalker((Node *) queryTree->jointree, NULL);
return hasOuterJoin;
}
/*
* HasOuterJoinWalker returns true if the query has an outer join. The context
* parameter should be NULL.
*/
static bool
HasOuterJoinWalker(Node *node, void *context)
{
bool hasOuterJoin = false;
if (node == NULL)
{
return false;
}
if (IsA(node, JoinExpr))
{
JoinExpr *joinExpr = (JoinExpr *) node;
JoinType joinType = joinExpr->jointype;
if (IS_OUTER_JOIN(joinType))
{
hasOuterJoin = true;
}
}
if (!hasOuterJoin)
{
hasOuterJoin = expression_tree_walker(node, HasOuterJoinWalker, NULL);
}
return hasOuterJoin;
}
/*
* HasComplexJoinOrder returns true if join tree is not a left-handed tree i.e.
* it has a join expression in at least one right argument.
*/
static bool
HasComplexJoinOrder(Query *queryTree)
{
bool hasComplexJoinOrder = false;
List *joinList = NIL;
ListCell *joinCell = NULL;
joinList = JoinExprList(queryTree->jointree);
foreach(joinCell, joinList)
{
JoinExpr *joinExpr = lfirst(joinCell);
if (IsA(joinExpr->rarg, JoinExpr))
{
hasComplexJoinOrder = true;
break;
}
}
return hasComplexJoinOrder;
}
/*
* HasComplexRangeTableType checks if the given query tree contains any complex
* range table types. For this, the function walks over all range tables in the

View File

@ -3483,7 +3483,6 @@ JoinSequenceArray(List *rangeTableFragmentsList, Query *jobQuery, List *depended
foreach(joinExprCell, joinExprList)
{
JoinExpr *joinExpr = (JoinExpr *) lfirst(joinExprCell);
JoinType joinType = joinExpr->jointype;
RangeTblRef *rightTableRef = (RangeTblRef *) joinExpr->rarg;
JoinSequenceNode *nextJoinSequenceNode = NULL;
uint32 nextRangeTableId = rightTableRef->rtindex;
@ -3551,44 +3550,6 @@ JoinSequenceArray(List *rangeTableFragmentsList, Query *jobQuery, List *depended
continue;
}
/*
* Check if this is a broadcast outer join, meaning the inner table has only
* 1 shard.
*
* Broadcast outer join is a special case. In a left join, we want to join
* every fragment on the left with the one fragment on the right to ensure
* that all results from the left are included. As an optimization, we could
* perform these joins with any empty set instead of an actual fragment, but
* in any case they must not be pruned.
*/
if (IS_OUTER_JOIN(joinType))
{
int innerRangeTableId = 0;
List *tableFragments = NIL;
int fragmentCount = 0;
if (joinType == JOIN_RIGHT)
{
innerRangeTableId = existingRangeTableId;
}
else
{
/*
* Note: For a full join the logical planner ensures a 1-1 mapping,
* thus it is sufficient to check one side.
*/
innerRangeTableId = nextRangeTableId;
}
tableFragments = FindRangeTableFragmentsList(rangeTableFragmentsList,
innerRangeTableId);
fragmentCount = list_length(tableFragments);
if (fragmentCount == 1)
{
continue;
}
}
leftPartitioned = PartitionedOnColumn(leftColumn, rangeTableList,
dependedJobList);
rightPartitioned = PartitionedOnColumn(rightColumn, rangeTableList,

View File

@ -62,6 +62,7 @@ bool SubqueryPushdown = false; /* is subquery pushdown enabled */
static bool JoinTreeContainsSubqueryWalker(Node *joinTreeNode, void *context);
static bool IsFunctionRTE(Node *node);
static bool IsNodeQuery(Node *node);
static bool IsOuterJoinExpr(Node *node);
static bool WindowPartitionOnDistributionColumn(Query *query);
static DeferredErrorMessage * DeferErrorIfFromClauseRecurs(Query *queryTree);
static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin(
@ -82,6 +83,8 @@ static void UpdateVarMappingsForExtendedOpNode(List *columnList,
List *subqueryTargetEntryList);
static MultiTable * MultiSubqueryPushdownTable(Query *subquery);
static List * CreateSubqueryTargetEntryList(List *columnList);
static bool RelationInfoContainsOnlyRecurringTuples(PlannerInfo *plannerInfo,
RelOptInfo *relationInfo);
/*
@ -125,6 +128,15 @@ ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery)
return true;
}
/*
* We handle outer joins as subqueries, since the join order planner
* does not know how to handle them.
*/
if (FindNodeCheck((Node *) originalQuery->jointree, IsOuterJoinExpr))
{
return true;
}
/*
* Some unsupported join clauses in logical planner
* may be supported by subquery pushdown planner.
@ -251,6 +263,33 @@ IsNodeQuery(Node *node)
}
/*
* IsOuterJoinExpr returns whether the given node is an outer join expression.
*/
static bool
IsOuterJoinExpr(Node *node)
{
bool isOuterJoin = false;
if (node == NULL)
{
return false;
}
if (IsA(node, JoinExpr))
{
JoinExpr *joinExpr = (JoinExpr *) node;
JoinType joinType = joinExpr->jointype;
if (IS_OUTER_JOIN(joinType))
{
isOuterJoin = true;
}
}
return isOuterJoin;
}
/*
* SafeToPushdownWindowFunction checks if the query with window function is supported.
* It returns the result accordingly and modifies the error detail.
@ -641,6 +680,17 @@ DeferredErrorIfUnsupportedRecurringTuplesJoin(
if (joinType == JOIN_SEMI || joinType == JOIN_ANTI || joinType == JOIN_LEFT)
{
/*
* If there are only recurring tuples on the inner side of a join then
* we can push it down, regardless of whether the outer side is
* recurring or not. Otherwise, we check the outer side for recurring
* tuples.
*/
if (RelationInfoContainsOnlyRecurringTuples(plannerInfo, innerrel))
{
continue;
}
if (ShouldRecurseForRecurringTuplesJoinChecks(outerrel) &&
RelationInfoContainsRecurringTuples(plannerInfo, outerrel, &recurType))
{
@ -1164,6 +1214,33 @@ ShouldRecurseForRecurringTuplesJoinChecks(RelOptInfo *relOptInfo)
}
/*
* RelationInfoContainsOnlyRecurringTuples returns false if any of the relations in
* a RelOptInfo is not recurring.
*/
static bool
RelationInfoContainsOnlyRecurringTuples(PlannerInfo *plannerInfo,
RelOptInfo *relationInfo)
{
RecurringTuplesType recurType;
Relids relids = bms_copy(relationInfo->relids);
int relationId = -1;
while ((relationId = bms_first_member(relids)) >= 0)
{
RangeTblEntry *rangeTableEntry = plannerInfo->simple_rte_array[relationId];
/* relationInfo has this range table entry */
if (!IsRecurringRTE(rangeTableEntry, &recurType))
{
return false;
}
}
return true;
}
/*
* RelationInfoContainsRecurringTuples checks whether the relationInfo
* contains any recurring table expression, namely a reference table,

View File

@ -80,7 +80,6 @@ extern bool EnableSingleHashRepartitioning;
/* Function declaration for determining table join orders */
extern List * FixedJoinOrderList(FromExpr *fromExpr, List *tableEntryList);
extern List * JoinExprList(FromExpr *fromExpr);
extern List * JoinOrderList(List *rangeTableEntryList, List *joinClauseList);
extern List * ApplicableJoinClauses(List *leftTableIdList, uint32 rightTableId,

View File

@ -59,5 +59,4 @@ FROM (customer LEFT OUTER JOIN orders ON (c_custkey = o_custkey)) AS
test(c_custkey, c_nationkey)
INNER JOIN lineitem ON (test.c_custkey = l_orderkey)
LIMIT 10;
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator

View File

@ -237,8 +237,7 @@ LIMIT 3;
6 | 10
(1 row)
-- should error out since reference table exist on the left side
-- of the left lateral join
-- reference tables in a subquery in the WHERE clause are ok
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 1 AND value_1 < 3
AND value_2 >= 5
@ -298,6 +297,73 @@ SELECT user_id, value_2 FROM users_table WHERE
group by e1.user_id
HAVING sum(submit_card_info) > 0
)
ORDER BY 1, 2;
user_id | value_2
---------+---------
5 | 5
5 | 5
(2 rows)
-- reference table LEFT JOIN distributed table in WHERE is still not ok
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 1 AND value_1 < 3
AND value_2 >= 5
AND user_id IN
(
SELECT
e1.user_id
FROM (
-- Get the first time each user viewed the homepage.
SELECT
user_id,
1 AS view_homepage,
min(time) AS view_homepage_time
FROM events_reference_table
WHERE
event_type IN (1, 2)
GROUP BY user_id
) e1 LEFT JOIN LATERAL (
SELECT
user_id,
1 AS use_demo,
time AS use_demo_time
FROM events_table
WHERE
user_id = e1.user_id AND
event_type IN (2, 3)
ORDER BY time
) e2 ON true LEFT JOIN LATERAL (
SELECT
user_id,
1 AS enter_credit_card,
time AS enter_credit_card_time
FROM events_reference_table
WHERE
user_id = e2.user_id AND
event_type IN (3, 4)
ORDER BY time
) e3 ON true LEFT JOIN LATERAL (
SELECT
1 AS submit_card_info,
user_id,
time AS enter_credit_card_time
FROM events_reference_table
WHERE
user_id = e3.user_id AND
event_type IN (4, 5)
ORDER BY time
) e4 ON true LEFT JOIN LATERAL (
SELECT
1 AS see_bought_screen
FROM events_reference_table
WHERE
user_id = e4.user_id AND
event_type IN (5, 6)
ORDER BY time
) e5 ON true
group by e1.user_id
HAVING sum(submit_card_info) > 0
)
ORDER BY 1, 2;
ERROR: cannot pushdown the subquery
DETAIL: There exist a reference table in the outer part of the outer join
@ -425,8 +491,8 @@ FROM
WHERE user_id
NOT IN
(SELECT users_table.value_2 FROM users_table JOIN users_reference_table as u2 ON users_table.value_2 = u2.value_2);
DEBUG: generating subplan 16_1 for subquery SELECT users_table.value_2 FROM (public.users_table JOIN public.users_reference_table u2 ON ((users_table.value_2 OPERATOR(pg_catalog.=) u2.value_2)))
DEBUG: Plan 16 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_reference_table WHERE (NOT (user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('16_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))))
DEBUG: generating subplan 18_1 for subquery SELECT users_table.value_2 FROM (public.users_table JOIN public.users_reference_table u2 ON ((users_table.value_2 OPERATOR(pg_catalog.=) u2.value_2)))
DEBUG: Plan 18 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_reference_table WHERE (NOT (user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('18_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))))
count
-------
10
@ -441,8 +507,8 @@ FROM
(SELECT users_table.value_2
FROM users_table
JOIN users_reference_table AS u2 ON users_table.value_2 = u2.value_2);
DEBUG: generating subplan 18_1 for subquery SELECT users_table.value_2 FROM (public.users_table JOIN public.users_reference_table u2 ON ((users_table.value_2 OPERATOR(pg_catalog.=) u2.value_2)))
DEBUG: Plan 18 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_reference_table.user_id, random() AS random FROM public.users_reference_table) vals WHERE (NOT (user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('18_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))))
DEBUG: generating subplan 20_1 for subquery SELECT users_table.value_2 FROM (public.users_table JOIN public.users_reference_table u2 ON ((users_table.value_2 OPERATOR(pg_catalog.=) u2.value_2)))
DEBUG: Plan 20 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_reference_table.user_id, random() AS random FROM public.users_reference_table) vals WHERE (NOT (user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('20_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))))
count
-------
10

View File

@ -148,10 +148,12 @@ SELECT distinct(o_orderkey) FROM priority_orders join air_shipped_lineitems
(1 row)
-- left join support depends on flattening of the query
-- following query fails since the inner part is kept as subquery
SELECT * FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey);
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries in outer joins are not supported
SELECT o_orderkey, l_orderkey FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey) ORDER BY o_orderkey LIMIT 1;
o_orderkey | l_orderkey
------------+------------
2 |
(1 row)
-- however, this works
SELECT count(*) FROM priority_orders left join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
count
@ -159,11 +161,14 @@ SELECT count(*) FROM priority_orders left join lineitem_hash_part ON (o_orderkey
700
(1 row)
-- view at the inner side of is not supported
-- view on the inner side is supported
SELECT count(*) FROM priority_orders right join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries in outer joins are not supported
-- but view at the outer side is. This is essentially the same as a left join with arguments reversed.
count
-------
1706
(1 row)
-- view on the outer side is supported
SELECT count(*) FROM lineitem_hash_part right join priority_orders ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
count
-------

View File

@ -148,10 +148,12 @@ SELECT distinct(o_orderkey) FROM priority_orders join air_shipped_lineitems
(1 row)
-- left join support depends on flattening of the query
-- following query fails since the inner part is kept as subquery
SELECT * FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey);
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries in outer joins are not supported
SELECT o_orderkey, l_orderkey FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey) ORDER BY o_orderkey LIMIT 1;
o_orderkey | l_orderkey
------------+------------
2 |
(1 row)
-- however, this works
SELECT count(*) FROM priority_orders left join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
count
@ -159,11 +161,14 @@ SELECT count(*) FROM priority_orders left join lineitem_hash_part ON (o_orderkey
700
(1 row)
-- view at the inner side of is not supported
-- view on the inner side is supported
SELECT count(*) FROM priority_orders right join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries in outer joins are not supported
-- but view at the outer side is. This is essentially the same as a left join with arguments reversed.
count
-------
1706
(1 row)
-- view on the outer side is supported
SELECT count(*) FROM lineitem_hash_part right join priority_orders ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
count
-------

View File

@ -166,7 +166,11 @@ DEBUG: cannot use real time executor with repartition jobs
DEBUG: generating subplan 16_1 for subquery SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))
DEBUG: generating subplan 16_2 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 4)
DEBUG: Plan 16 query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT intermediate_result.user_id, intermediate_result.event_type FROM read_intermediate_result('16_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event_type integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('16_2'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer))))
ERROR: cannot pushdown the subquery
valid
-------
t
(1 row)
-- The inner subqueries and the subquery in WHERE are non-located joins
SELECT true AS valid FROM explain_json_2($$
SELECT foo_top.*, events_table.user_id FROM
@ -857,8 +861,6 @@ DEBUG: Plan 91 query after replacing subqueries and CTEs: SELECT count(*) AS co
(1 row)
-- a combination of subqueries in FROM and WHERE clauses
-- we actually recursively plan non colocated subqueries
-- pretty accurate, however, we hit our join checks, which seems too restrictive
SELECT true AS valid FROM explain_json_2($$
SELECT
@ -889,7 +891,11 @@ DEBUG: generating subplan 93_1 for subquery SELECT value_1, value_2 FROM public
DEBUG: generating subplan 93_2 for subquery SELECT value_1 FROM public.users_table WHERE (value_2 OPERATOR(pg_catalog.<) 1)
DEBUG: generating subplan 93_3 for subquery SELECT value_2 FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.<) 2)
DEBUG: Plan 93 query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((SELECT users_table.user_id FROM public.users_table) foo JOIN (SELECT users_table.user_id FROM public.users_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])) UNION SELECT users_table.user_id FROM public.users_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) a ON ((a.user_id OPERATOR(pg_catalog.=) foo.user_id))) JOIN (SELECT intermediate_result.value_1, intermediate_result.value_2 FROM read_intermediate_result('93_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer, value_2 integer)) bar ON ((foo.user_id OPERATOR(pg_catalog.=) bar.value_1))) WHERE ((bar.value_2 OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_1 FROM read_intermediate_result('93_2'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer))) AND (bar.value_1 OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('93_3'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))) AND (foo.user_id OPERATOR(pg_catalog.=) ANY (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2]))))))
ERROR: cannot pushdown the subquery
valid
-------
t
(1 row)
-- make sure that we don't pick the refeence table as
-- the anchor
SELECT true AS valid FROM explain_json_2($$

View File

@ -262,25 +262,29 @@ FROM
multi_outer_join_right a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
-- complex query tree should error out
-- Mix of outer joins on partition column
SELECT
*
l1.l_custkey
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_right r2 ON (l1.l_custkey = r2.r_custkey)
RIGHT JOIN multi_outer_join_left l2 ON (r2.r_custkey = l2.l_custkey);
RIGHT JOIN multi_outer_join_left l2 ON (r2.r_custkey = l2.l_custkey)
ORDER BY 1
LIMIT 1;
-- add an anti-join, this should also error out
-- add an anti-join
SELECT
*
l1.l_custkey
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_right r2 ON (l1.l_custkey = r2.r_custkey)
RIGHT JOIN multi_outer_join_left l2 ON (r2.r_custkey = l2.l_custkey)
WHERE
r1.r_custkey is NULL;
r1.r_custkey is NULL
ORDER BY 1
LIMIT 1;
-- Three way join 2-2-1 (local + broadcast join) should work
SELECT
@ -316,13 +320,14 @@ FROM
WHERE
l_custkey is NULL;
-- Cascading right join with single shard left most table should error out
-- Cascading right join with single shard left most table
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
RIGHT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey)
ORDER BY 1,2,3;
-- full outer join should work with 1-1 matched shards
SELECT
@ -381,15 +386,17 @@ FROM
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
-- inner (local) join + 2 shards left (dual partition) join should error out
-- inner (local) join + 2 shards left (dual partition) join
SELECT
t_custkey, l_custkey, r_custkey
FROM
multi_outer_join_third_reference t1
INNER JOIN multi_outer_join_left l1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY
t_custkey, l_custkey, r_custkey;
-- inner (local) join + 2 shards left (dual partition) join should error out
-- inner (local) join + 2 shards left (dual partition) join
SELECT
l_custkey, t_custkey, r_custkey
FROM
@ -416,7 +423,7 @@ FROM
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third_reference t1 ON (test.c_custkey = t1.t_custkey);
-- flattened out subqueries with outer joins are not supported
-- Outer joins with subqueries on distribution column
SELECT
l1.l_custkey,
count(*) as cnt

View File

@ -315,13 +315,14 @@ WHERE
l_custkey is NULL
ORDER BY 1;
-- Cascading right join with single shard left most table should error out
-- Cascading right join with single shard left most table should work
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right_hash r1 ON (t1.t_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_left_hash l1 ON (r1.r_custkey = l1.l_custkey);
RIGHT JOIN multi_outer_join_left_hash l1 ON (r1.r_custkey = l1.l_custkey)
ORDER BY l_custkey;
-- full outer join should work with 1-1 matched shards
SELECT
@ -386,13 +387,14 @@ FROM
LEFT JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
-- inner (local) join + 2 shards left (dual partition) join should error out
-- inner (local) join + 2 shards left (dual partition) join
SELECT
t_custkey, l_custkey, r_custkey
FROM
multi_outer_join_third_reference t1
INNER JOIN multi_outer_join_left_hash l1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey);
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
-- inner (local) join + 2 shards left (dual partition) join should work
SELECT
@ -450,6 +452,19 @@ FROM
multi_outer_join_third_reference ON (t_custkey = r_custkey)
ORDER BY 1;
-- complex join order with multiple children on the right
SELECT
count(*)
FROM
multi_outer_join_left_hash l1
LEFT JOIN (
multi_outer_join_right_reference r1
INNER JOIN
multi_outer_join_third_reference r2
ON (r_name = t_name)
) AS bar
ON (l_name = r_name);
-- DROP unused tables to clean up workspace
DROP TABLE multi_outer_join_left_hash;
DROP TABLE multi_outer_join_right_reference;

View File

@ -91,7 +91,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_third b ON (l_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_third" ]
min | max
-----+-----
|
@ -108,13 +107,11 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_third b ON (l_custkey = t_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: shard counts of co-located tables do not match
SELECT
min(t_custkey), max(t_custkey)
FROM
multi_outer_join_third a LEFT JOIN multi_outer_join_right_reference b ON (r_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_third" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
|
@ -128,7 +125,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -139,7 +135,6 @@ SELECT
count(*)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_nationkey = r_nationkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count
-------
28
@ -152,7 +147,6 @@ FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
16 | 20
@ -165,7 +159,6 @@ FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 5;
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 20
@ -179,7 +172,6 @@ FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 5 or r_custkey > 15;
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 5
@ -191,7 +183,6 @@ SELECT
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 5);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 1
@ -203,7 +194,6 @@ SELECT
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -215,7 +205,6 @@ SELECT
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -226,14 +215,12 @@ SELECT
min(r_custkey), max(r_custkey)
FROM
multi_outer_join_left a RIGHT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: shard counts of co-located tables do not match
-- Reverse right join should be same as left join
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -246,8 +233,11 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
min | max
-----+-----
1 | 20
(1 row)
-- empty tables
SELECT * FROM master_apply_delete_command('DELETE FROM multi_outer_join_left');
master_apply_delete_command
@ -272,7 +262,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
min | max
-----+-----
11 | 30
@ -283,8 +272,7 @@ SELECT
count(*)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_nationkey = r_nationkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- Anti-join should return customers for which there is no row in the right table
SELECT
min(l_custkey), max(l_custkey)
@ -292,7 +280,6 @@ FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
min | max
-----+-----
23 | 29
@ -305,7 +292,6 @@ FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 15;
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
min | max
-----+-----
23 | 29
@ -319,7 +305,6 @@ FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
WHERE
r_custkey = 21 or r_custkey < 10;
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
min | max
-----+-----
21 | 21
@ -331,7 +316,6 @@ SELECT
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
ON (l_custkey = r_custkey AND r_custkey = 21);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
count | count
-------+-------
17 | 1
@ -342,7 +326,6 @@ SELECT
min(r_custkey), max(r_custkey)
FROM
multi_outer_join_left a RIGHT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_right" ][ local partition join "multi_outer_join_left" ]
min | max
-----+-----
11 | 30
@ -353,32 +336,43 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
min | max
-----+-----
11 | 30
(1 row)
-- complex query tree should error out
-- Mix of outer joins on partition column
SELECT
*
l1.l_custkey
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_right r2 ON (l1.l_custkey = r2.r_custkey)
RIGHT JOIN multi_outer_join_left l2 ON (r2.r_custkey = l2.l_custkey);
ERROR: could not run distributed query with complex join orders
-- add an anti-join, this should also error out
RIGHT JOIN multi_outer_join_left l2 ON (r2.r_custkey = l2.l_custkey)
ORDER BY 1
LIMIT 1;
l_custkey
-----------
11
(1 row)
-- add an anti-join
SELECT
*
l1.l_custkey
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_right r2 ON (l1.l_custkey = r2.r_custkey)
RIGHT JOIN multi_outer_join_left l2 ON (r2.r_custkey = l2.l_custkey)
WHERE
r1.r_custkey is NULL;
ERROR: could not run distributed query with complex join orders
r1.r_custkey is NULL
ORDER BY 1
LIMIT 1;
l_custkey
-----------
(1 row)
-- Three way join 2-2-1 (local + broadcast join) should work
SELECT
l_custkey, r_custkey, t_custkey
@ -386,7 +380,6 @@ FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -415,7 +408,8 @@ FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
ERROR: could not run distributed query with complex join orders
ERROR: cannot pushdown the subquery
DETAIL: There exist a reference table in the outer part of the outer join
-- Right join with single shard left most table should work
SELECT
t_custkey, r_custkey, l_custkey
@ -423,7 +417,6 @@ FROM
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
LOG: join order: [ "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -454,7 +447,6 @@ FROM
LEFT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey)
WHERE
l_custkey is NULL;
LOG: join order: [ "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
13 | 13 |
@ -462,21 +454,41 @@ LOG: join order: [ "multi_outer_join_right" ][ reference join "multi_outer_join
19 | 19 |
(3 rows)
-- Cascading right join with single shard left most table should error out
-- Cascading right join with single shard left most table
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
ERROR: could not run distributed query with complex join orders
RIGHT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey)
ORDER BY 1,2,3;
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
11 | 11 | 11
12 | 12 | 12
14 | 14 | 14
16 | 16 | 16
17 | 17 | 17
18 | 18 | 18
20 | 20 | 20
21 | 21 | 21
22 | 22 | 22
24 | 24 | 24
26 | 26 | 26
27 | 27 | 27
28 | 28 | 28
30 | 30 | 30
| | 23
| | 25
| | 29
(17 rows)
-- full outer join should work with 1-1 matched shards
SELECT
l_custkey, r_custkey
FROM
multi_outer_join_left l1
FULL JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
l_custkey | r_custkey
-----------+-----------
11 | 11
@ -509,7 +521,6 @@ FROM
FULL JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
WHERE
r_custkey is NULL;
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
l_custkey | r_custkey
-----------+-----------
23 |
@ -525,7 +536,6 @@ FROM
FULL JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
WHERE
l_custkey is NULL;
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
l_custkey | r_custkey
-----------+-----------
| 15
@ -541,7 +551,6 @@ FROM
FULL JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
WHERE
l_custkey is NULL or r_custkey is NULL;
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ]
l_custkey | r_custkey
-----------+-----------
| 15
@ -558,8 +567,7 @@ SELECT
FROM
multi_outer_join_left l1
FULL JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: shard counts of co-located tables do not match
-- inner join + single shard left join should work
SELECT
l_custkey, r_custkey, t_custkey
@ -567,7 +575,6 @@ FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -593,7 +600,6 @@ FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -615,23 +621,43 @@ LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_
30 | 30 | 30
(17 rows)
-- inner (local) join + 2 shards left (dual partition) join should error out
-- inner (local) join + 2 shards left (dual partition) join
SELECT
t_custkey, l_custkey, r_custkey
FROM
multi_outer_join_third_reference t1
INNER JOIN multi_outer_join_left l1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
-- inner (local) join + 2 shards left (dual partition) join should error out
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY
t_custkey, l_custkey, r_custkey;
t_custkey | l_custkey | r_custkey
-----------+-----------+-----------
11 | 11 | 11
12 | 12 | 12
14 | 14 | 14
16 | 16 | 16
17 | 17 | 17
18 | 18 | 18
20 | 20 | 20
21 | 21 | 21
22 | 22 | 22
23 | 23 |
24 | 24 | 24
25 | 25 |
26 | 26 | 26
27 | 27 | 27
28 | 28 | 28
29 | 29 |
30 | 30 | 30
(17 rows)
-- inner (local) join + 2 shards left (dual partition) join
SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -662,7 +688,6 @@ FROM
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
WHERE
r_custkey is NULL;
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
23 | 23 |
@ -678,7 +703,6 @@ FROM
LEFT OUTER JOIN multi_outer_join_left l1 ON (l1.l_custkey = r1.r_custkey)) AS
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third_reference t1 ON (test.c_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_right" ][ local partition join "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ]
t_custkey
-----------
11
@ -700,7 +724,7 @@ LOG: join order: [ "multi_outer_join_right" ][ local partition join "multi_oute
30
(17 rows)
-- flattened out subqueries with outer joins are not supported
-- Outer joins with subqueries on distribution column
SELECT
l1.l_custkey,
count(*) as cnt
@ -717,8 +741,26 @@ LEFT JOIN (
GROUP BY l1.l_custkey
ORDER BY cnt DESC, l1.l_custkey DESC
LIMIT 20;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries in outer joins are not supported
l_custkey | cnt
-----------+-----
30 | 1
29 | 1
28 | 1
27 | 1
25 | 1
24 | 1
23 | 1
22 | 1
21 | 1
20 | 1
18 | 1
17 | 1
16 | 1
14 | 1
12 | 1
11 | 1
(16 rows)
-- Add a shard to the left table that overlaps with multiple shards in the right
\copy multi_outer_join_left FROM '@abs_srcdir@/data/customer.1.data' with delimiter '|'
-- All outer joins should error out
@ -726,20 +768,20 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot push down this subquery
DETAIL: Currently append partitioned relations with overlapping shard intervals are not supported
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a RIGHT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot push down this subquery
DETAIL: Currently append partitioned relations with overlapping shard intervals are not supported
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a FULL JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot push down this subquery
DETAIL: Currently append partitioned relations with overlapping shard intervals are not supported
SELECT
t_custkey
FROM
@ -747,8 +789,8 @@ FROM
LEFT OUTER JOIN multi_outer_join_left l1 ON (l1.l_custkey = r1.r_custkey)) AS
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third t1 ON (test.c_custkey = t1.t_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot push down this subquery
DETAIL: Currently append partitioned relations with overlapping shard intervals are not supported
-- simple test to ensure anti-joins work with hash-partitioned tables
CREATE TABLE left_values(val int);
SET citus.shard_count to 16;
@ -775,7 +817,6 @@ FROM
LEFT JOIN right_values AS r ON l.val = r.val
WHERE
r.val IS NULL;
LOG: join order: [ "left_values" ][ local partition join "right_values" ]
val | val
-----+-----
1 |

View File

@ -75,7 +75,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_third_reference b ON (l_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ]
min | max
-----+-----
|
@ -92,7 +91,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_third_reference b ON (l_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ]
min | max
-----+-----
1 | 20
@ -115,7 +113,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -126,7 +123,6 @@ SELECT
count(*)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_nationkey = r_nationkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count
-------
28
@ -139,7 +135,6 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
16 | 20
@ -152,7 +147,6 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 5;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 20
@ -166,7 +160,6 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 5 or r_custkey > 15;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 5
@ -178,7 +171,6 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 5);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 1
@ -190,7 +182,6 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -202,7 +193,6 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -213,14 +203,13 @@ SELECT
min(r_custkey), max(r_custkey)
FROM
multi_outer_join_left_hash a RIGHT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot pushdown the subquery
DETAIL: There exist a reference table in the outer part of the outer join
-- Reverse right join should be same as left join
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left_hash b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -265,7 +254,6 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_hash b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ]
min | max
-----+-----
1 | 30
@ -276,8 +264,7 @@ SELECT
count(*)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_hash b ON (l_nationkey = r_nationkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- Anti-join should return customers for which there is no row in the right table
SELECT
min(l_custkey), max(l_custkey)
@ -285,7 +272,6 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 10
@ -298,7 +284,6 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 15;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 15
@ -312,7 +297,6 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 21 or r_custkey < 10;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
21 | 21
@ -324,7 +308,6 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 21);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
25 | 1
@ -335,14 +318,13 @@ SELECT
min(r_custkey), max(r_custkey)
FROM
multi_outer_join_left_hash a RIGHT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot pushdown the subquery
DETAIL: There exist a reference table in the outer part of the outer join
-- Reverse right join should be same as left join
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left_hash b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 30
@ -356,8 +338,7 @@ FROM
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_right_reference r2 ON (l1.l_custkey = r2.r_custkey)
RIGHT JOIN multi_outer_join_left_hash l2 ON (r2.r_custkey = l2.l_custkey);
ERROR: could not run distributed query with complex join orders
HINT: Consider joining tables on partition column and have equal filter on joining columns.
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- add an anti-join, this should also error out
SELECT
*
@ -368,8 +349,7 @@ FROM
RIGHT JOIN multi_outer_join_left_hash l2 ON (r2.r_custkey = l2.l_custkey)
WHERE
r1.r_custkey is NULL;
ERROR: could not run distributed query with complex join orders
HINT: Consider joining tables on partition column and have equal filter on joining columns.
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- Three way join 2-1-1 (broadcast + broadcast join) should work
SELECT
l_custkey, r_custkey, t_custkey
@ -378,7 +358,6 @@ FROM
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey)
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
1 | |
@ -415,8 +394,8 @@ FROM
multi_outer_join_left_hash l1
LEFT JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
ERROR: could not run distributed query with complex join orders
HINT: Consider joining tables on partition column and have equal filter on joining columns.
ERROR: cannot pushdown the subquery
DETAIL: There exist a reference table in the outer part of the outer join
-- Right join with single shard left most table should work
SELECT
t_custkey, r_custkey, l_custkey
@ -425,7 +404,6 @@ FROM
RIGHT JOIN multi_outer_join_right_hash r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left_hash l1 ON (r1.r_custkey = l1.l_custkey)
ORDER BY 1,2,3;
LOG: join order: [ "multi_outer_join_right_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left_hash" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -460,7 +438,6 @@ FROM
WHERE
l_custkey is NULL
ORDER BY 1;
LOG: join order: [ "multi_outer_join_right_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left_hash" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
16 | 16 |
@ -470,15 +447,43 @@ LOG: join order: [ "multi_outer_join_right_hash" ][ reference join "multi_outer
20 | 20 |
(5 rows)
-- Cascading right join with single shard left most table should error out
-- Cascading right join with single shard left most table should work
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right_hash r1 ON (t1.t_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_left_hash l1 ON (r1.r_custkey = l1.l_custkey);
ERROR: could not run distributed query with complex join orders
HINT: Consider joining tables on partition column and have equal filter on joining columns.
RIGHT JOIN multi_outer_join_left_hash l1 ON (r1.r_custkey = l1.l_custkey)
ORDER BY l_custkey;
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
| | 1
| | 2
| | 3
| | 4
| | 5
| | 6
| | 7
| | 8
| | 9
| | 10
11 | 11 | 11
12 | 12 | 12
13 | 13 | 13
14 | 14 | 14
15 | 15 | 15
21 | 21 | 21
22 | 22 | 22
23 | 23 | 23
24 | 24 | 24
25 | 25 | 25
26 | 26 | 26
27 | 27 | 27
28 | 28 | 28
29 | 29 | 29
30 | 30 | 30
(25 rows)
-- full outer join should work with 1-1 matched shards
SELECT
l_custkey, r_custkey
@ -486,7 +491,6 @@ FROM
multi_outer_join_left_hash l1
FULL JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2;
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | r_custkey
-----------+-----------
1 |
@ -530,7 +534,6 @@ FROM
WHERE
r_custkey is NULL
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | r_custkey
-----------+-----------
1 |
@ -554,7 +557,6 @@ FROM
WHERE
l_custkey is NULL
ORDER BY 2;
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | r_custkey
-----------+-----------
| 16
@ -573,7 +575,6 @@ FROM
WHERE
l_custkey is NULL or r_custkey is NULL
ORDER BY 1,2 DESC;
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | r_custkey
-----------+-----------
1 |
@ -599,8 +600,8 @@ SELECT
FROM
multi_outer_join_left_hash l1
FULL JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
ERROR: cannot pushdown the subquery
DETAIL: There exist a reference table in the outer part of the outer join
-- inner join + single shard left join should work
SELECT
l_custkey, r_custkey, t_custkey
@ -609,7 +610,6 @@ FROM
INNER JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey)
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -637,7 +637,6 @@ FROM
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
@ -667,15 +666,43 @@ LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_
30 | 30 | 30
(25 rows)
-- inner (local) join + 2 shards left (dual partition) join should error out
-- inner (local) join + 2 shards left (dual partition) join
SELECT
t_custkey, l_custkey, r_custkey
FROM
multi_outer_join_third_reference t1
INNER JOIN multi_outer_join_left_hash l1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
DETAIL: Outer joins requiring repartitioning are not supported.
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
t_custkey | l_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
2 | 2 |
3 | 3 |
4 | 4 |
5 | 5 |
6 | 6 |
7 | 7 |
8 | 8 |
9 | 9 |
10 | 10 |
11 | 11 | 11
12 | 12 | 12
13 | 13 | 13
14 | 14 | 14
15 | 15 | 15
21 | 21 | 21
22 | 22 | 22
23 | 23 | 23
24 | 24 | 24
25 | 25 | 25
26 | 26 | 26
27 | 27 | 27
28 | 28 | 28
29 | 29 | 29
30 | 30 | 30
(25 rows)
-- inner (local) join + 2 shards left (dual partition) join should work
SELECT
l_custkey, t_custkey, r_custkey
@ -684,7 +711,6 @@ FROM
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
@ -724,7 +750,6 @@ FROM
WHERE
r_custkey is NULL
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
@ -748,7 +773,6 @@ FROM
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third_reference t1 ON (test.c_custkey = t1.t_custkey)
ORDER BY 1;
LOG: join order: [ "multi_outer_join_right_hash" ][ local partition join "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ]
t_custkey
-----------
11
@ -790,8 +814,30 @@ LEFT JOIN (
GROUP BY l1.l_custkey
ORDER BY cnt DESC, l1.l_custkey DESC
LIMIT 20;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries in outer joins are not supported
l_custkey | cnt
-----------+-----
30 | 1
29 | 1
28 | 1
27 | 1
25 | 1
24 | 1
23 | 1
22 | 1
21 | 1
15 | 1
14 | 1
13 | 1
12 | 1
11 | 1
10 | 1
9 | 1
8 | 1
7 | 1
6 | 1
5 | 1
(20 rows)
-- full join among reference tables should go thourgh router planner
SELECT
t_custkey, r_custkey
@ -833,6 +879,23 @@ ORDER BY 1;
30 | 30
(30 rows)
-- complex join order with multiple children on the right
SELECT
count(*)
FROM
multi_outer_join_left_hash l1
LEFT JOIN (
multi_outer_join_right_reference r1
INNER JOIN
multi_outer_join_third_reference r2
ON (r_name = t_name)
) AS bar
ON (l_name = r_name);
count
-------
25
(1 row)
-- DROP unused tables to clean up workspace
DROP TABLE multi_outer_join_left_hash;
DROP TABLE multi_outer_join_right_reference;

View File

@ -198,8 +198,7 @@ ORDER BY 2 DESC, 1 DESC
LIMIT 3;
-- should error out since reference table exist on the left side
-- of the left lateral join
-- reference tables in a subquery in the WHERE clause are ok
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 1 AND value_1 < 3
AND value_2 >= 5
@ -261,6 +260,69 @@ SELECT user_id, value_2 FROM users_table WHERE
)
ORDER BY 1, 2;
-- reference table LEFT JOIN distributed table in WHERE is still not ok
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 1 AND value_1 < 3
AND value_2 >= 5
AND user_id IN
(
SELECT
e1.user_id
FROM (
-- Get the first time each user viewed the homepage.
SELECT
user_id,
1 AS view_homepage,
min(time) AS view_homepage_time
FROM events_reference_table
WHERE
event_type IN (1, 2)
GROUP BY user_id
) e1 LEFT JOIN LATERAL (
SELECT
user_id,
1 AS use_demo,
time AS use_demo_time
FROM events_table
WHERE
user_id = e1.user_id AND
event_type IN (2, 3)
ORDER BY time
) e2 ON true LEFT JOIN LATERAL (
SELECT
user_id,
1 AS enter_credit_card,
time AS enter_credit_card_time
FROM events_reference_table
WHERE
user_id = e2.user_id AND
event_type IN (3, 4)
ORDER BY time
) e3 ON true LEFT JOIN LATERAL (
SELECT
1 AS submit_card_info,
user_id,
time AS enter_credit_card_time
FROM events_reference_table
WHERE
user_id = e3.user_id AND
event_type IN (4, 5)
ORDER BY time
) e4 ON true LEFT JOIN LATERAL (
SELECT
1 AS see_bought_screen
FROM events_reference_table
WHERE
user_id = e4.user_id AND
event_type IN (5, 6)
ORDER BY time
) e5 ON true
group by e1.user_id
HAVING sum(submit_card_info) > 0
)
ORDER BY 1, 2;
-- non-partition key equality with reference table
SELECT
user_id, count(*)

View File

@ -66,16 +66,15 @@ SELECT distinct(o_orderkey) FROM priority_orders join air_shipped_lineitems
WHERE (o_orderkey = 231);
-- left join support depends on flattening of the query
-- following query fails since the inner part is kept as subquery
SELECT * FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey);
SELECT o_orderkey, l_orderkey FROM priority_orders left join air_shipped_lineitems ON (o_orderkey = l_orderkey) ORDER BY o_orderkey LIMIT 1;
-- however, this works
SELECT count(*) FROM priority_orders left join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
-- view at the inner side of is not supported
-- view on the inner side is supported
SELECT count(*) FROM priority_orders right join lineitem_hash_part ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
-- but view at the outer side is. This is essentially the same as a left join with arguments reversed.
-- view on the outer side is supported
SELECT count(*) FROM lineitem_hash_part right join priority_orders ON (o_orderkey = l_orderkey) WHERE l_shipmode ='AIR';
-- left join on router query is supported

View File

@ -650,8 +650,6 @@ SELECT true AS valid FROM explain_json_2($$
$$);
-- a combination of subqueries in FROM and WHERE clauses
-- we actually recursively plan non colocated subqueries
-- pretty accurate, however, we hit our join checks, which seems too restrictive
SELECT true AS valid FROM explain_json_2($$
SELECT