outer-join-noncolocated-dist-tables
aykutbozkurt 2023-02-01 18:59:22 +03:00
parent d569c1a5b4
commit 2ac2399005
9 changed files with 170 additions and 206 deletions

View File

@ -135,6 +135,8 @@ static RTEListProperties * GetRTEListProperties(List *rangeTableList);
static List * TranslatedVars(PlannerInfo *root, int relationIndex);
static void WarnIfListHasForeignDistributedTable(List *rangeTableList);
static void ErrorIfMergeHasUnsupportedTables(Query *parse, List *rangeTableList);
static void AddFastPathRestrictInfoIntoPlannerContext(
PlannerRestrictionContext *plannerRestrictionContext);
static List * GenerateImplicitJoinRestrictInfoList(PlannerInfo *plannerInfo,
RelOptInfo *innerrel,
RelOptInfo *outerrel);
@ -724,7 +726,7 @@ PlanFastPathDistributedStmt(DistributedPlanningContext *planContext,
planContext->boundParams);
/* generate simple base restrict info inside plannerRestrictionContext */
RelabelPlannerRestrictionContext(planContext->plannerRestrictionContext);
AddFastPathRestrictInfoIntoPlannerContext(planContext->plannerRestrictionContext);
return CreateDistributedPlannedStmt(planContext);
}
@ -973,7 +975,7 @@ TryCreateDistributedPlannedStmt(PlannedStmt *localPlan,
/*
* ReplanAfterQueryModification replans given query to update plannercontext
* accordingly. Returns modified query without chnaging given query.
* accordingly. Returns modified query without changing original query.
*/
Query *
ReplanAfterQueryModification(Query *originalQuery, ParamListInfo boundParams)
@ -1851,19 +1853,60 @@ CheckNodeCopyAndSerialization(Node *node)
}
/*
* AddFastPathRestrictInfoIntoPlannerContext creates a single base restrictinfo as we do not call
* standard_planner and not generate restrictinfo for fastpath queries.
*/
static
void
AddFastPathRestrictInfoIntoPlannerContext(
PlannerRestrictionContext *plannerRestrictionContext)
{
if (plannerRestrictionContext->fastPathRestrictionContext == NULL ||
plannerRestrictionContext->fastPathRestrictionContext->distributionKeyValue ==
NULL)
{
return;
}
Const *distKeyVal =
plannerRestrictionContext->fastPathRestrictionContext->distributionKeyValue;
Var *partitionColumn = PartitionColumn(
plannerRestrictionContext->fastPathRestrictionContext->distRelId, 1);
OpExpr *partitionExpression = MakeOpExpression(partitionColumn,
BTEqualStrategyNumber);
Node *rightOp = get_rightop((Expr *) partitionExpression);
Const *rightConst = (Const *) rightOp;
*rightConst = *distKeyVal;
RestrictInfo *fastpathRestrictInfo = makeNode(RestrictInfo);
fastpathRestrictInfo->can_join = false;
fastpathRestrictInfo->is_pushed_down = true;
fastpathRestrictInfo->clause = (Expr *) partitionExpression;
RelationRestriction *relationRestriction = palloc0(sizeof(RelationRestriction));
relationRestriction->relOptInfo = palloc0(sizeof(RelOptInfo));
relationRestriction->relOptInfo->baserestrictinfo = list_make1(
fastpathRestrictInfo);
plannerRestrictionContext->relationRestrictionContext->relationRestrictionList =
list_make1(relationRestriction);
}
/*
* GenerateImplicitJoinRestrictInfoList generates implicit join restrict infos
* by using planner information. As we rely on join restriction infos at join
* planner, we generate implicit join restrict infos. When join condition contains
* a reducible constant, generate_join_implied_equalities does not generate implicit
* join clauses. Hence, we mark ec_has_const to false beforehand. At the end, we
* restore previous values.
* planner, we generate implicit join restrict infos.
*/
static List *
GenerateImplicitJoinRestrictInfoList(PlannerInfo *plannerInfo,
RelOptInfo *innerrel, RelOptInfo *outerrel)
{
Relids joinrelids = bms_union(innerrel->relids, outerrel->relids);
/*
* when join condition contains a reducible constant, generate_join_implied_equalities
* does not generate implicit join clauses. Hence, we mark ec_has_const to false
* beforehand. At the end, we restore previous values.
*/
List *prevVals = NIL;
EquivalenceClass *eqclass = NULL;
foreach_ptr(eqclass, plannerInfo->eq_classes)
@ -1872,12 +1915,15 @@ GenerateImplicitJoinRestrictInfoList(PlannerInfo *plannerInfo,
eqclass->ec_has_const = false;
}
/* generate implicit join restrictinfos */
Relids joinrelids = bms_union(innerrel->relids, outerrel->relids);
List *generatedRestrictInfoList = generate_join_implied_equalities(
plannerInfo,
joinrelids,
outerrel->relids,
innerrel);
/* restore previous values for ec_has_const */
int i;
for (i = 0; i < list_length(prevVals); i++)
{

View File

@ -419,7 +419,9 @@ DistKeyInSimpleOpExpression(Expr *clause, Var *distColumn, Node **distributionKe
}
else
{
/* if the vartypes do not match, let shard pruning handle it later */
/*
* if the vartypes do not match, try to transform the constant
*/
bool missingOk = true;
Const *transformedConstantClause =
TransformPartitionRestrictionValue(distColumn, constantClause, missingOk);

View File

@ -89,6 +89,8 @@ static DistributedPlan * CreateNonPushableInsertSelectPlan(uint64 planId, Query
static DeferredErrorMessage * NonPushableInsertSelectSupported(Query *insertSelectQuery);
static Query * WrapSubquery(Query *subquery);
static void RelabelTargetEntryList(List *selectTargetList, List *insertTargetList);
static void RelabelPlannerRestrictionContextForInsertSelect(
PlannerRestrictionContext *plannerRestrictionContext);
static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
Oid targetRelationId);
static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation,
@ -364,43 +366,18 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
/*
* RelabelPlannerRestrictionContext relabels all Var varnos inside plannerRestrictionContext
* restriction infos to 1. If we have an unempty fastpath context, we manually create a single
* base RestrictInfo as we didnot call standard_planner to create it.
* RelabelPlannerRestrictionContextForInsertSelect relabels all Var varnos of restrictioninfos
* inside plannerRestrictionContext to 1. We wrap SELECT part into subquery for INSERT .. SELECT
* queries but do not update restrictinfos inside plannerContext. It is safe to do that as we are
* sure that SELECT part has only single table.
*/
void
RelabelPlannerRestrictionContext(PlannerRestrictionContext *plannerRestrictionContext)
static void
RelabelPlannerRestrictionContextForInsertSelect(
PlannerRestrictionContext *plannerRestrictionContext)
{
List *relationRestrictionList =
plannerRestrictionContext->relationRestrictionContext->relationRestrictionList;
if (plannerRestrictionContext->fastPathRestrictionContext &&
plannerRestrictionContext->fastPathRestrictionContext->distributionKeyValue)
{
Const *distKeyVal =
plannerRestrictionContext->fastPathRestrictionContext->distributionKeyValue;
Var *partitionColumn = PartitionColumn(
plannerRestrictionContext->fastPathRestrictionContext->distRelId, 1);
OpExpr *partitionExpression = MakeOpExpression(partitionColumn,
BTEqualStrategyNumber);
Node *rightOp = get_rightop((Expr *) partitionExpression);
Const *rightConst = (Const *) rightOp;
*rightConst = *distKeyVal;
RestrictInfo *fastpathRestrictInfo = makeNode(RestrictInfo);
fastpathRestrictInfo->can_join = false;
fastpathRestrictInfo->is_pushed_down = true;
fastpathRestrictInfo->clause = (Expr *) partitionExpression;
RelationRestriction *relationRestriction = palloc0(sizeof(RelationRestriction));
relationRestriction->relOptInfo = palloc0(sizeof(RelOptInfo));
relationRestriction->relOptInfo->baserestrictinfo = list_make1(
fastpathRestrictInfo);
plannerRestrictionContext->relationRestrictionContext->relationRestrictionList =
list_make1(relationRestriction);
return;
}
RelationRestriction *relationRestriction = NULL;
foreach_ptr(relationRestriction, relationRestrictionList)
{
@ -445,8 +422,8 @@ CreateInsertSelectIntoLocalTablePlan(uint64 planId, Query *insertSelectQuery,
/* get the SELECT query (may have changed after PrepareInsertSelectForCitusPlanner) */
Query *selectQuery = selectRte->subquery;
/* relabels all Var varnos inside plannerRestrictionContext after we modify query */
RelabelPlannerRestrictionContext(plannerRestrictionContext);
/* relabels all Var varnos inside plannerRestrictionContext after we modify SELECT query */
RelabelPlannerRestrictionContextForInsertSelect(plannerRestrictionContext);
bool allowRecursivePlanning = true;
DistributedPlan *distPlan = CreateDistributedPlan(planId, allowRecursivePlanning,

View File

@ -75,9 +75,12 @@ static List * FindApplicableJoinClausesForTables(List *joinRestrictInfoListList,
uint32 rhsTableId,
JoinType joinType);
static const char * JoinTypeName(JoinType jointype);
static List * ExtractPushdownJoinRestrictInfos(List *joinRestrictInfoList,
RestrictInfo *joinRestrictInfo,
JoinType joinType);
static List * ExtractPushdownableJoinRestrictInfos(List *joinRestrictInfoList,
RestrictInfo *joinRestrictInfo);
static ApplicableJoinClauseContext * CreateApplicableJoinClauseContext(
List *pushdownableJoinRestrictClauseList,
List *
nonPushdownableJoinRestrictClauseList);
/* Local functions forward declarations for join evaluations */
static JoinOrderNode * EvaluateJoinRules(List *joinedTableList,
@ -307,7 +310,7 @@ NodeIsEqualsOpExpr(Node *node)
*/
List *
JoinOrderList(List *tableEntryList, List *joinRestrictInfoListList,
List *generatedEcJoinClauseList, List *pseudoClauseList)
List *generatedEcJoinClauseList)
{
List *candidateJoinOrderList = NIL;
ListCell *tableEntryCell = NULL;
@ -368,13 +371,12 @@ TableEntryByRangeTableId(List *tableEntryList, uint32 rangeTableIdx)
/*
* ExtractPushdownJoinRestrictInfos extracts clauses, which are pushed down and treated like a normal filter,
* ExtractPushdownableJoinRestrictInfos extracts clauses, which are pushed down and treated like a normal filter,
* inside given join restriction infos.
*/
static List *
ExtractPushdownJoinRestrictInfos(List *restrictInfoListOfJoin,
RestrictInfo *joinRestrictInfo,
JoinType joinType)
ExtractPushdownableJoinRestrictInfos(List *restrictInfoListOfJoin,
RestrictInfo *joinRestrictInfo)
{
List *joinFilterRestrictInfoList = NIL;
@ -422,24 +424,24 @@ FindApplicableJoinClausesForTables(List *joinRestrictInfoListList,
if (joinRestrictInfo->can_join &&
IsApplicableJoinClause(lhsTableIdList, rhsTableId, restrictClause))
{
List *pushdownableJoinRestrictInfoList = ExtractPushdownJoinRestrictInfos(
joinRestrictInfoList, joinRestrictInfo, joinType);
List *pushdownableJoinRestrictInfoList =
ExtractPushdownableJoinRestrictInfos(joinRestrictInfoList,
joinRestrictInfo);
List *nonPushdownableJoinRestrictInfoList =
list_difference(joinRestrictInfoList,
pushdownableJoinRestrictInfoList);
List *pushdownableJoinRestrictClauseList =
get_all_actual_clauses(pushdownableJoinRestrictInfoList);
List *nonPushdownableJoinRestrictInfoList = list_difference(
joinRestrictInfoList,
pushdownableJoinRestrictInfoList);
List *nonPushdownableJoinRestrictClauseList =
get_all_actual_clauses(nonPushdownableJoinRestrictInfoList);
ApplicableJoinClauseContext *applicableJoinClauseContext = palloc0(
sizeof(ApplicableJoinClauseContext));
applicableJoinClauseContext->joinClauseList = get_all_actual_clauses(
joinRestrictInfoList);
applicableJoinClauseContext->pushdownableJoinClauseList =
pushdownableJoinRestrictClauseList;
applicableJoinClauseContext->nonPushdownableJoinClauseList =
nonPushdownableJoinRestrictClauseList;
ApplicableJoinClauseContext *applicableJoinClauseContext =
CreateApplicableJoinClauseContext(pushdownableJoinRestrictClauseList,
nonPushdownableJoinRestrictClauseList);
applicableJoinClauseContextList = lappend(applicableJoinClauseContextList,
applicableJoinClauseContext);
}
@ -447,33 +449,25 @@ FindApplicableJoinClausesForTables(List *joinRestrictInfoListList,
}
/* we can find applicable join clause inside generated implicit join clauses */
if (joinType == JOIN_INNER)
Node *ecClause = NULL;
foreach_ptr(ecClause, generatedEcJoinClauseList)
{
Node *ecClause = NULL;
foreach_ptr(ecClause, generatedEcJoinClauseList)
if (IsApplicableJoinClause(lhsTableIdList, rhsTableId, ecClause))
{
if (IsApplicableJoinClause(lhsTableIdList, rhsTableId, ecClause))
{
List *generatedJoinClauseList = list_make1(ecClause);
ApplicableJoinClauseContext *applicableJoinClauseContext = palloc0(
sizeof(ApplicableJoinClauseContext));
applicableJoinClauseContext->joinClauseList = generatedJoinClauseList;
applicableJoinClauseContext->pushdownableJoinClauseList = NIL;
applicableJoinClauseContext->nonPushdownableJoinClauseList =
generatedJoinClauseList;
applicableJoinClauseContextList = lappend(applicableJoinClauseContextList,
applicableJoinClauseContext);
}
List *generatedJoinClauseList = list_make1(ecClause);
ApplicableJoinClauseContext *applicableJoinClauseContext =
CreateApplicableJoinClauseContext(NIL, generatedJoinClauseList);
applicableJoinClauseContextList = lappend(applicableJoinClauseContextList,
applicableJoinClauseContext);
}
}
/* add an empty join clause list to be evaluated by cartesian rules */
List *emptyClauseList = NIL;
ApplicableJoinClauseContext *emptyApplicableJoinClauseContext = palloc0(
sizeof(ApplicableJoinClauseContext));
emptyApplicableJoinClauseContext->joinClauseList = emptyClauseList;
emptyApplicableJoinClauseContext->pushdownableJoinClauseList = emptyClauseList;
emptyApplicableJoinClauseContext->nonPushdownableJoinClauseList = emptyClauseList;
ApplicableJoinClauseContext *emptyApplicableJoinClauseContext =
CreateApplicableJoinClauseContext(emptyClauseList, emptyClauseList);
applicableJoinClauseContextList = lappend(applicableJoinClauseContextList,
emptyApplicableJoinClauseContext);
@ -481,14 +475,65 @@ FindApplicableJoinClausesForTables(List *joinRestrictInfoListList,
}
static ApplicableJoinClauseContext *
CreateApplicableJoinClauseContext(List *pushdownableJoinRestrictClauseList,
List *nonPushdownableJoinRestrictClauseList)
{
ApplicableJoinClauseContext *applicableJoinClauseContext = palloc0(
sizeof(ApplicableJoinClauseContext));
applicableJoinClauseContext->
pushdownableJoinClauseList = pushdownableJoinRestrictClauseList;
applicableJoinClauseContext->
nonPushdownableJoinClauseList = nonPushdownableJoinRestrictClauseList;
return applicableJoinClauseContext;
}
/*
* ExtractApplicableJoinClauseContextFromJoinList returns ApplicableJoinClauseContext which contains
* all pushdownable and nonpushdownable clauses from given joinOrderList.
*/
ApplicableJoinClauseContext *
ExtractApplicableJoinClauseContextFromJoinList(List *joinOrderList)
{
List *pushdownableJoinClauseList = NIL;
List *nonPushdownableJoinClauseList = NIL;
JoinOrderNode *joinOrderNode = NULL;
foreach_ptr(joinOrderNode, joinOrderList)
{
ApplicableJoinClauseContext *nodeApplicableJoinClauseContext =
joinOrderNode->applicableJoinClauseContext;
/* first node does not contain ApplicableJoinClauseContext */
if (nodeApplicableJoinClauseContext == NULL)
{
continue;
}
pushdownableJoinClauseList = list_concat_unique(pushdownableJoinClauseList,
nodeApplicableJoinClauseContext->
pushdownableJoinClauseList);
nonPushdownableJoinClauseList = list_concat_unique(nonPushdownableJoinClauseList,
nodeApplicableJoinClauseContext
->nonPushdownableJoinClauseList);
}
ApplicableJoinClauseContext *applicableJoinClauseContext =
CreateApplicableJoinClauseContext(pushdownableJoinClauseList,
nonPushdownableJoinClauseList);
return applicableJoinClauseContext;
}
/*
* FixedJoinOrderList returns the best fixed join order according to
* applicable join rules for the nodes in the list.
*/
List *
FixedJoinOrderList(List *tableEntryList, JoinInfoContext *joinInfoContext,
List *joinRestrictInfoListList, List *generatedEcJoinClauseList,
List *pseudoClauseList)
List *joinRestrictInfoListList, List *generatedEcJoinClauseList)
{
/* we donot support anti joins as ruleutils files cannot deparse JOIN_ANTI */
if (JoinInfoContextHasAntiJoin(joinInfoContext))

View File

@ -91,12 +91,10 @@ static MultiCollect * CollectNodeForTable(List *collectTableList, uint32 rangeTa
static MultiSelect * MultiSelectNode(List *pushdownableClauseList,
List *nonPushdownableClauseList);
static bool IsSelectClause(Node *clause);
static List * SelectClauses(List *clauseList);
static List * SelectClauseList(List *clauseList);
static JoinInfoContext * FetchJoinOrderContext(FromExpr *fromExpr);
static bool JoinInfoWalker(Node *node, JoinInfoContext *joinInfoContext);
static ApplicableJoinClauseContext * ExtractApplicableJoinClauseContext(
List *joinOrderList);
/* Local functions forward declarations for applying joins */
static MultiNode * ApplyJoinRule(MultiNode *leftNode, MultiNode *rightNode,
@ -589,7 +587,7 @@ MultiNodeTree(Query *queryTree, PlannerRestrictionContext *plannerRestrictionCon
}
/* extract join and nonjoin clauses from plannerRestrictionContext */
RestrictInfoContext *restrictInfoContext = ExtractRestrictionInfosFromPlannerContext(
RestrictInfoContext *restrictInfoContext = ExtractRestrictInfosFromPlannerContext(
plannerRestrictionContext);
List *joinRestrictInfoList = restrictInfoContext->joinRestrictInfoList;
List *nonJoinRestrictionInfoList = restrictInfoContext->baseRestrictInfoList;
@ -706,14 +704,13 @@ MultiNodeTree(Query *queryTree, PlannerRestrictionContext *plannerRestrictionCon
/* we simply donot commute joins as we have at least 1 outer join */
joinOrderList = FixedJoinOrderList(tableEntryList, joinInfoContext,
joinRestrictInfoListList,
generatedEcJoinClauseList,
pseudoClauseList);
generatedEcJoinClauseList);
}
else
{
/* find best join order for commutative inner joins */
joinOrderList = JoinOrderList(tableEntryList, joinRestrictInfoListList,
generatedEcJoinClauseList, pseudoClauseList);
generatedEcJoinClauseList);
}
/* build join tree using the join order and collected tables */
@ -737,7 +734,7 @@ MultiNodeTree(Query *queryTree, PlannerRestrictionContext *plannerRestrictionCon
* - some of join clauses can be pushed down. See below for details.
*/
ApplicableJoinClauseContext *applicableJoinClauseContext =
ExtractApplicableJoinClauseContext(
ExtractApplicableJoinClauseContextFromJoinList(
joinOrderList);
/*
@ -754,7 +751,7 @@ MultiNodeTree(Query *queryTree, PlannerRestrictionContext *plannerRestrictionCon
* be put into pushdownable part of MultiSelect clause, so we differentiate those from
* actual pushdownable parts in join clauses.
*/
List *pushdownableSelectJoinClauseList = SelectClauses(pushdownableJoinClauseList);
List *pushdownableSelectJoinClauseList = SelectClauseList(pushdownableJoinClauseList);
List *nonPushdownableSelectJoinClauseList = list_difference(
pushdownableJoinClauseList,
pushdownableSelectJoinClauseList);
@ -790,10 +787,10 @@ MultiNodeTree(Query *queryTree, PlannerRestrictionContext *plannerRestrictionCon
/*
* SelectClauses returns select clauses from given clause list.
* SelectClauseList returns select clauses from given clause list.
*/
static List *
SelectClauses(List *clauseList)
SelectClauseList(List *clauseList)
{
List *selectClauseList = NIL;
@ -810,51 +807,11 @@ SelectClauses(List *clauseList)
}
/*
* ExtractApplicableJoinClauseContext returns ApplicableJoinClauseContext which contains
* all pushdownable and nonpushdownable clauses from given joinOrderList.
*/
static ApplicableJoinClauseContext *
ExtractApplicableJoinClauseContext(List *joinOrderList)
{
List *pushdownableJoinClauseList = NIL;
List *nonPushdownableJoinClauseList = NIL;
JoinOrderNode *joinOrderNode = NULL;
foreach_ptr(joinOrderNode, joinOrderList)
{
ApplicableJoinClauseContext *nodeApplicableJoinClauseContext =
joinOrderNode->applicableJoinClauseContext;
/* first node does not contain ApplicableJoinClauseContext */
if (nodeApplicableJoinClauseContext == NULL)
{
continue;
}
pushdownableJoinClauseList = list_concat_unique(pushdownableJoinClauseList,
nodeApplicableJoinClauseContext->
pushdownableJoinClauseList);
nonPushdownableJoinClauseList = list_concat_unique(nonPushdownableJoinClauseList,
nodeApplicableJoinClauseContext
->nonPushdownableJoinClauseList);
}
ApplicableJoinClauseContext *applicableJoinClauseContext = palloc0(
sizeof(ApplicableJoinClauseContext));
applicableJoinClauseContext->joinClauseList = list_concat_copy(
pushdownableJoinClauseList, nonPushdownableJoinClauseList);
applicableJoinClauseContext->pushdownableJoinClauseList = pushdownableJoinClauseList;
applicableJoinClauseContext->nonPushdownableJoinClauseList =
nonPushdownableJoinClauseList;
return applicableJoinClauseContext;
}
/*
* RestrictInfoContext extracts all RestrictionInfo from PlannerRestrictionContext.
*/
RestrictInfoContext *
ExtractRestrictionInfosFromPlannerContext(
ExtractRestrictInfosFromPlannerContext(
PlannerRestrictionContext *plannerRestrictionContext)
{
RelationRestrictionContext *relationRestrictionContext =

View File

@ -114,8 +114,6 @@ static bool ContainsReferencesToRelidsWalker(Node *node,
RelidsReferenceWalkerContext *context);
static bool HasRightRecursiveJoin(FromExpr *fromExpr);
static bool RightRecursiveJoinExprWalker(Node *node, void *context);
static bool HasCartesianJoin(FromExpr *fromExpr);
static bool CartesianJoinExprWalker(Node *node, void *context);
/*
@ -206,22 +204,13 @@ ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery,
{
return true;
}
/*
* join order planner cannot handle cartesian joins when query tree contains outer
* join.
*/
if (HasCartesianJoin(rewrittenQuery->jointree))
{
return true;
}
}
/*
* Some unsupported join clauses in logical planner
* may be supported by subquery pushdown planner.
*/
RestrictInfoContext *restrictInfoContext = ExtractRestrictionInfosFromPlannerContext(
RestrictInfoContext *restrictInfoContext = ExtractRestrictInfosFromPlannerContext(
plannerRestrictionContext);
List *joinRestrictionInfoList = restrictInfoContext->joinRestrictInfoList;
List *nonJoinRestrictionInfoList = restrictInfoContext->baseRestrictInfoList;
@ -291,55 +280,6 @@ RightRecursiveJoinExprWalker(Node *node, void *context)
}
/*
* HasCartesianJoin returns true if join tree contains any cartesian join.
*/
static bool
HasCartesianJoin(FromExpr *fromExpr)
{
if (fromExpr && list_length(fromExpr->fromlist) > 1)
{
return true;
}
JoinExpr *joinExpr = NULL;
foreach_ptr(joinExpr, fromExpr->fromlist)
{
if (CartesianJoinExprWalker((Node *) joinExpr, NULL))
{
return true;
}
}
return false;
}
/*
* CartesianJoinExprWalker is helper method for HasCartesianJoin.
*/
static bool
CartesianJoinExprWalker(Node *node, void *context)
{
if (node == NULL)
{
return false;
}
if (IsA(node, FromExpr))
{
FromExpr *fromExpr = (FromExpr *) node;
if (list_length(fromExpr->fromlist) > 1)
{
return true;
}
}
return expression_tree_walker(node, CartesianJoinExprWalker, NULL);
}
/*
* JoinTreeContainsSubquery returns true if the input query contains any subqueries
* in the join tree (e.g., FROM clause).

View File

@ -44,8 +44,6 @@ extern DistributedPlan * CreateInsertSelectIntoLocalTablePlan(uint64 planId,
plannerRestrictionContext);
extern char * InsertSelectResultIdPrefix(uint64 planId);
extern bool PlanningInsertSelect(void);
extern void RelabelPlannerRestrictionContext(
PlannerRestrictionContext *plannerRestrictionContext);
#endif /* INSERT_SELECT_PLANNER_H */

View File

@ -66,7 +66,6 @@ typedef struct TableEntry
*/
typedef struct ApplicableJoinClauseContext
{
List *joinClauseList;
List *pushdownableJoinClauseList;
List *nonPushdownableJoinClauseList;
} ApplicableJoinClauseContext;
@ -129,12 +128,11 @@ extern bool EnableSingleHashRepartitioning;
/* Function declaration for determining table join orders */
extern List * JoinExprList(FromExpr *fromExpr);
extern List * JoinOrderList(List *rangeTableEntryList, List *joinRestrictInfoListList,
List *generatedEcJoinClauseList, List *pseudoClauseList);
List *generatedEcJoinClauseList);
extern List * FixedJoinOrderList(List *rangeTableEntryList,
JoinInfoContext *joinInfoContext,
List *joinRestrictInfoListList,
List *generatedEcJoinClauseList,
List *pseudoClauseList);
List *generatedEcJoinClauseList);
extern bool IsApplicableJoinClause(List *leftTableIdList, uint32 rightTableId,
Node *joinClause);
extern bool IsApplicableFalseConstantJoinClause(List *leftTableIdList,
@ -154,6 +152,7 @@ extern Var * DistPartitionKeyOrError(Oid relationId);
extern char PartitionMethod(Oid relationId);
extern char TableReplicationModel(Oid relationId);
extern bool ExtractLeftMostRangeTableIndex(Node *node, int *rangeTableIndex);
extern ApplicableJoinClauseContext * ExtractApplicableJoinClauseContextFromJoinList(
List *joinOrderList);
#endif /* MULTI_JOIN_ORDER_H */

View File

@ -238,7 +238,7 @@ extern bool IsJoinClause(Node *clause);
extern List * SubqueryEntryList(Query *queryTree);
extern DeferredErrorMessage * DeferErrorIfQueryNotSupported(Query *queryTree);
extern List * WhereClauseList(FromExpr *fromExpr);
extern RestrictInfoContext * ExtractRestrictionInfosFromPlannerContext(
extern RestrictInfoContext * ExtractRestrictInfosFromPlannerContext(
PlannerRestrictionContext *plannerRestrictionContext);
extern List * TableEntryList(List *rangeTableList);
extern List * UsedTableEntryList(Query *query);