mirror of https://github.com/citusdata/citus.git
* add tests for unsupported outer queries,
* refactor and commentsouter-join-noncolocated-dist-tables
parent
e06bed1a7f
commit
6cc72a584e
|
@ -382,7 +382,7 @@ FixedJoinOrderList(List *tableEntryList, JoinInfoContext *joinInfoContext)
|
|||
|
||||
/* add first table into joinedtable list */
|
||||
TableEntry *firstTable = TableEntryByRangeTableId(tableEntryList,
|
||||
firstJoinInfo->ltableIdx);
|
||||
firstJoinInfo->leftTableIdx);
|
||||
joinedTableList = lappend(joinedTableList, firstTable);
|
||||
|
||||
/* create join node for the first table */
|
||||
|
@ -402,7 +402,7 @@ FixedJoinOrderList(List *tableEntryList, JoinInfoContext *joinInfoContext)
|
|||
foreach_ptr(joinInfo, joinInfoContext->joinInfoList)
|
||||
{
|
||||
TableEntry *nextTable = TableEntryByRangeTableId(tableEntryList,
|
||||
joinInfo->rtableIdx);
|
||||
joinInfo->rightTableIdx);
|
||||
|
||||
bool passJoinClauseDirectly = true;
|
||||
nextJoinNode = EvaluateJoinRules(joinedTableList,
|
||||
|
@ -872,6 +872,17 @@ TableEntryListDifference(List *lhsTableList, List *rhsTableList)
|
|||
* next table, evaluates different join rules between the two tables, and finds
|
||||
* the best join rule that applies. The function returns the applicable join
|
||||
* order node which includes the join rule and the partition information.
|
||||
*
|
||||
* When we have only inner joins, we can commute the joins as we wish and it also
|
||||
* does not matter if we merge or move join and where clauses. For query trees with
|
||||
* only inner joins, `joinClauseList` contains join and where clauses combined so that
|
||||
* we can push down some where clauses which are applicable as join clause, which is
|
||||
* determined by `ApplicableJoinClauses`.
|
||||
* When we have at least 1 outer join in a query tree, we cannot commute joins(that is
|
||||
* why we have `FixedJoinOrderList`) or move join and where clauses as we wish because
|
||||
* we would have incorrect results. We should pass join and where clauses separately while
|
||||
* creating tasks. `joinClauseList` contains only join clauses when `passJoinClauseDirectly`
|
||||
* is set true.
|
||||
*/
|
||||
static JoinOrderNode *
|
||||
EvaluateJoinRules(List *joinedTableList, JoinOrderNode *currentJoinNode,
|
||||
|
@ -882,15 +893,19 @@ EvaluateJoinRules(List *joinedTableList, JoinOrderNode *currentJoinNode,
|
|||
uint32 lowestValidIndex = JOIN_RULE_INVALID_FIRST + 1;
|
||||
uint32 highestValidIndex = JOIN_RULE_LAST - 1;
|
||||
|
||||
/*
|
||||
* We first find all applicable join clauses between already joined tables
|
||||
* and the candidate table.
|
||||
*/
|
||||
List *joinedTableIdList = RangeTableIdList(joinedTableList);
|
||||
uint32 candidateTableId = candidateTable->rangeTableId;
|
||||
List *applicableJoinClauses = ApplicableJoinClauses(joinedTableIdList,
|
||||
candidateTableId,
|
||||
joinClauseList);
|
||||
List *joinClauses = joinClauseList;
|
||||
if (!passJoinClauseDirectly)
|
||||
{
|
||||
/*
|
||||
* We first find all applicable join clauses between already joined tables
|
||||
* and the candidate table.
|
||||
*/
|
||||
List *joinedTableIdList = RangeTableIdList(joinedTableList);
|
||||
uint32 candidateTableId = candidateTable->rangeTableId;
|
||||
joinClauses = ApplicableJoinClauses(joinedTableIdList,
|
||||
candidateTableId,
|
||||
joinClauseList);
|
||||
}
|
||||
|
||||
/* we then evaluate all join rules in order */
|
||||
for (uint32 ruleIndex = lowestValidIndex; ruleIndex <= highestValidIndex; ruleIndex++)
|
||||
|
@ -900,8 +915,7 @@ EvaluateJoinRules(List *joinedTableList, JoinOrderNode *currentJoinNode,
|
|||
|
||||
nextJoinNode = (*ruleEvalFunction)(currentJoinNode,
|
||||
candidateTable,
|
||||
(passJoinClauseDirectly) ? joinClauseList :
|
||||
applicableJoinClauses,
|
||||
joinClauses,
|
||||
joinType);
|
||||
|
||||
/* break after finding the first join rule that applies */
|
||||
|
@ -918,8 +932,7 @@ EvaluateJoinRules(List *joinedTableList, JoinOrderNode *currentJoinNode,
|
|||
|
||||
Assert(nextJoinNode != NULL);
|
||||
nextJoinNode->joinType = joinType;
|
||||
nextJoinNode->joinClauseList = (passJoinClauseDirectly) ? joinClauseList :
|
||||
applicableJoinClauses;
|
||||
nextJoinNode->joinClauseList = joinClauses;
|
||||
return nextJoinNode;
|
||||
}
|
||||
|
||||
|
|
|
@ -85,8 +85,8 @@ static bool ExtractFromExpressionWalker(Node *node,
|
|||
QualifierWalkerContext *walkerContext);
|
||||
static List * MultiTableNodeList(List *tableEntryList, List *rangeTableList);
|
||||
static List * AddMultiCollectNodes(List *tableNodeList);
|
||||
static MultiNode * MultiJoinTree(List *joinOrderList, List *collectTableList, bool
|
||||
passJoinClauseDirectly);
|
||||
static MultiNode * MultiJoinTree(List *joinOrderList, List *collectTableList,
|
||||
bool passJoinClauseDirectly);
|
||||
static MultiCollect * CollectNodeForTable(List *collectTableList, uint32 rangeTableId);
|
||||
static MultiSelect * MultiSelectNode(List *whereClauseList, bool passWhereClauseDirectly);
|
||||
static bool IsSelectClause(Node *clause);
|
||||
|
@ -587,14 +587,17 @@ MultiNodeTree(Query *queryTree)
|
|||
RaiseDeferredError(unsupportedQueryError, ERROR);
|
||||
}
|
||||
|
||||
/* extract where clause qualifiers and verify we can plan for them */
|
||||
List *whereClauseList = WhereClauseList(queryTree->jointree);
|
||||
unsupportedQueryError = DeferErrorIfUnsupportedClause(whereClauseList);
|
||||
/* extract where and join clause qualifiers(including outer join quals) and verify we can plan for them. */
|
||||
List *qualClauseList = QualifierList(queryTree->jointree);
|
||||
unsupportedQueryError = DeferErrorIfUnsupportedClause(qualClauseList);
|
||||
if (unsupportedQueryError)
|
||||
{
|
||||
RaiseDeferredErrorInternal(unsupportedQueryError, ERROR);
|
||||
}
|
||||
|
||||
/* WhereClauseList() merges join qualifiers and base qualifiers into result list */
|
||||
List *whereClauseList = WhereClauseList(queryTree->jointree);
|
||||
|
||||
/*
|
||||
* If we have a subquery, build a multi table node for the subquery and
|
||||
* add a collect node on top of the multi table node.
|
||||
|
@ -665,22 +668,36 @@ MultiNodeTree(Query *queryTree)
|
|||
/* add collect nodes on top of the multi table nodes */
|
||||
collectTableList = AddMultiCollectNodes(tableNodeList);
|
||||
|
||||
/*
|
||||
* We have 2 different join order methods.
|
||||
*
|
||||
* JoinOrderList:
|
||||
* When we have only inner joins, we can commute the joins as we wish and it also
|
||||
* does not matter if we merge or move join and where clauses. We can push down some
|
||||
* where clauses which are applicable as join clause.
|
||||
*
|
||||
* FixedJoinOrderList:
|
||||
* When we have at least 1 outer join in a query tree, we cannot commute joins or move join
|
||||
* and where clauses as we wish because we would have incorrect results. We should pass join
|
||||
* and where clauses separately as they are while creating tasks.
|
||||
*/
|
||||
if (FindNodeMatchingCheckFunction((Node *) queryTree->jointree, IsOuterJoinExpr))
|
||||
{
|
||||
/* pass join clauses directly into fix join order */
|
||||
/* extract join infos for left recursive join tree */
|
||||
JoinInfoContext *joinInfoContext = FetchJoinOrderContext(queryTree->jointree);
|
||||
|
||||
/* where clause should not contain join clause */
|
||||
/* where clause should only contain base qualifiers */
|
||||
whereClauseList = joinInfoContext->baseQualifierList;
|
||||
|
||||
/* we simply donot commute joins as we have at least 1 outer join */
|
||||
joinOrderList = FixedJoinOrderList(tableEntryList, joinInfoContext);
|
||||
|
||||
/* pass join clauses directly as they are while creating tasks */
|
||||
passQualClauseDirectly = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
/* only consider base qualifications */
|
||||
/* consider also base qualifications */
|
||||
joinClauseList = JoinClauseList(whereClauseList);
|
||||
|
||||
/* find best join order for commutative inner joins */
|
||||
|
@ -740,7 +757,7 @@ FetchJoinOrderContext(FromExpr *fromExpr)
|
|||
ExtractLeftMostRangeTableIndex((Node *) fromExpr, &leftMostTableIdx);
|
||||
Assert(list_length(joinInfoContext->joinInfoList) > 0);
|
||||
JoinInfo *leftMostJoinInfo = list_nth(joinInfoContext->joinInfoList, 0);
|
||||
leftMostJoinInfo->ltableIdx = leftMostTableIdx;
|
||||
leftMostJoinInfo->leftTableIdx = leftMostTableIdx;
|
||||
|
||||
return joinInfoContext;
|
||||
}
|
||||
|
@ -771,26 +788,23 @@ JoinInfoWalker(Node *node, JoinInfoContext *joinInfoContext)
|
|||
if (IsA(node, JoinExpr))
|
||||
{
|
||||
JoinExpr *joinExpression = (JoinExpr *) node;
|
||||
if (!(IsA(joinExpression->rarg, RangeTblRef) || IsA(joinExpression->rarg,
|
||||
FromExpr)))
|
||||
if (!IsA(joinExpression->rarg, RangeTblRef))
|
||||
{
|
||||
ereport(WARNING, (errmsg("unexpected node in joininfowalker")));
|
||||
/*
|
||||
* occurs when we have subquery which is not recursively plan. Here is only
|
||||
* expected when we have lateral outer join. ??? (subqueries should have been
|
||||
* already planned by recursive planner)
|
||||
*/
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg(
|
||||
"complex joins are only supported when all distributed "
|
||||
"tables are joined on their distribution columns with "
|
||||
"equal operator")));
|
||||
}
|
||||
|
||||
Node *joinQualifiersNode = joinExpression->quals;
|
||||
JoinType joinType = joinExpression->jointype;
|
||||
RangeTblRef *rightTableRef = NULL;
|
||||
if (IsA(joinExpression->rarg, RangeTblRef))
|
||||
{
|
||||
rightTableRef = (RangeTblRef *) joinExpression->rarg;
|
||||
}
|
||||
else
|
||||
{
|
||||
Assert(IsA(joinExpression->rarg, FromExpr));
|
||||
FromExpr *fromExpr = (FromExpr *) joinExpression->rarg;
|
||||
Assert(list_length(fromExpr->fromlist) == 1);
|
||||
rightTableRef = (RangeTblRef *) list_nth(fromExpr->fromlist, 0);
|
||||
}
|
||||
RangeTblRef *rightTableRef = (RangeTblRef *) joinExpression->rarg;
|
||||
|
||||
List *joinQualifierList = NIL;
|
||||
if (joinQualifiersNode != NULL)
|
||||
|
@ -810,7 +824,7 @@ JoinInfoWalker(Node *node, JoinInfoContext *joinInfoContext)
|
|||
|
||||
JoinInfo *joinInfo = palloc0(sizeof(JoinInfo));
|
||||
joinInfo->joinType = joinType;
|
||||
joinInfo->rtableIdx = rightTableRef->rtindex;
|
||||
joinInfo->rightTableIdx = rightTableRef->rtindex;
|
||||
joinInfo->joinQualifierList = joinQualifierList;
|
||||
|
||||
joinInfoContext->joinInfoList = lappend(joinInfoContext->joinInfoList, joinInfo);
|
||||
|
@ -1778,6 +1792,12 @@ CollectNodeForTable(List *collectTableList, uint32 rangeTableId)
|
|||
* MultiSelectNode extracts the select clauses from the given where clause list,
|
||||
* and builds a MultiSelect node from these clauses. If the expression tree does
|
||||
* not have any select clauses, the function return null.
|
||||
*
|
||||
* When we have at least 1 outer join in a query tree, we cannot commute joins(that is
|
||||
* why we have `FixedJoinOrderList`) or move join and where clauses as we wish because
|
||||
* we would have incorrect results. We should pass join and where clauses separately as
|
||||
* they are while creating tasks. `whereClauseList` should be passed as it is when
|
||||
* `passWhereClauseDirectly` is set true.
|
||||
*/
|
||||
static MultiSelect *
|
||||
MultiSelectNode(List *whereClauseList, bool passWhereClauseDirectly)
|
||||
|
@ -2090,16 +2110,19 @@ ApplyJoinRule(MultiNode *leftNode, MultiNode *rightNode, JoinRuleType ruleType,
|
|||
rightTableIdCount = list_length(rightTableIdList);
|
||||
Assert(rightTableIdCount == 1);
|
||||
|
||||
/* find applicable join clauses between the left and right data sources */
|
||||
uint32 rightTableId = (uint32) linitial_int(rightTableIdList);
|
||||
List *applicableJoinClauses = ApplicableJoinClauses(leftTableIdList, rightTableId,
|
||||
joinClauseList);
|
||||
List *joinClauses = joinClauseList;
|
||||
if (!passJoinClauseDirectly)
|
||||
{
|
||||
/* find applicable join clauses between the left and right data sources */
|
||||
uint32 rightTableId = (uint32) linitial_int(rightTableIdList);
|
||||
joinClauses = ApplicableJoinClauses(leftTableIdList, rightTableId,
|
||||
joinClauseList);
|
||||
}
|
||||
|
||||
/* call the join rule application function to create the new join node */
|
||||
RuleApplyFunction ruleApplyFunction = JoinRuleApplyFunction(ruleType);
|
||||
MultiNode *multiNode = (*ruleApplyFunction)(leftNode, rightNode, partitionColumnList,
|
||||
joinType, (passJoinClauseDirectly) ?
|
||||
joinClauseList : applicableJoinClauses);
|
||||
joinType, joinClauses);
|
||||
|
||||
if (joinType != JOIN_INNER && CitusIsA(multiNode, MultiJoin))
|
||||
{
|
||||
|
|
|
@ -115,7 +115,6 @@ static bool HasRightRecursiveJoin(FromExpr *fromExpr);
|
|||
static bool RightRecursiveJoinExprWalker(Node *node, void *context);
|
||||
static bool HasCartesianJoin(FromExpr *fromExpr);
|
||||
static bool CartesianJoinExprWalker(Node *node, void *context);
|
||||
static bool HasLateralJoin(JoinRestrictionContext *joinRestrictionContext);
|
||||
|
||||
|
||||
/*
|
||||
|
@ -192,7 +191,7 @@ ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery,
|
|||
*/
|
||||
if (FindNodeMatchingCheckFunction((Node *) rewrittenQuery->jointree, IsOuterJoinExpr))
|
||||
{
|
||||
/* we can pushdown outer joins if all restrictions are on partition columns */
|
||||
/* we can try to pushdown outer joins if all restrictions are on partition columns */
|
||||
if (RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext))
|
||||
{
|
||||
return true;
|
||||
|
@ -217,12 +216,8 @@ ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery,
|
|||
}
|
||||
|
||||
/*
|
||||
* join order planner cannot handle lateral join trees for outer joins.
|
||||
* todo: join order planner cannot handle lateral join trees for outer joins.
|
||||
*/
|
||||
if (HasLateralJoin(plannerRestrictionContext->joinRestrictionContext))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -340,25 +335,6 @@ CartesianJoinExprWalker(Node *node, void *context)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* HasLateralJoin returns true if join restriction context contain lateral join.
|
||||
*/
|
||||
static bool
|
||||
HasLateralJoin(JoinRestrictionContext *joinRestrictionContext)
|
||||
{
|
||||
JoinRestriction *joinRestriction = NULL;
|
||||
foreach_ptr(joinRestriction, joinRestrictionContext->joinRestrictionList)
|
||||
{
|
||||
if (joinRestriction->plannerInfo && joinRestriction->plannerInfo->hasLateralRTEs)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JoinTreeContainsSubquery returns true if the input query contains any subqueries
|
||||
* in the join tree (e.g., FROM clause).
|
||||
|
|
|
@ -83,7 +83,7 @@ typedef struct JoinOrderNode
|
|||
} JoinOrderNode;
|
||||
|
||||
|
||||
/* JoinInfoContext stores joinInfo list and base qualifications */
|
||||
/* JoinInfoContext stores list of JoinInfo and base qualifications */
|
||||
typedef struct JoinInfoContext
|
||||
{
|
||||
List *baseQualifierList;
|
||||
|
@ -91,12 +91,18 @@ typedef struct JoinInfoContext
|
|||
} JoinInfoContext;
|
||||
|
||||
|
||||
/* JoinInfoContext stores joinInfo list and base qualifications */
|
||||
/*
|
||||
* JoinInfo stores information about a join between 2 tables.
|
||||
* joinType: join type between left and right tables in join
|
||||
* leftTableIdx: rtable index for left table in join
|
||||
* rightTableIdx: rtable index for right table in join
|
||||
* joinQualifierList: list of join qualifications in join, i.e. ON (...)
|
||||
*/
|
||||
typedef struct JoinInfo
|
||||
{
|
||||
JoinType joinType;
|
||||
uint32 ltableIdx;
|
||||
uint32 rtableIdx;
|
||||
uint32 leftTableIdx;
|
||||
uint32 rightTableIdx;
|
||||
List *joinQualifierList;
|
||||
} JoinInfo;
|
||||
|
||||
|
|
|
@ -161,14 +161,22 @@ SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 RIGHT J
|
|||
|
||||
-- a reference tables CROSS JOINed with a distribted table, and later JOINED with distributed tables on reference table column
|
||||
-- so not safe to pushdown
|
||||
SET citus.enable_repartition_joins TO on;
|
||||
SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 LEFT JOIN users_table u2 ON (ref2.id = u2.user_id);
|
||||
ERROR: the query contains a join that requires repartitioning
|
||||
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10201
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 FULL JOIN users_table u2 ON (ref2.id = u2.user_id);
|
||||
ERROR: the query contains a join that requires repartitioning
|
||||
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10201
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 RIGHT JOIN users_table u2 ON (ref2.id = u2.user_id);
|
||||
ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
|
||||
RESET citus.enable_repartition_joins;
|
||||
-- via repartitioning, Citus can handle this query as the result of "u1 CROSS JOIN ref2"
|
||||
-- can be repartitioned on ref2.id
|
||||
Set citus.enable_repartition_joins to on;
|
||||
|
|
|
@ -354,8 +354,7 @@ FROM (
|
|||
GROUP BY user_id
|
||||
) AS shard_union
|
||||
ORDER BY user_lastseen DESC;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Cartesian products are currently unsupported
|
||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
-- not pushable since lateral join is not on the partition key
|
||||
INSERT INTO agg_results_third (user_id, agg_time, value_2_agg)
|
||||
SELECT
|
||||
|
@ -383,8 +382,7 @@ FROM (
|
|||
GROUP BY user_id
|
||||
) AS shard_union
|
||||
ORDER BY user_lastseen DESC;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Cartesian products are currently unsupported
|
||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
-- not pushable since lateral join is not on the partition key
|
||||
INSERT INTO agg_results_third (user_id, agg_time, value_2_agg)
|
||||
SELECT
|
||||
|
@ -412,8 +410,7 @@ FROM (
|
|||
GROUP BY user_id
|
||||
) AS shard_union
|
||||
ORDER BY user_lastseen DESC;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Cartesian products are currently unsupported
|
||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
---------------------------------------------------------------------
|
||||
---------------------------------------------------------------------
|
||||
-- Count the number of distinct users_table who are in segment X and Y and Z
|
||||
|
|
|
@ -273,13 +273,18 @@ FROM
|
|||
(1 row)
|
||||
|
||||
-- Since we cannot broadcast or re-partition, joining on a different key should error out
|
||||
SET citus.enable_repartition_joins TO on;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_nationkey = r_nationkey);
|
||||
LOG: join order: [ "multi_outer_join_left" ][ dual partition join(LEFT) "multi_outer_join_right" ]
|
||||
ERROR: the query contains a join that requires repartitioning
|
||||
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
32
|
||||
(1 row)
|
||||
|
||||
RESET citus.enable_repartition_joins;
|
||||
-- Anti-join should return customers for which there is no row in the right table
|
||||
SELECT
|
||||
min(l_custkey), max(l_custkey)
|
||||
|
|
|
@ -271,13 +271,18 @@ FROM
|
|||
(1 row)
|
||||
|
||||
-- Citus can use broadcast join here
|
||||
SET citus.enable_repartition_joins TO on;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_hash b ON (l_nationkey = r_nationkey);
|
||||
LOG: join order: [ "multi_outer_join_left_hash" ][ dual partition join(LEFT) "multi_outer_join_right_hash" ]
|
||||
ERROR: the query contains a join that requires repartitioning
|
||||
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
52
|
||||
(1 row)
|
||||
|
||||
RESET citus.enable_repartition_joins;
|
||||
-- Anti-join should return customers for which there is no row in the right table
|
||||
SELECT
|
||||
min(l_custkey), max(l_custkey)
|
||||
|
|
|
@ -389,7 +389,7 @@ SELECT t2.* FROM t2 WHERE EXISTS (SELECT * FROM t1 WHERE t1.col1 = t2.col1) ORDE
|
|||
ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
|
||||
SELECT t2.* FROM t2 WHERE EXISTS (SELECT * FROM t1 WHERE t1.col2 = t2.col2) ORDER BY 1,2;
|
||||
ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
|
||||
-- join order planner cannot handle anti join
|
||||
-- join order planner cannot handle anti joins
|
||||
SELECT t1.* FROM t1 WHERE NOT EXISTS (SELECT * FROM t2 WHERE t1.col1 = t2.col1) ORDER BY 1,2;
|
||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
SELECT t1.* FROM t1 WHERE NOT EXISTS (SELECT * FROM t2 WHERE t1.col2 = t2.col2) ORDER BY 1,2;
|
||||
|
@ -398,6 +398,17 @@ SELECT t2.* FROM t2 WHERE NOT EXISTS (SELECT * FROM t1 WHERE t1.col1 = t2.col1)
|
|||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
SELECT t2.* FROM t2 WHERE NOT EXISTS (SELECT * FROM t1 WHERE t1.col2 = t2.col2) ORDER BY 1,2;
|
||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
-- join order planner cannot handle lateral outer joins
|
||||
SELECT t1.*, tt2.* FROM t1 LEFT JOIN LATERAL (SELECT * FROM t2 WHERE t1.col1 = t2.col1) tt2 ON (t1.col1 = tt2.col1) ORDER BY 1,2,3,4;
|
||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||
-- join order planner cannot handle cartesian joins
|
||||
SELECT tt1.*, t3.* FROM (SELECT t1.* FROM t1,t2) tt1 LEFT JOIN t3 ON (tt1.col1 = t3.col1) ORDER BY 1,2,3,4;
|
||||
LOG: join order: [ "t1" ][ cartesian product(INNER) "t2" ]
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Cartesian products are currently unsupported
|
||||
-- join order planner cannot handle right recursive joins
|
||||
SELECT t1.*, t2.* FROM t1 LEFT JOIN ( t2 JOIN t3 ON t2.col2 = t3.col1) ON (t1.col1 = t2.col1) ORDER BY 1,2,3,4;
|
||||
ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns
|
||||
DROP SCHEMA non_colocated_outer_joins CASCADE;
|
||||
NOTICE: drop cascades to 3 other objects
|
||||
DETAIL: drop cascades to table t1
|
||||
|
|
|
@ -51,9 +51,11 @@ SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 RIGHT J
|
|||
|
||||
-- a reference tables CROSS JOINed with a distribted table, and later JOINED with distributed tables on reference table column
|
||||
-- so not safe to pushdown
|
||||
SET citus.enable_repartition_joins TO on;
|
||||
SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 LEFT JOIN users_table u2 ON (ref2.id = u2.user_id);
|
||||
SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 FULL JOIN users_table u2 ON (ref2.id = u2.user_id);
|
||||
SELECT count(*) FROM users_table u1 CROSS JOIN users_ref_test_table ref2 RIGHT JOIN users_table u2 ON (ref2.id = u2.user_id);
|
||||
RESET citus.enable_repartition_joins;
|
||||
|
||||
-- via repartitioning, Citus can handle this query as the result of "u1 CROSS JOIN ref2"
|
||||
-- can be repartitioned on ref2.id
|
||||
|
|
|
@ -213,11 +213,12 @@ FROM
|
|||
|
||||
|
||||
-- Since we cannot broadcast or re-partition, joining on a different key should error out
|
||||
SET citus.enable_repartition_joins TO on;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_nationkey = r_nationkey);
|
||||
|
||||
RESET citus.enable_repartition_joins;
|
||||
|
||||
-- Anti-join should return customers for which there is no row in the right table
|
||||
SELECT
|
||||
|
|
|
@ -217,11 +217,12 @@ FROM
|
|||
|
||||
|
||||
-- Citus can use broadcast join here
|
||||
SET citus.enable_repartition_joins TO on;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_hash b ON (l_nationkey = r_nationkey);
|
||||
|
||||
RESET citus.enable_repartition_joins;
|
||||
|
||||
-- Anti-join should return customers for which there is no row in the right table
|
||||
SELECT
|
||||
|
|
|
@ -83,13 +83,25 @@ SELECT t1.* FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t1.col2 = t2.col2) ORDE
|
|||
SELECT t2.* FROM t2 WHERE EXISTS (SELECT * FROM t1 WHERE t1.col1 = t2.col1) ORDER BY 1,2;
|
||||
SELECT t2.* FROM t2 WHERE EXISTS (SELECT * FROM t1 WHERE t1.col2 = t2.col2) ORDER BY 1,2;
|
||||
|
||||
-- join order planner cannot handle anti join
|
||||
-- join order planner cannot handle anti joins
|
||||
|
||||
SELECT t1.* FROM t1 WHERE NOT EXISTS (SELECT * FROM t2 WHERE t1.col1 = t2.col1) ORDER BY 1,2;
|
||||
SELECT t1.* FROM t1 WHERE NOT EXISTS (SELECT * FROM t2 WHERE t1.col2 = t2.col2) ORDER BY 1,2;
|
||||
SELECT t2.* FROM t2 WHERE NOT EXISTS (SELECT * FROM t1 WHERE t1.col1 = t2.col1) ORDER BY 1,2;
|
||||
SELECT t2.* FROM t2 WHERE NOT EXISTS (SELECT * FROM t1 WHERE t1.col2 = t2.col2) ORDER BY 1,2;
|
||||
|
||||
-- join order planner cannot handle lateral outer joins
|
||||
|
||||
SELECT t1.*, tt2.* FROM t1 LEFT JOIN LATERAL (SELECT * FROM t2 WHERE t1.col1 = t2.col1) tt2 ON (t1.col1 = tt2.col1) ORDER BY 1,2,3,4;
|
||||
|
||||
-- join order planner cannot handle cartesian joins
|
||||
|
||||
SELECT tt1.*, t3.* FROM (SELECT t1.* FROM t1,t2) tt1 LEFT JOIN t3 ON (tt1.col1 = t3.col1) ORDER BY 1,2,3,4;
|
||||
|
||||
-- join order planner cannot handle right recursive joins
|
||||
|
||||
SELECT t1.*, t2.* FROM t1 LEFT JOIN ( t2 JOIN t3 ON t2.col2 = t3.col1) ON (t1.col1 = t2.col1) ORDER BY 1,2,3,4;
|
||||
|
||||
DROP SCHEMA non_colocated_outer_joins CASCADE;
|
||||
RESET client_min_messages;
|
||||
RESET citus.log_multi_join_order;
|
||||
|
|
Loading…
Reference in New Issue