From 80c6c52a2e3d7cce545adf8a4502c4e28aee0f57 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Wed, 30 Oct 2024 14:42:38 +0800 Subject: [PATCH] [feature](Nereids): Optimize Query Plan by Pulling Up Join with Common Child from Union (#42033) This pr adds rewrite rule PullUpJoinFromUnion, supports pull up join from union all separately, separates this rule from #28682, deletes the original PullUpJoinFromUnionAll rule. --- .../doris/nereids/jobs/executor/Rewriter.java | 20 +- .../apache/doris/nereids/rules/RuleType.java | 2 +- .../rules/rewrite/PullUpJoinFromUnionAll.java | 1025 ++++++++--------- .../rewrite/PullUpJoinFromUnionTest.java | 171 +++ .../nereids_hint_tpcds_p0/shape/query11.out | 2 +- .../nereids_hint_tpcds_p0/shape/query14.out | 62 +- .../nereids_hint_tpcds_p0/shape/query4.out | 2 +- .../nereids_hint_tpcds_p0/shape/query71.out | 41 +- .../nereids_hint_tpcds_p0/shape/query74.out | 2 +- .../nereids_hint_tpcds_p0/shape/query76.out | 58 +- .../pull_up_join_from_union.out | 306 +++++ .../shape/query11.out | 2 +- .../shape/query14.out | 62 +- .../shape/query4.out | 2 +- .../shape/query71.out | 45 +- .../shape/query74.out | 2 +- .../shape/query76.out | 58 +- .../noStatsRfPrune/query11.out | 2 +- .../noStatsRfPrune/query14.out | 44 +- .../noStatsRfPrune/query4.out | 2 +- .../noStatsRfPrune/query71.out | 41 +- .../noStatsRfPrune/query74.out | 2 +- .../noStatsRfPrune/query76.out | 63 +- .../no_stats_shape/query11.out | 2 +- .../no_stats_shape/query14.out | 62 +- .../no_stats_shape/query4.out | 2 +- .../no_stats_shape/query71.out | 41 +- .../no_stats_shape/query74.out | 2 +- .../no_stats_shape/query76.out | 63 +- .../rf_prune/query11.out | 2 +- .../rf_prune/query14.out | 44 +- .../rf_prune/query4.out | 2 +- .../rf_prune/query71.out | 45 +- .../rf_prune/query74.out | 2 +- .../rf_prune/query76.out | 63 +- .../shape/query11.out | 2 +- .../shape/query14.out | 62 +- .../shape/query4.out | 2 +- .../shape/query71.out | 45 +- .../shape/query74.out | 2 +- .../shape/query76.out | 63 +- .../shape/query14.out | 62 +- .../shape/query71.out | 41 +- .../shape/query76.out | 63 +- .../tpcds_sf100/noStatsRfPrune/query11.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query14.out | 44 +- .../tpcds_sf100/noStatsRfPrune/query4.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query71.out | 41 +- .../tpcds_sf100/noStatsRfPrune/query74.out | 2 +- .../tpcds_sf100/noStatsRfPrune/query76.out | 63 +- .../tpcds_sf100/no_stats_shape/query11.out | 2 +- .../tpcds_sf100/no_stats_shape/query14.out | 62 +- .../tpcds_sf100/no_stats_shape/query4.out | 2 +- .../tpcds_sf100/no_stats_shape/query71.out | 41 +- .../tpcds_sf100/no_stats_shape/query74.out | 2 +- .../tpcds_sf100/no_stats_shape/query76.out | 63 +- .../tpcds_sf100/rf_prune/query11.out | 2 +- .../tpcds_sf100/rf_prune/query14.out | 44 +- .../tpcds_sf100/rf_prune/query4.out | 2 +- .../tpcds_sf100/rf_prune/query71.out | 45 +- .../tpcds_sf100/rf_prune/query74.out | 2 +- .../tpcds_sf100/rf_prune/query76.out | 63 +- .../tpcds_sf100/shape/query11.out | 2 +- .../tpcds_sf100/shape/query14.out | 62 +- .../tpcds_sf100/shape/query4.out | 2 +- .../tpcds_sf100/shape/query71.out | 45 +- .../tpcds_sf100/shape/query74.out | 2 +- .../tpcds_sf100/shape/query76.out | 63 +- .../tpcds_sf1000/shape/query11.out | 2 +- .../tpcds_sf1000/shape/query14.out | 62 +- .../tpcds_sf1000/shape/query4.out | 2 +- .../tpcds_sf1000/shape/query71.out | 45 +- .../tpcds_sf1000/shape/query74.out | 2 +- .../tpcds_sf1000/shape/query76.out | 63 +- .../pull_up_join_from_union.groovy | 429 +++++++ 75 files changed, 2242 insertions(+), 1711 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionTest.java create mode 100644 regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out create mode 100644 regression-test/suites/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index 28d43076bca7ec..4feeb6439e46c8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -355,26 +355,10 @@ public class Rewriter extends AbstractBatchJobExecutor { bottomUp(new EliminateJoinByFK()), topDown(new EliminateJoinByUnique()) ), - - // this rule should be after topic "Column pruning and infer predicate" - topic("Join pull up", - topDown( - new EliminateFilter(), - new PushDownFilterThroughProject(), - new MergeProjects() - ), - topDown( - new PullUpJoinFromUnionAll() - ), - custom(RuleType.COLUMN_PRUNING, ColumnPruning::new), - bottomUp(RuleSet.PUSH_DOWN_FILTERS), - custom(RuleType.ELIMINATE_UNNECESSARY_PROJECT, EliminateUnnecessaryProject::new) - ), - - // this rule should be invoked after topic "Join pull up" topic("eliminate Aggregate according to fd items", topDown(new EliminateGroupByKey()), - topDown(new PushDownAggThroughJoinOnPkFk()) + topDown(new PushDownAggThroughJoinOnPkFk()), + topDown(new PullUpJoinFromUnionAll()) ), topic("Limit optimization", diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index e0875c63e653f6..430d2b35b35727 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -303,7 +303,7 @@ public enum RuleType { // split limit SPLIT_LIMIT(RuleTypeClass.REWRITE), - PULL_UP_JOIN_FROM_UNIONALL(RuleTypeClass.REWRITE), + PULL_UP_JOIN_FROM_UNION_ALL(RuleTypeClass.REWRITE), // limit push down PUSH_LIMIT_THROUGH_JOIN(RuleTypeClass.REWRITE), PUSH_LIMIT_THROUGH_PROJECT_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java index 28d70ad558e656..b3df9b92c56a84 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java @@ -17,648 +17,599 @@ package org.apache.doris.nereids.rules.rewrite; -import org.apache.doris.catalog.constraint.ForeignKeyConstraint; -import org.apache.doris.catalog.constraint.PrimaryKeyConstraint; -import org.apache.doris.catalog.constraint.UniqueConstraint; -import org.apache.doris.nereids.hint.DistributeHint; -import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.catalog.constraint.TableIdentifier; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.EqualTo; -import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.plans.DistributeType; -import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; -import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; -import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; -import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import java.util.ArrayList; -import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; +import javax.annotation.Nullable; /** - * Pull up join from union all rule. + * Pull up join from union all rules with project: + * Union + * / \ + * project project + * (optional) (optional) + * | | + * Join Join + * / \ / \ + * t1 t2 t1 t3 (t1 is common side; t2,t3 is other side) + * =====> + * project + * | + * Join + * / \ + * Union t1 + * / \ + * project project + * (optional) (optional) + * | | + * t2 t3 */ public class PullUpJoinFromUnionAll extends OneRewriteRuleFactory { - private static final Set> SUPPORTED_PLAN_TYPE = ImmutableSet.of( - LogicalFilter.class, - LogicalJoin.class, - LogicalProject.class, - LogicalCatalogRelation.class - ); - - private static class PullUpContext { - public static final String unifiedOutputAlias = "PULL_UP_UNIFIED_OUTPUT_ALIAS"; - public final Map> pullUpCandidatesMaps = Maps.newHashMap(); - public final Map tableToJoinRootMap = Maps.newHashMap(); - public final Map tableToAggrRootMap = Maps.newHashMap(); - public final Map origChild0ToNewUnionOutputMap = Maps.newHashMap(); - public final List aggrChildList = Lists.newArrayList(); - public final List joinChildList = Lists.newArrayList(); - public final List replaceColumns = Lists.newArrayList(); - public final Map pullUpTableToPkSlotMap = Maps.newHashMap(); - public int replacedColumnIndex = -1; - public LogicalCatalogRelation pullUpTable; - - // the slot will replace the original pk in group by and select list - public SlotReference replaceColumn; - public boolean needAddReplaceColumn = false; - - public PullUpContext() {} - - public void setReplacedColumn(SlotReference slot) { - this.replaceColumn = slot; - } - - public void setPullUpTable(LogicalCatalogRelation table) { - this.pullUpTable = table; - } - - public void setNeedAddReplaceColumn(boolean needAdd) { - this.needAddReplaceColumn = needAdd; - } - } - @Override public Rule build() { return logicalUnion() - .when(union -> union.getQualifier() != Qualifier.DISTINCT) - .then(union -> { - PullUpContext context = new PullUpContext(); - if (!checkUnionPattern(union, context) - || !checkJoinCondition(context) - || !checkGroupByKeys(context)) { - return null; - } - // only support single table pull up currently - if (context.pullUpCandidatesMaps.entrySet().size() != 1) { - return null; - } + .when(union -> union.getQualifier() != Qualifier.DISTINCT + && union.getConstantExprsList().isEmpty()) + .then(union -> { + HashMap, Plan>>> commonChildrenMap = + tryToExtractCommonChild(union); + if (commonChildrenMap == null) { + return null; + } - List pullUpTableList = context.pullUpCandidatesMaps - .entrySet().iterator().next().getValue(); - if (pullUpTableList.size() != union.children().size() - || context.replaceColumns.size() != union.children().size() - || !checkNoFilterOnPullUpTable(pullUpTableList, context)) { - return null; - } - // make new union node - LogicalUnion newUnionNode = makeNewUnionNode(union, pullUpTableList, context); - // make new join node - LogicalJoin newJoin = makeNewJoin(newUnionNode, pullUpTableList.get(0), context); - // add project on pull up table with origin union output - List newProjectOutputs = makeNewProjectOutputs(union, newJoin, context); + // The joinsAndCommonSides size is the same as the number of union children. + List, Plan>> joinsAndCommonSides = null; + for (List, Plan>> childSet : commonChildrenMap.values()) { + if (childSet.size() == union.children().size()) { + joinsAndCommonSides = childSet; + break; + } + } + if (joinsAndCommonSides == null) { + return null; + } - return new LogicalProject(newProjectOutputs, newJoin); - }).toRule(RuleType.PULL_UP_JOIN_FROM_UNIONALL); - } + List> otherOutputsList = new ArrayList<>(); + List> upperProjectExpressionOrIndex = new ArrayList<>(); + // First, check whether the output of the union child meets the requirements. + if (!checkUnionChildrenOutput(union, joinsAndCommonSides, otherOutputsList, + upperProjectExpressionOrIndex)) { + return null; + } - private boolean checkUnionPattern(LogicalUnion union, PullUpContext context) { - int tableListNumber = -1; - for (Plan child : union.children()) { - if (!(child instanceof LogicalProject - && child.child(0) instanceof LogicalAggregate - && child.child(0).child(0) instanceof LogicalProject - && child.child(0).child(0).child(0) instanceof LogicalJoin)) { - return false; - } - LogicalAggregate aggrRoot = (LogicalAggregate) child.child(0); - if (!checkAggrRoot(aggrRoot)) { - return false; - } - context.aggrChildList.add(aggrRoot); - LogicalJoin joinRoot = (LogicalJoin) aggrRoot.child().child(0); - // check join under union is spj - if (!checkJoinRoot(joinRoot)) { - return false; - } - context.joinChildList.add(joinRoot); + List>> commonSlotToOtherSlotMaps = new ArrayList<>(); + Set joinCommonSlots = new LinkedHashSet<>(); + if (!checkJoinCondition(joinsAndCommonSides, commonSlotToOtherSlotMaps, joinCommonSlots)) { + return null; + } - List tableList = getTableListUnderJoin(joinRoot); - // add into table -> joinRoot map - for (LogicalCatalogRelation table : tableList) { - context.tableToJoinRootMap.put(table, joinRoot); - context.tableToAggrRootMap.put(table, aggrRoot); - } - if (tableListNumber == -1) { - tableListNumber = tableList.size(); - } else { - // check all union children have the same number of tables - if (tableListNumber != tableList.size()) { - return false; - } - } + Map> commonSlotToProjectsIndex = new HashMap<>(); + LogicalUnion newUnion = constructNewUnion(joinsAndCommonSides, otherOutputsList, + commonSlotToOtherSlotMaps, joinCommonSlots, commonSlotToProjectsIndex); + LogicalJoin newJoin = constructNewJoin(newUnion, + commonSlotToProjectsIndex, joinsAndCommonSides); + LogicalProject newProject = constructNewProject(union, newJoin, upperProjectExpressionOrIndex); + return newProject; + }).toRule(RuleType.PULL_UP_JOIN_FROM_UNION_ALL); + } - for (LogicalCatalogRelation table : tableList) { - // key: qualified table name - // value: table list in all union children - String qName = makeQualifiedName(table); - if (context.pullUpCandidatesMaps.get(qName) == null) { - List newList = new ArrayList<>(); - newList.add(table); - context.pullUpCandidatesMaps.put(qName, newList); - } else { - context.pullUpCandidatesMaps.get(qName).add(table); - } - } + private LogicalProject constructNewProject(LogicalUnion originUnion, LogicalJoin newJoin, + List> upperProjectExpressionOrIndex) { + List originOutput = originUnion.getOutput(); + List upperProjects = new ArrayList<>(); + List newUnionOutput = newJoin.left().getOutput(); + if (originOutput.size() != upperProjectExpressionOrIndex.size()) { + return null; } - int expectedNumber = union.children().size(); - List toBeRemoved = new ArrayList<>(); - // check the pull up table candidate exists in all union children - for (Map.Entry> e : context.pullUpCandidatesMaps.entrySet()) { - if (e.getValue().size() != expectedNumber) { - toBeRemoved.add(e.getKey()); + for (int i = 0; i < upperProjectExpressionOrIndex.size(); ++i) { + Pair pair = upperProjectExpressionOrIndex.get(i); + boolean fromCommon = pair.first; + if (fromCommon) { + upperProjects.add(new Alias(originOutput.get(i).getExprId(), pair.second.exprFromCommonSide, + originOutput.get(i).getName())); + } else { + upperProjects.add(new Alias(originOutput.get(i).getExprId(), + newUnionOutput.get(pair.second.indexOfNewUnionOutput), originOutput.get(i).getName())); } } - for (String key : toBeRemoved) { - context.pullUpCandidatesMaps.remove(key); - } - return !context.pullUpCandidatesMaps.isEmpty(); + return new LogicalProject<>(upperProjects, newJoin); } - private boolean checkJoinCondition(PullUpContext context) { - List toBeRemoved = new ArrayList<>(); - for (Map.Entry> e : context.pullUpCandidatesMaps.entrySet()) { - List tableList = e.getValue(); - boolean allFound = true; - for (LogicalCatalogRelation table : tableList) { - LogicalJoin joinRoot = context.tableToJoinRootMap.get(table); - if (joinRoot == null) { - return false; - } else if (!checkJoinConditionOnPk(joinRoot, table, context)) { - allFound = false; - break; - } - } - if (!allFound) { - toBeRemoved.add(e.getKey()); + private LogicalJoin constructNewJoin(LogicalUnion union, + Map> commonSlotToProjectsIndex, + List, Plan>> commonChild) { + LogicalJoin originalJoin = commonChild.iterator().next().first; + Plan newCommon = commonChild.iterator().next().second; + List newHashExpressions = new ArrayList<>(); + List unionOutputs = union.getOutput(); + for (Map.Entry> entry : commonSlotToProjectsIndex.entrySet()) { + SlotReference commonSlot = entry.getKey(); + for (Integer index : entry.getValue()) { + newHashExpressions.add(new EqualTo(unionOutputs.get(index), commonSlot)); } } - for (String table : toBeRemoved) { - context.pullUpCandidatesMaps.remove(table); - } - - if (context.pullUpCandidatesMaps.isEmpty()) { - return false; - } - return true; + return (LogicalJoin) originalJoin + .withJoinConjuncts(newHashExpressions, ImmutableList.of(), originalJoin.getJoinReorderContext()) + .withChildren(union, newCommon); } - private boolean checkGroupByKeys(PullUpContext context) { - List toBeRemoved = new ArrayList<>(); - for (Map.Entry> e : context.pullUpCandidatesMaps.entrySet()) { - List tableList = e.getValue(); - boolean allFound = true; - for (LogicalCatalogRelation table : tableList) { - LogicalAggregate aggrRoot = context.tableToAggrRootMap.get(table); - if (aggrRoot == null) { - return false; - } else if (!checkAggrKeyOnUkOrPk(aggrRoot, table)) { - allFound = false; - break; - } + // Output parameter: commonSlotToProjectsIndex, key is the common slot of join condition, + // value is the index of the other slot corresponding to this common slot in the union output, + // which is used to construct the join condition of the new join. + private LogicalUnion constructNewUnion(List, Plan>> joinsAndCommonSides, + List> otherOutputsList, + List>> commonSlotToOtherSlotMaps, + Set joinCommonSlots, Map> commonSlotToProjectsIndex) { + List newChildren = new ArrayList<>(); + for (int i = 0; i < joinsAndCommonSides.size(); ++i) { + Pair, Plan> pair = joinsAndCommonSides.get(i); + // find the child that is not the common side + Plan otherSide; + if (pair.second == pair.first.left()) { + otherSide = pair.first.right(); + } else { + otherSide = pair.first.left(); } - if (!allFound) { - toBeRemoved.add(e.getKey()); + List projects = otherOutputsList.get(i); + // In projects, we also need to add the other slot in join condition + // TODO: may eliminate repeated output slots: + // e.g.select t2.a from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + // select t3.a from test_like1 t1 join test_like3 t3 on t1.a=t3.a; + // new union child will output t2.a/t3.a twice. one for output, the other for join condition. + Map> commonSlotToOtherSlotMap = commonSlotToOtherSlotMaps.get(i); + for (SlotReference commonSlot : joinCommonSlots) { + List otherSlots = commonSlotToOtherSlotMap.get(commonSlot); + for (SlotReference otherSlot : otherSlots) { + if (i == 0) { + int index = projects.size(); + commonSlotToProjectsIndex.computeIfAbsent(commonSlot, k -> new ArrayList<>()).add(index); + } + projects.add(otherSlot); + } } - } - for (String table : toBeRemoved) { - context.pullUpCandidatesMaps.remove(table); + LogicalProject logicalProject = new LogicalProject<>(projects, otherSide); + newChildren.add(logicalProject); } - if (context.pullUpCandidatesMaps.isEmpty()) { - return false; - } - return true; + //2. construct new union + LogicalUnion newUnion = new LogicalUnion(Qualifier.ALL, newChildren); + List> childrenOutputs = newChildren.stream() + .map(p -> p.getOutput().stream() + .map(SlotReference.class::cast) + .collect(ImmutableList.toImmutableList())) + .collect(ImmutableList.toImmutableList()); + newUnion = (LogicalUnion) newUnion.withChildrenAndTheirOutputs(newChildren, childrenOutputs); + newUnion = newUnion.withNewOutputs(newUnion.buildNewOutputs()); + return newUnion; } - private boolean checkNoFilterOnPullUpTable(List pullUpTableList, PullUpContext context) { - for (LogicalCatalogRelation table : pullUpTableList) { - LogicalJoin joinRoot = context.tableToJoinRootMap.get(table); - if (joinRoot == null) { - return false; + /** This function is used to check whether the join condition meets the optimization condition + * Check the join condition, requiring that the join condition of each join is equal and the number is the same. + * Generate commonSlotToOtherSlotMaps. In each map of the list, the keySet must be the same, + * and the length of the value list of the same key must be the same. + * These are sql that can not do this transform: + * SQL1: select t2.a+1,2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select t3.a+1,3 from test_like1 t1 join test_like3 t3 on t1.a=t3.a and t1.b=t3.b; + * SQL2: select t2.a+1,2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select t3.a+1,3 from test_like1 t1 join test_like3 t3 on t1.b=t3.a; + * SQL3: select t2.a+1,2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select t3.a+1,3 from test_like1 t1 join test_like3 t3 on t1.a=t3.a and t1.a=t3.b; + * @param commonSlotToOtherSlotMaps Output parameter that records the join conditions for each join operation. + * The key represents the slot on the common side of the join, while the value + * corresponds to the slot on the other side. + * Example: + * For the following SQL: + * SELECT t2.a + 1, 2 FROM test_like1 t1 + * JOIN test_like2 t2 ON t1.a = t2.a AND t1.a = t2.c AND t1.b = t2.b + * UNION ALL + * SELECT t3.a + 1, 3 FROM test_like1 t1 + * JOIN test_like3 t3 ON t1.a = t3.a AND t1.a = t3.d AND t1.b = t3.b; + * commonSlotToOtherSlotMaps would be: + * {{t1.a: t2.a, t2.c; t1.b: t2.b}, {t1.a: t3.a, t3.d; t1.b: t3.b}} + * This parameter is used to verify if the join conditions meet + * optimization requirements and to help generate new join conditions. + * @param joinCommonSlots output parameter, which records join common side slots. + * */ + private boolean checkJoinCondition(List, Plan>> joinsAndCommonSides, + List>> commonSlotToOtherSlotMaps, + Set joinCommonSlots) { + Map> conditionMapFirst = new HashMap<>(); + Map commonJoinSlotMap = buildCommonJoinMap(joinsAndCommonSides); + for (int i = 0; i < joinsAndCommonSides.size(); ++i) { + Pair, Plan> pair = joinsAndCommonSides.get(i); + LogicalJoin join = pair.first; + Plan commonSide = pair.second; + Map> conditionMapSubsequent = new HashMap<>(); + for (Expression condition : join.getHashJoinConjuncts()) { + if (!(condition instanceof EqualTo)) { + return false; + } + EqualTo equalTo = (EqualTo) condition; + if (!(equalTo.left() instanceof SlotReference) || !(equalTo.right() instanceof SlotReference)) { + return false; + } + SlotReference commonSideSlot; + SlotReference otherSideSlot; + if (commonSide.getOutputSet().contains(equalTo.left())) { + commonSideSlot = (SlotReference) equalTo.left(); + otherSideSlot = (SlotReference) equalTo.right(); + } else { + commonSideSlot = (SlotReference) equalTo.right(); + otherSideSlot = (SlotReference) equalTo.left(); + } + if (i == 0) { + conditionMapFirst.computeIfAbsent(commonSideSlot, k -> new ArrayList<>()).add(otherSideSlot); + joinCommonSlots.add(commonSideSlot); + } else { + conditionMapSubsequent.computeIfAbsent( + (SlotReference) ExpressionUtils.replace(commonSideSlot, commonJoinSlotMap), + k -> new ArrayList<>()).add(otherSideSlot); + } + } + if (i == 0) { + commonSlotToOtherSlotMaps.add(conditionMapFirst); } else { - List filterList = new ArrayList<>(); - filterList.addAll((Collection) - joinRoot.collect(LogicalFilter.class::isInstance)); - for (LogicalFilter filter : filterList) { - if (filter.child().equals(context.pullUpTable)) { + // reject SQL1 + if (conditionMapSubsequent.size() != conditionMapFirst.size()) { + return false; + } + // reject SQL2 + if (!conditionMapSubsequent.keySet().equals(conditionMapFirst.keySet())) { + return false; + } + // reject SQL3 + for (Map.Entry> entry : conditionMapFirst.entrySet()) { + SlotReference commonSlot = entry.getKey(); + if (conditionMapSubsequent.get(commonSlot).size() != entry.getValue().size()) { return false; } } + commonSlotToOtherSlotMaps.add(conditionMapSubsequent); } } return true; } - private boolean checkAggrKeyOnUkOrPk(LogicalAggregate aggregate, LogicalCatalogRelation table) { - List groupByKeys = aggregate.getGroupByExpressions(); - boolean isAllSlotReference = groupByKeys.stream().allMatch(e -> e instanceof SlotReference); - if (!isAllSlotReference) { - return false; - } else { - Set ukInfo = getUkInfoFromConstraint(table); - Set pkInfo = getPkInfoFromConstraint(table); - if (ukInfo == null || pkInfo == null || ukInfo.size() != 1 || pkInfo.size() != 1) { - return false; + // Make a map to map the output of all other joins to the output of the first join + private Map buildCommonJoinMap(List, Plan>> commonChild) { + Map commonJoinSlotMap = new HashMap<>(); + List firstJoinOutput = new ArrayList<>(); + for (int i = 0; i < commonChild.size(); ++i) { + Pair, Plan> pair = commonChild.get(i); + Plan commonSide = pair.second; + if (i == 0) { + firstJoinOutput.addAll(commonSide.getOutput()); + for (Slot slot : commonSide.getOutput()) { + commonJoinSlotMap.put(slot, slot); + } } else { - String ukName = ukInfo.iterator().next(); - String pkName = pkInfo.iterator().next(); - for (Object expr : aggregate.getGroupByExpressions()) { - SlotReference slot = (SlotReference) expr; - if (table.getOutputExprIds().contains(slot.getExprId()) - && (slot.getName().equals(ukName) || slot.getName().equals(pkName))) { - return true; - } + for (int j = 0; j < commonSide.getOutput().size(); ++j) { + commonJoinSlotMap.put(commonSide.getOutput().get(j), firstJoinOutput.get(j)); } - return false; } } + return commonJoinSlotMap; } - private boolean checkJoinConditionOnPk(LogicalJoin joinRoot, LogicalCatalogRelation table, PullUpContext context) { - Set pkInfos = getPkInfoFromConstraint(table); - if (pkInfos == null || pkInfos.size() != 1) { + private class ExpressionOrIndex { + Expression exprFromCommonSide = null; + int indexOfNewUnionOutput = -1; + + private ExpressionOrIndex(Expression expr) { + exprFromCommonSide = expr; + } + + private ExpressionOrIndex(int index) { + indexOfNewUnionOutput = index; + } + } + + /** In the union child output, the number of outputs from the common side must be the same in each child output, + * and the outputs from the common side must be isomorphic (both a+1) and have the same index in the union output. + * In the union child output, the number of outputs from the non-common side must also be the same, + * but they do not need to be isomorphic. + * These are sql that can not do this transform: + * SQL1: select t2.a+t1.a from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select t3.a+1 from test_like1 t1 join test_like3 t3 on t1.a=t3.a; + * SQL2: select t2.a from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select t1.a from test_like1 t1 join test_like3 t3 on t1.a=t3.a; + * SQL3: select t1.a+1 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select t1.a+2 from test_like1 t1 join test_like3 t3 on t1.a=t3.a; + * SQL4: select t1.a from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + * select 1 from test_like1 t1 join test_like3 t3 on t1.a=t3.a; + * @param otherOutputsList output parameter that stores the outputs of the other side. + * The length of each element in otherOutputsList must be the same. + * The i-th element represents the output of the other side in the i-th child of the union. + * This parameter is used to create child nodes of a new Union + * in the constructNewUnion function. + * + * @param upperProjectExpressionOrIndex Output parameter used in the constructNewProject function to create + * the top-level project.This parameter records the output column order of + * the original union and determines,based on the new join output, the columns + * or expressions to output in the upper-level project operator. The size of + * upperProjectExpressionOrIndex must match the output size of + * the original union。 + * Each Pair in the List represents an output source: + * - Pair.first (Boolean): Indicates whether the output is from + * the common side (true) or the other side (false). + * - Pair.second (Object): When Pair.first is true, it stores + * the common side's output expression.When false, it saves the output index + * of the other side. Since the new union output is not yet constructed + * at this point, only the index is stored. + * The function’s check ensures that outputs at the same position in + * union children either come from the common side or from the other side. + * When the final join is constructed, the common side uses the first join's + * common side, so only the first child’s outputs need to be processed to + * fill in upperProjectExpressionOrIndex. + */ + private boolean checkUnionChildrenOutput(LogicalUnion union, + List, Plan>> joinsAndCommonSides, + List> otherOutputsList, + List> upperProjectExpressionOrIndex) { + List> regularChildrenOutputs = union.getRegularChildrenOutputs(); + int arity = union.arity(); + if (arity == 0) { return false; } - String pkSlot = pkInfos.iterator().next(); - List joinList = new ArrayList<>(); - joinList.addAll((Collection) joinRoot.collect(LogicalJoin.class::isInstance)); - boolean found = false; - for (LogicalJoin join : joinList) { - List conditions = join.getHashJoinConjuncts(); - List basicTableList = new ArrayList<>(); - basicTableList.addAll((Collection) join - .collect(LogicalCatalogRelation.class::isInstance)); - for (Expression equalTo : conditions) { - if (equalTo instanceof EqualTo - && ((EqualTo) equalTo).left() instanceof SlotReference - && ((EqualTo) equalTo).right() instanceof SlotReference) { - SlotReference leftSlot = (SlotReference) ((EqualTo) equalTo).left(); - SlotReference rightSlot = (SlotReference) ((EqualTo) equalTo).right(); - if (table.getOutputExprIds().contains(leftSlot.getExprId()) - && pkSlot.equals(leftSlot.getName())) { - // pk-fk join condition, check other side's join key is on fk - LogicalCatalogRelation rightTable = findTableFromSlot(rightSlot, basicTableList); - if (rightTable != null && getFkInfoFromConstraint(rightTable) != null) { - ForeignKeyConstraint fkInfo = getFkInfoFromConstraint(rightTable); - if (fkInfo.getReferencedTable().getId() == table.getTable().getId()) { - for (Map.Entry entry : fkInfo.getForeignToReference().entrySet()) { - if (entry.getValue().equals(pkSlot) && entry.getKey().equals(rightSlot.getName())) { - found = true; - context.replaceColumns.add(rightSlot); - context.pullUpTableToPkSlotMap.put(table, leftSlot); - break; - } + // fromCommonSide is used to ensure that the outputs at the same position in the union children + // must all come from the common side or from the other side + boolean[] fromCommonSide = new boolean[regularChildrenOutputs.get(0).size()]; + // checkSameExpr and commonJoinSlotMap are used to ensure that Expr from the common side have the same structure + Expression[] checkSameExpr = new Expression[regularChildrenOutputs.get(0).size()]; + Map commonJoinSlotMap = buildCommonJoinMap(joinsAndCommonSides); + for (int i = 0; i < arity; ++i) { + List regularChildrenOutput = regularChildrenOutputs.get(i); + Plan child = union.child(i); + List otherOutputs = new ArrayList<>(); + for (int j = 0; j < regularChildrenOutput.size(); ++j) { + SlotReference slot = regularChildrenOutput.get(j); + if (child instanceof LogicalProject) { + LogicalProject project = (LogicalProject) child; + int index = project.getOutput().indexOf(slot); + NamedExpression expr = project.getOutputs().get(index); + Slot insideSlot; + Expression insideExpr; + Set inputSlots = expr.getInputSlots(); + // reject SQL1 + if (inputSlots.size() > 1) { + return false; + } else if (inputSlots.size() == 1) { + if (expr instanceof Alias) { + insideSlot = inputSlots.iterator().next(); + insideExpr = expr.child(0); + } else if (expr instanceof SlotReference) { + insideSlot = (Slot) expr; + insideExpr = expr; + } else { + return false; + } + + Plan commonSide = joinsAndCommonSides.get(i).second; + if (i == 0) { + if (commonSide.getOutputSet().contains(insideSlot)) { + fromCommonSide[j] = true; + checkSameExpr[j] = insideExpr; + upperProjectExpressionOrIndex.add(Pair.of(true, new ExpressionOrIndex(insideExpr))); + } else { + fromCommonSide[j] = false; + upperProjectExpressionOrIndex.add(Pair.of(false, new ExpressionOrIndex( + otherOutputs.size()))); + otherOutputs.add(expr); + } + } else { + // reject SQL2 + if (commonSide.getOutputSet().contains(insideSlot) != fromCommonSide[j]) { + return false; + } + // reject SQL3 + if (commonSide.getOutputSet().contains(insideSlot)) { + Expression sameExpr = ExpressionUtils.replace(insideExpr, commonJoinSlotMap); + if (!sameExpr.equals(checkSameExpr[j])) { + return false; } + } else { + otherOutputs.add(expr); } } - } else if (table.getOutputExprIds().contains(rightSlot.getExprId()) - && pkSlot.equals(rightSlot.getName())) { - // pk-fk join condition, check other side's join key is on fk - LogicalCatalogRelation leftTable = findTableFromSlot(leftSlot, basicTableList); - if (leftTable != null && getFkInfoFromConstraint(leftTable) != null) { - ForeignKeyConstraint fkInfo = getFkInfoFromConstraint(leftTable); - if (fkInfo.getReferencedTable().getId() == table.getTable().getId()) { - for (Map.Entry entry : fkInfo.getForeignToReference().entrySet()) { - if (entry.getValue().equals(pkSlot) && entry.getKey().equals(leftSlot.getName())) { - found = true; - context.replaceColumns.add(leftSlot); - context.pullUpTableToPkSlotMap.put(table, rightSlot); - break; - } - } + } else if (expr.getInputSlots().isEmpty()) { + // Constants must come from other side + if (i == 0) { + fromCommonSide[j] = false; + upperProjectExpressionOrIndex.add(Pair.of(false, new ExpressionOrIndex( + otherOutputs.size()))); + } else { + // reject SQL4 + if (fromCommonSide[j]) { + return false; } } + otherOutputs.add(expr); } - if (found) { - break; + } else if (child instanceof LogicalJoin) { + Plan commonSide = joinsAndCommonSides.get(i).second; + if (i == 0) { + if (commonSide.getOutputSet().contains(slot)) { + fromCommonSide[j] = true; + checkSameExpr[j] = slot; + upperProjectExpressionOrIndex.add(Pair.of(true, new ExpressionOrIndex(slot))); + } else { + fromCommonSide[j] = false; + upperProjectExpressionOrIndex.add(Pair.of(false, + new ExpressionOrIndex(otherOutputs.size()))); + otherOutputs.add(slot); + } + } else { + // reject SQL2 + if (commonSide.getOutputSet().contains(slot) != fromCommonSide[j]) { + return false; + } + // reject SQL3 + if (commonSide.getOutputSet().contains(slot)) { + Expression sameExpr = ExpressionUtils.replace(slot, commonJoinSlotMap); + if (!sameExpr.equals(checkSameExpr[j])) { + return false; + } + } else { + otherOutputs.add(slot); + } } } } - if (found) { - break; - } + otherOutputsList.add(otherOutputs); } - return found; + return true; } - private LogicalCatalogRelation findTableFromSlot(SlotReference targetSlot, - List tableList) { - for (LogicalCatalogRelation table : tableList) { - if (table.getOutputExprIds().contains(targetSlot.getExprId())) { - return table; + /** + * Attempts to extract common children from a LogicalUnion. + * + * This method iterates through all children of the union, looking for LogicalJoin operations, + * and tries to identify common left or right subtrees. The results are stored in a Map where + * keys are potential common subtrees and values are lists of pairs containing the original + * join and the corresponding subtree. + * + * For example, given the following union: + * Union + * ├─ Join(A, B) + * ├─ Join(A, C) + * └─ Join(D, B) + * + * The returned Map would contain: + * A -> [(Join(A,B), A), (Join(A,C), A)] + * B -> [(Join(A,B), B), (Join(D,B), B)] + * + * This indicates that both A and B are potential common subtrees that could be extracted. + * + * @param union The LogicalUnion to analyze + * @return A Map containing potential common subtrees, or null if extraction is not possible + */ + private @Nullable HashMap, Plan>>> tryToExtractCommonChild(LogicalUnion union) { + HashMap, Plan>>> planCount = new HashMap<>(); + for (Plan child : union.children()) { + LogicalJoin join = tryToGetJoin(child); + if (join == null) { + return null; + } + boolean added = false; + for (Plan plan : planCount.keySet()) { + LogicalPlanComparator comparator = new LogicalPlanComparator(); + if (comparator.isLogicalEqual(join.left(), plan)) { + planCount.get(plan).add(Pair.of(join, join.left())); + added = true; + break; + } else if (comparator.isLogicalEqual(join.right(), plan)) { + planCount.get(plan).add(Pair.of(join, join.right())); + added = true; + break; + } } - } - return null; - } - private ForeignKeyConstraint getFkInfoFromConstraint(LogicalCatalogRelation table) { - Set foreignKeyConstraints = table.getTable().getForeignKeyConstraints(); - if (foreignKeyConstraints.isEmpty()) { - return null; + if (!added) { + planCount.put(join.left(), Lists.newArrayList(Pair.of(join, join.left()))); + planCount.put(join.right(), Lists.newArrayList(Pair.of(join, join.right()))); + } } - return foreignKeyConstraints.stream().iterator().next(); + return planCount; } - private Set getPkInfoFromConstraint(LogicalCatalogRelation table) { - Set primaryKeyConstraints = table.getTable().getPrimaryKeyConstraints(); - if (primaryKeyConstraints.isEmpty()) { - return null; + // we only allow project(join) or join() + private @Nullable LogicalJoin tryToGetJoin(Plan child) { + if (child instanceof LogicalProject) { + child = child.child(0); } - return primaryKeyConstraints.stream().iterator().next().getPrimaryKeyNames(); - } - - private Set getUkInfoFromConstraint(LogicalCatalogRelation table) { - Set uniqueConstraints = table.getTable().getUniqueConstraints(); - if (uniqueConstraints.isEmpty()) { - return null; + if (child instanceof LogicalJoin + && ((LogicalJoin) child).getJoinType().isInnerJoin() + && ((LogicalJoin) child).getOtherJoinConjuncts().isEmpty() + && !((LogicalJoin) child).isMarkJoin()) { + return (LogicalJoin) child; } - return uniqueConstraints.stream().iterator().next().getUniqueColumnNames(); + return null; } - private boolean checkJoinRoot(LogicalJoin joinRoot) { - List joinChildrenPlans = Lists.newArrayList(); - joinChildrenPlans.addAll((Collection) joinRoot - .collect(LogicalPlan.class::isInstance)); - boolean planTypeMatch = joinChildrenPlans.stream() - .allMatch(p -> SUPPORTED_PLAN_TYPE.stream().anyMatch(c -> c.isInstance(p))); - if (!planTypeMatch) { - return false; - } - - List allJoinNodes = Lists.newArrayList(); - allJoinNodes.addAll((Collection) joinRoot.collect(LogicalJoin.class::isInstance)); - boolean joinTypeMatch = allJoinNodes.stream().allMatch(e -> e.getJoinType() == JoinType.INNER_JOIN); - boolean joinConditionMatch = allJoinNodes.stream() - .allMatch(e -> !e.getHashJoinConjuncts().isEmpty() && e.getOtherJoinConjuncts().isEmpty()); - if (!joinTypeMatch || !joinConditionMatch) { - return false; - } - - return true; - } + class LogicalPlanComparator { + private HashMap plan1ToPlan2 = new HashMap<>(); - private boolean checkAggrRoot(LogicalAggregate aggrRoot) { - for (Object expr : aggrRoot.getGroupByExpressions()) { - if (!(expr instanceof NamedExpression)) { + public boolean isLogicalEqual(Plan plan1, Plan plan2) { + if (plan1.children().size() != plan2.children().size()) { return false; } - } - return true; - } - - private List getTableListUnderJoin(LogicalJoin joinRoot) { - List tableLists = new ArrayList<>(); - tableLists.addAll((Collection) joinRoot - .collect(LogicalCatalogRelation.class::isInstance)); - return tableLists; - } - - private String makeQualifiedName(LogicalCatalogRelation table) { - String dbName = table.getTable().getDatabase().getFullName(); - String tableName = table.getTable().getName(); - return dbName + ":" + tableName; - } - - private Plan doPullUpJoinFromUnionAll(Plan unionChildPlan, PullUpContext context) { - return PullUpRewriter.INSTANCE.rewrite(unionChildPlan, context); - } - - private List doWrapReplaceColumnForUnionChildren(List unionChildren, PullUpContext context) { - List newUnionChildren = new ArrayList<>(); - for (int i = 0; i < unionChildren.size(); i++) { - // has been checked before - LogicalProject oldProject = (LogicalProject) unionChildren.get(i); - List newNamedExpressionList = new ArrayList<>(); - for (int j = 0; j < oldProject.getProjects().size(); j++) { - Object child = oldProject.getProjects().get(j); - if (context.replaceColumns.contains(child)) { - Alias newExpr = new Alias((Expression) child, context.unifiedOutputAlias); - newNamedExpressionList.add(newExpr); - context.replacedColumnIndex = j; - } else { - newNamedExpressionList.add((NamedExpression) child); - } - } - LogicalProject newProject = new LogicalProject(newNamedExpressionList, (LogicalPlan) oldProject.child()); - newUnionChildren.add(newProject); - } - return newUnionChildren; - } - - private List makeNewProjectOutputs(LogicalUnion origUnion, - LogicalJoin newJoin, PullUpContext context) { - List newProjectOutputs = new ArrayList<>(); - List origUnionSlots = origUnion.getOutput(); - List origUnionChildOutput = ((LogicalProject) origUnion.child(0)).getOutputs(); - for (int i = 0; i < origUnionChildOutput.size(); i++) { - NamedExpression unionOutputExpr = origUnionChildOutput.get(i); - if (unionOutputExpr instanceof Alias) { - if (!(unionOutputExpr.child(0) instanceof Literal)) { - unionOutputExpr = (Slot) unionOutputExpr.child(0); - } - } - boolean found = false; - Slot matchedJoinSlot = null; - for (Slot joinOutput : newJoin.getOutput()) { - Slot slot = joinOutput; - if (context.origChild0ToNewUnionOutputMap.get(slot) != null) { - slot = context.origChild0ToNewUnionOutputMap.get(slot); - } - if (slot.equals(unionOutputExpr) || slot.getExprId() == unionOutputExpr.getExprId()) { - matchedJoinSlot = joinOutput; - found = true; - break; + for (int i = 0; i < plan1.children().size(); i++) { + if (!isLogicalEqual(plan1.child(i), plan2.child(i))) { + return false; } } - if (found) { - ExprId exprId = origUnionSlots.get(i).getExprId(); - Alias aliasExpr = new Alias(exprId, matchedJoinSlot, matchedJoinSlot.toSql()); - newProjectOutputs.add(aliasExpr); + if (isNotSupported(plan1) || isNotSupported(plan2)) { + return false; } + return comparePlan(plan1, plan2); } - return newProjectOutputs; - } - - private LogicalJoin makeNewJoin(LogicalUnion newUnionNode, - LogicalCatalogRelation pullUpTable, PullUpContext context) { - List newHashJoinConjuncts = new ArrayList<>(); - Slot unionSideExpr = newUnionNode.getOutput().get(context.replacedColumnIndex); - Slot pullUpSidePkSlot = context.pullUpTableToPkSlotMap.get(pullUpTable); - if (pullUpSidePkSlot == null) { - return null; + boolean isNotSupported(Plan plan) { + return !(plan instanceof LogicalFilter) + && !(plan instanceof LogicalCatalogRelation) + && !(plan instanceof LogicalProject); } - EqualTo pullUpJoinCondition = new EqualTo(unionSideExpr, pullUpSidePkSlot); - newHashJoinConjuncts.add(pullUpJoinCondition); - - // new a join with the newUnion and the pulled up table - return new LogicalJoin<>( - JoinType.INNER_JOIN, - newHashJoinConjuncts, - ExpressionUtils.EMPTY_CONDITION, - new DistributeHint(DistributeType.NONE), - Optional.empty(), - newUnionNode, - pullUpTable, null); - } - - private LogicalUnion makeNewUnionNode(LogicalUnion origUnion, - List pullUpTableList, PullUpContext context) { - List newUnionChildren = new ArrayList<>(); - for (int i = 0; i < origUnion.children().size(); i++) { - Plan unionChild = origUnion.child(i); - context.setPullUpTable(pullUpTableList.get(i)); - context.setReplacedColumn(context.replaceColumns.get(i)); - Plan newChild = doPullUpJoinFromUnionAll(unionChild, context); - newUnionChildren.add(newChild); - } - - // wrap the replaced column with a shared alias which is exposed to outside - List formalizedNewUnionChildren = doWrapReplaceColumnForUnionChildren(newUnionChildren, context); - List> childrenOutputs = formalizedNewUnionChildren.stream() - .map(j -> j.getOutput().stream() - .map(SlotReference.class::cast) - .collect(ImmutableList.toImmutableList())) - .collect(ImmutableList.toImmutableList()); - - LogicalUnion newUnionNode = new LogicalUnion(Qualifier.ALL, formalizedNewUnionChildren); - newUnionNode = (LogicalUnion) newUnionNode.withChildrenAndTheirOutputs( - formalizedNewUnionChildren, childrenOutputs); - List newOutputs = newUnionNode.buildNewOutputs(); - newUnionNode = newUnionNode.withNewOutputs(newOutputs); - - // set up origin child 0 output to new union output mapping - List origChild0Output = childrenOutputs.get(0); - for (int i = 0; i < origChild0Output.size(); i++) { - SlotReference slot = origChild0Output.get(i); - NamedExpression newExpr = newOutputs.get(i); - context.origChild0ToNewUnionOutputMap.put(newExpr, slot); - } - - return newUnionNode; - } - - private static class PullUpRewriter extends DefaultPlanRewriter implements CustomRewriter { - public static final PullUpRewriter INSTANCE = new PullUpRewriter(); - @Override - public Plan rewriteRoot(Plan plan, JobContext context) { - return null; - } - - public Plan rewrite(Plan plan, PullUpContext context) { - return plan.accept(this, context); - } - - @Override - public Plan visitLogicalAggregate(LogicalAggregate agg, PullUpContext context) { - Plan input = agg.child().accept(this, context); - - LogicalCatalogRelation pullTable = context.pullUpTable; - SlotReference replaceColumn = context.replaceColumn; - - // eliminate group by keys - List groupByExprList = new ArrayList<>(); - for (Expression expr : agg.getGroupByExpressions()) { - // expr has been checked before - if (!pullTable.getOutputExprIds().contains(((NamedExpression) expr).getExprId())) { - groupByExprList.add(expr); + boolean comparePlan(Plan plan1, Plan plan2) { + boolean isEqual = true; + if (plan1 instanceof LogicalCatalogRelation && plan2 instanceof LogicalCatalogRelation) { + isEqual = new TableIdentifier(((LogicalCatalogRelation) plan1).getTable()) + .equals(new TableIdentifier(((LogicalCatalogRelation) plan2).getTable())); + } else if (plan1 instanceof LogicalProject && plan2 instanceof LogicalProject) { + if (plan1.getOutput().size() != plan2.getOutput().size()) { + isEqual = false; } - } - // add replaced group by key - groupByExprList.add(replaceColumn); - - // eliminate outputs keys - List outputExprList = new ArrayList<>(); - for (NamedExpression expr : agg.getOutputExpressions()) { - if (!pullTable.getOutputExprIds().contains(expr.getExprId())) { - outputExprList.add(expr); + for (int i = 0; isEqual && i < plan2.getOutput().size(); i++) { + NamedExpression expr = ((LogicalProject) plan1).getProjects().get(i); + NamedExpression replacedExpr = (NamedExpression) + expr.rewriteUp(e -> plan1ToPlan2.getOrDefault(e, e)); + if (!replacedExpr.equals(((LogicalProject) plan2).getProjects().get(i))) { + isEqual = false; + } } - } - // add replaced group by key - outputExprList.add(replaceColumn); - return new LogicalAggregate<>(groupByExprList, outputExprList, input); - } - - public Plan visitLogicalJoin(LogicalJoin join, PullUpContext context) { - Plan leftChild = join.child(0).accept(this, context); - Plan rightChild = join.child(1).accept(this, context); - LogicalCatalogRelation pullUpTable = context.pullUpTable; - - // no filter on pull up table, which has been checked before - if (leftChild instanceof LogicalCatalogRelation - && leftChild.equals(pullUpTable)) { - context.setNeedAddReplaceColumn(true); - return rightChild; - } else if (rightChild instanceof LogicalCatalogRelation - && rightChild.equals(pullUpTable)) { - context.setNeedAddReplaceColumn(true); - return leftChild; - } else if (leftChild instanceof LogicalProject - && leftChild.child(0) instanceof LogicalCatalogRelation - && leftChild.child(0).equals(pullUpTable)) { - context.setNeedAddReplaceColumn(true); - return rightChild; - } else if (rightChild instanceof LogicalProject - && rightChild.child(0) instanceof LogicalCatalogRelation - && rightChild.child(0).equals(pullUpTable)) { - context.setNeedAddReplaceColumn(true); - return leftChild; + } else if (plan1 instanceof LogicalFilter && plan2 instanceof LogicalFilter) { + Set replacedConjuncts = new HashSet<>(); + for (Expression expr : ((LogicalFilter) plan1).getConjuncts()) { + replacedConjuncts.add(expr.rewriteUp(e -> plan1ToPlan2.getOrDefault(e, e))); + } + isEqual = replacedConjuncts.equals(((LogicalFilter) plan2).getConjuncts()); } else { - return new LogicalJoin(JoinType.INNER_JOIN, - join.getHashJoinConjuncts(), - join.getOtherJoinConjuncts(), - new DistributeHint(DistributeType.NONE), - Optional.empty(), - leftChild, rightChild, null); + isEqual = false; } - } - - @Override - public Plan visitLogicalProject(LogicalProject project, PullUpContext context) { - Plan input = project.child().accept(this, context); - List outputs = input.getOutput().stream() - .map(e -> (NamedExpression) e).collect(Collectors.toList()); - for (NamedExpression expr : project.getProjects()) { - // handle alias - if (expr instanceof Alias && expr.child(0) instanceof Literal) { - outputs.add(expr); - } + if (!isEqual) { + return false; } - return new LogicalProject<>(outputs, input); - } - - @Override - public Plan visitLogicalFilter(LogicalFilter filter, PullUpContext context) { - Plan input = filter.child().accept(this, context); - return new LogicalFilter<>(filter.getConjuncts(), input); + for (int i = 0; i < plan1.getOutput().size(); i++) { + plan1ToPlan2.put(plan1.getOutput().get(i), plan2.getOutput().get(i)); + } + return true; } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionTest.java new file mode 100644 index 00000000000000..3c533216b1acb0 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionTest.java @@ -0,0 +1,171 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +class PullUpJoinFromUnionTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("default_cluster:test"); + createTables( + "CREATE TABLE IF NOT EXISTS t1 (\n" + + " id int not null,\n" + + " name char\n" + + ")\n" + + "DUPLICATE KEY(id)\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\"replication_num\" = \"1\")\n", + "CREATE TABLE IF NOT EXISTS t2 (\n" + + " id int not null,\n" + + " name char\n" + + ")\n" + + "DUPLICATE KEY(id)\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\"replication_num\" = \"1\")\n", + "CREATE TABLE IF NOT EXISTS t3 (\n" + + " id int,\n" + + " name char\n" + + ")\n" + + "DUPLICATE KEY(id)\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\"replication_num\" = \"1\")\n" + ); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + void testSimple() { + String sql = "select * from t1 join t2 on t1.id = t2.id " + + "union all " + + "select * from t1 join t3 on t1.id = t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + } + + @Test + void testProject() { + String sql = "select t2.id from t1 join t2 on t1.id = t2.id " + + "union all " + + "select t3.id from t1 join t3 on t1.id = t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + + sql = "select t2.id, t1.name from t1 join t2 on t1.id = t2.id " + + "union all " + + "select t3.id, t1.name from t1 join t3 on t1.id = t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + } + + @Test + void testConstant() { + String sql = "select t2.id, t1.name, 1 as id1 from t1 join t2 on t1.id = t2.id " + + "union all " + + "select t3.id, t1.name, 2 as id2 from t1 join t3 on t1.id = t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + } + + @Test + void testComplexProject() { + String sql = "select t2.id + 1, t1.name + 1, 1 as id1 from t1 join t2 on t1.id = t2.id " + + "union all " + + "select t3.id + 1, t1.name + 1, 2 as id2 from t1 join t3 on t1.id = t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalUnion(), any())); + } + + @Test + void testMissJoinSlot() { + String sql = "select t1.name + 1, 1 as id1 from t1 join t2 on t1.id = t2.id " + + "union all " + + "select t1.name + 1, 2 as id2 from t1 join t3 on t1.id = t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalUnion(), any())); + } + + @Test + void testFilter() { + String sql = "select * from t1 join t2 on t1.id = t2.id where t1.name = '' " + + "union all " + + "select * from t1 join t3 on t1.id = t3.id where t1.name = '' ;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + + sql = "select t2.id from t1 join t2 on t1.id = t2.id where t1.name = '' " + + "union all " + + "select t3.id from t1 join t3 on t1.id = t3.id where t1.name = '' ;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + } + + @Test + void testMultipleJoinConditions() { + String sql = "select * from t1 join t2 on t1.id = t2.id and t1.name = t2.name " + + "union all " + + "select * from t1 join t3 on t1.id = t3.id and t1.name = t3.name;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + } + + @Test + void testNonEqualityJoinConditions() { + String sql = "select * from t1 join t2 on t1.id < t2.id " + + "union all " + + "select * from t1 join t3 on t1.id < t3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .nonMatch(logicalJoin(logicalProject(logicalUnion()), any())); + } + + @Test + void testSubqueries() { + String sql = "select * from t1 join (select * from t2 where t2.id > 10) s2 on t1.id = s2.id " + + "union all " + + "select * from t1 join (select * from t3 where t3.id > 10) s3 on t1.id = s3.id;"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalJoin(logicalProject(logicalUnion()), any())); + } +} diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out index 16316c630a5d62..d8cf8455c334a3 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out index 61fba77c876be0..0d4af14e34d2cc 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,16 +87,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[ss_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -120,16 +110,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[cs_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -143,16 +133,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF20 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF19 ss_item_sk->[ws_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF20 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out index 57c89d80df9807..ee0be517b75e68 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query71.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query71.out index 0df4b3fff81a7f..fc0e6dc12ce55e 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query71.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query71.out @@ -9,34 +9,25 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((item.i_manager_id = 1)) ----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query74.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query74.out index 8b171914ebd371..30e95b3fd06a84 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query74.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out index b612ba67e96ec1..473b9fded85715 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out @@ -7,40 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------PhysicalProject -----------------------------filter(ss_customer_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] apply RFs: RF2 +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------filter(ss_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------------PhysicalProject ---------------------------filter(ws_promo_sk IS NULL) -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 ---------------------PhysicalProject -----------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_promo_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------filter(cs_bill_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------filter(cs_bill_customer_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out b/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out new file mode 100644 index 00000000000000..10820e9ee48414 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out @@ -0,0 +1,306 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !basic_join_union -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] + +-- !three_way_union -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +------PhysicalOlapScan[table_d] +----PhysicalOlapScan[table_a] + +-- !union_with_projections -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] + +-- !union_with_constants -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] + +-- !union_with_loss_slots -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] + +-- !different_join_conditions -- +PhysicalResultSink +--PhysicalUnion +----hashJoin[INNER_JOIN] hashCondition=((a.id = b.id)) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_b] +----hashJoin[INNER_JOIN] hashCondition=((a.name = c.name)) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_c] + +-- !multi_column_join -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id) and (name = name)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] + +-- !left_joins -- +PhysicalResultSink +--PhysicalUnion +----hashJoin[LEFT_OUTER_JOIN] hashCondition=((a.id = b.id)) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_b] +----hashJoin[LEFT_OUTER_JOIN] hashCondition=((a.id = c.id)) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_c] + +-- !subquery_join -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------hashAgg[LOCAL] +--------PhysicalOlapScan[table_b] +------hashAgg[LOCAL] +--------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] + +-- !complex_join_condition1 -- +PhysicalResultSink +--PhysicalUnion +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_b] +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_c] + +-- !complex_join_condition2 -- +PhysicalResultSink +--PhysicalUnion +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_b] +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as DOUBLE) = expr_(cast(id as DOUBLE) - 1.0))) otherCondition=() +------PhysicalOlapScan[table_a] +------PhysicalOlapScan[table_c] + +-- !union_filter1 -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------filter((b.id = 1)) +--------PhysicalOlapScan[table_b] +------filter((c.id = 1)) +--------PhysicalOlapScan[table_c] +----filter((a.id = 1)) +------PhysicalOlapScan[table_a] + +-- !union_filter2 -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[table_b] +------PhysicalOlapScan[table_c] +----filter((cast(value as DOUBLE) = 1.0)) +------PhysicalOlapScan[table_a] + +-- !basic_join_union_res -- +1 Alice Value_B1 +1 Alice Value_C1 +2 Bob Value_B2 +3 Charlie Value_C3 + +-- !three_way_union_res -- +1 Alice Value_B1 +1 Alice Value_C1 +1 Alice Value_D1 +2 Bob Value_B2 +2 Bob Value_D2 +3 Charlie Value_C3 +3 Charlie Value_D3 + +-- !union_with_projections_res -- +1 Alice VALUE_B1 +1 Alice value_c1 +2 Bob VALUE_B2 +3 Charlie value_c3 + +-- !union_with_constants_res -- +1 Alice Value_B1 B +1 Alice Value_C1 C +2 Bob Value_B2 B +3 Charlie Value_C3 C + +-- !union_with_loss_slots_res -- +1 +1 +2 +3 + +-- !different_join_conditions_res -- +1 Alice Value_B1 +1 Alice Value_C1 +2 Bob Value_B2 +3 Charlie Value_C3 + +-- !multi_column_join_res -- +1 Alice Value_B1 +1 Alice Value_C1 +2 Bob Value_B2 +3 Charlie Value_C3 + +-- !left_joins_res -- +1 Alice Value_B1 +1 Alice Value_C1 +2 Bob \N +2 Bob Value_B2 +3 Charlie \N +3 Charlie Value_C3 +5 Eva \N +5 Eva \N + +-- !subquery_join_res -- +1 Alice Value_B1 +1 Alice Value_C1 +2 Bob Value_B2 +3 Charlie Value_C3 + +-- !complex_join_condition1_res -- +1 Alice Value_B2 +2 Bob Value_C3 +3 Charlie Value_B4 +3 Charlie Value_C4 +5 Eva Value_B6 + +-- !complex_join_condition2_res -- +1 Alice Value_B2 +2 Bob Value_C3 +3 Charlie Value_B4 +3 Charlie Value_C4 +5 Eva Value_B6 + +-- !union_filter1_res -- +1 Alice Value_B1 B +1 Alice Value_C1 C + +-- !union_filter2_res -- + +-- !expr -- +1 4 +34 25 +34 35 +34 35 +34 35 +79 25 +101 5 + +-- !const -- +1 2 +34 2 +34 3 +34 3 +34 3 +79 2 +101 2 + +-- !multi_condition -- +\N 2 +\N 2 +\N 2 +1 2 +2 3 +11 2 + +-- !multi_condition2 -- +1 2 +2 3 +11 2 + +-- !multi_differenct_count_condition -- + +-- !no_common_side_project -- +1 4 +34 25 +34 35 +34 35 +34 35 +79 25 +101 5 + +-- !common_slot_differnt -- +34 5 +34 25 +34 101 + +-- !other_expr_differnt -- +1 4 +34 25 +34 103 +34 123 +34 199 +79 25 +101 5 + +-- !2_same_tables -- +0 0 +0 1 +33 1 +33 33 +78 1 +78 78 +100 1 +100 100 + +-- !simple_column -- +0 0 +0 0 +33 33 +33 33 +78 78 +78 78 +100 100 +100 100 + +-- !func_column -- +1 2 +34 2 +34 3 +34 3 +34 3 +79 2 +101 2 + +-- !other_join_slot_differnt -- +34 +34 +34 + +-- !join_common_slot_has_expr -- + +-- !can_not_transform -- +0 2 +2 0 +3 100 +23 33 +23 78 +33 23 +78 23 +100 3 + +-- !other_side_condition_slot_has_expr_do_transform -- +13 + diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out index 5593df9194937d..35504b7f44d24e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out index 813924b9b99e18..61f29b11211346 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,16 +87,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[ss_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -120,16 +110,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[cs_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -143,16 +133,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF20 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF19 ss_item_sk->[ws_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF20 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out index d99f0294700040..709da33d851bff 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query71.out index 068d0b83b15427..0d26e1f81ccb94 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query71.out @@ -9,37 +9,28 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_manager_id = 1)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +--------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(t_meal_time IN ('breakfast', 'dinner')) ------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out index 8b171914ebd371..30e95b3fd06a84 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out index b612ba67e96ec1..473b9fded85715 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out @@ -7,40 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------PhysicalProject -----------------------------filter(ss_customer_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] apply RFs: RF2 +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------filter(ss_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] ------------------------PhysicalProject ---------------------------filter(ws_promo_sk IS NULL) -----------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 ---------------------PhysicalProject -----------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_promo_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------filter(cs_bill_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------filter(cs_bill_customer_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out index a30f38dbe4c49a..8cab83d94f65ac 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out index d08a6aedb094e3..d817c6f0053791 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,12 +87,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF14 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() --------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 ----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] @@ -120,12 +110,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[cs_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() --------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF17 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 ----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] @@ -143,12 +133,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF20 d_date_sk->[ws_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() --------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF20 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 ----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out index 19e9098ee4555e..a083e5a72ef86a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query71.out index 4af8cfcf0e3ea9..7ae1c5b71ddaf1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query71.out @@ -9,34 +9,25 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((item.i_manager_id = 1)) ----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query74.out index d8a82ca998ac09..6915274e1a1301 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out index 1bbcdaee7491b7..2f21640b079929 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out index eaf1cde7b0a304..e963fc6a8caa7a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out index 04bf133065b269..e73d45b0732736 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,13 +87,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF14 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF12 ss_item_sk->[ss_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF13 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject @@ -120,13 +110,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[cs_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF16 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject @@ -143,13 +133,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF20 d_date_sk->[ws_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[ws_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF19 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out index 546119842b58bd..9a590246f64a4a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query71.out index 4af8cfcf0e3ea9..7ae1c5b71ddaf1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query71.out @@ -9,34 +9,25 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((item.i_manager_id = 1)) ----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query74.out index 64a56e4e850db7..c32a9187e34e92 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out index 768d8a9e8dcf7f..8f739a1d12b35a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ws_item_sk] ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out index b699aa67e934a0..e7ae73f8e00980 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out index 9d7cfc860ad6a4..48ac240d961d98 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -100,9 +90,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] @@ -123,9 +113,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] @@ -146,9 +136,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out index 43138dc7c62651..980ceef87cedc2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query71.out index b78dcd1d31e61d..3010f0b574e03b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query71.out @@ -9,37 +9,28 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_manager_id = 1)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +--------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(t_meal_time IN ('breakfast', 'dinner')) ------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out index f300a896a4d563..c3687dadd21872 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out index 0e72c30a539c8c..668c3625c56841 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 ss_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalOlapScan[date_dim] apply RFs: RF3 ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ws_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 cs_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out index bcfe7ba3d74e79..8abb7de87e97f9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out index 8a1467be7a4a58..5aad6142d951f9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,16 +87,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[ss_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -120,16 +110,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[cs_item_sk] +--------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -143,16 +133,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF20 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF19 ss_item_sk->[ws_item_sk] +--------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF20 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index d99f0294700040..709da33d851bff 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out index b78dcd1d31e61d..3010f0b574e03b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out @@ -9,37 +9,28 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_manager_id = 1)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +--------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(t_meal_time IN ('breakfast', 'dinner')) ------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index 0f159a647c03de..421b74396da876 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out index 0e72c30a539c8c..668c3625c56841 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 ss_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalOlapScan[date_dim] apply RFs: RF3 ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ws_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 cs_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out index 299549d2b2ce02..9d5c47615cb77c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,13 +87,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF14 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF12 ss_item_sk->[ss_item_sk] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF13 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject @@ -120,13 +110,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[cs_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF16 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject @@ -143,13 +133,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF20 d_date_sk->[ws_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[ws_item_sk] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF19 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query71.out index b2717237b73c8b..b76b7d6566e595 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query71.out @@ -9,34 +9,25 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((item.i_manager_id = 1)) ----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out index d929aca4ca8137..47d1baebaf8c15 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------PhysicalProject -----------------------------filter(ss_customer_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------filter(ss_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +----------------------------PhysicalProject +------------------------------filter(ws_ship_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------filter(cs_ship_mode_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ws_item_sk] ---------------------------PhysicalProject -----------------------------filter(ws_ship_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------filter(cs_ship_mode_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out index a30f38dbe4c49a..8cab83d94f65ac 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out index 1c17f5a78a9ca3..10192bf86cb782 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,12 +87,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF14 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() --------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 ----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] @@ -120,13 +110,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[cs_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF17 +--------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[item] --------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) @@ -144,13 +134,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF20 d_date_sk->[ws_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF20 +--------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[item] --------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out index 19e9098ee4555e..a083e5a72ef86a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query71.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query71.out index 4af8cfcf0e3ea9..7ae1c5b71ddaf1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query71.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query71.out @@ -9,34 +9,25 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((item.i_manager_id = 1)) ----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query74.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query74.out index d8a82ca998ac09..6915274e1a1301 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query74.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out index 1bbcdaee7491b7..2f21640b079929 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out index eaf1cde7b0a304..e963fc6a8caa7a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out index e4f277daf67cac..966f8701126465 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,13 +87,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF14 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF12 ss_item_sk->[ss_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF13 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[item] ----------------------------------------PhysicalProject @@ -120,15 +110,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[cs_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[cs_item_sk,i_item_sk] +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk,i_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[cs_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 +--------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] apply RFs: RF16 +--------------------------------------------------PhysicalOlapScan[item] apply RFs: RF14 --------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------------------------PhysicalProject ------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) @@ -144,15 +134,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF20 d_date_sk->[ws_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF19 ss_item_sk->[i_item_sk,ws_item_sk] +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[i_item_sk,ws_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ws_item_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[ws_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 +--------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] apply RFs: RF19 +--------------------------------------------------PhysicalOlapScan[item] apply RFs: RF17 --------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------------------------PhysicalProject ------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out index 546119842b58bd..9a590246f64a4a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query71.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query71.out index 4af8cfcf0e3ea9..7ae1c5b71ddaf1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query71.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query71.out @@ -9,34 +9,25 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------filter((item.i_manager_id = 1)) ----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query74.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query74.out index 64a56e4e850db7..c32a9187e34e92 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query74.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out index 768d8a9e8dcf7f..8f739a1d12b35a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ws_item_sk] ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out index b699aa67e934a0..e7ae73f8e00980 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out index 361e56fa135dab..2a29746e37ef07 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -100,9 +90,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] @@ -124,9 +114,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +--------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 ------------------------------------------------PhysicalProject --------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ----------------------------------------------------PhysicalOlapScan[date_dim] @@ -148,9 +138,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +--------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 ------------------------------------------------PhysicalProject --------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ----------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out index 43138dc7c62651..980ceef87cedc2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query71.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query71.out index b78dcd1d31e61d..3010f0b574e03b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query71.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query71.out @@ -9,37 +9,28 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_manager_id = 1)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +--------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(t_meal_time IN ('breakfast', 'dinner')) ------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query74.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query74.out index f300a896a4d563..c3687dadd21872 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query74.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out index 0e72c30a539c8c..668c3625c56841 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 ss_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalOlapScan[date_dim] apply RFs: RF3 ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ws_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 cs_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out index bcfe7ba3d74e79..8abb7de87e97f9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out index d54d10fc56004f..196a98b5a2f51d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,16 +87,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[ss_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -120,18 +110,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[cs_item_sk,i_item_sk] +--------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk,i_item_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[cs_item_sk] +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 +--------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 ------------------------------------------------PhysicalProject --------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ----------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] apply RFs: RF17 +----------------------------------------------PhysicalOlapScan[item] apply RFs: RF15 ----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------------PhysicalProject ------------------------------PhysicalAssertNumRows @@ -144,18 +134,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF20 ss_item_sk->[i_item_sk,ws_item_sk] +--------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[i_item_sk,ws_item_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[ws_item_sk] +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ws_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 +--------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 ------------------------------------------------PhysicalProject --------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ----------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] apply RFs: RF20 +----------------------------------------------PhysicalOlapScan[item] apply RFs: RF18 ----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------------PhysicalProject ------------------------------PhysicalAssertNumRows diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out index d99f0294700040..709da33d851bff 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query71.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query71.out index b78dcd1d31e61d..3010f0b574e03b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query71.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query71.out @@ -9,37 +9,28 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_manager_id = 1)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +--------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(t_meal_time IN ('breakfast', 'dinner')) ------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query74.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query74.out index 0f159a647c03de..421b74396da876 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query74.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out index 0e72c30a539c8c..668c3625c56841 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 ss_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter(ss_hdemo_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalOlapScan[date_dim] apply RFs: RF3 ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ws_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------filter(ws_bill_addr_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 cs_sold_date_sk->[d_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------filter(cs_warehouse_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter(ss_hdemo_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_bill_addr_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------filter(cs_warehouse_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out index 5593df9194937d..35504b7f44d24e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out index 813924b9b99e18..61f29b11211346 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out @@ -55,31 +55,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecGather] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF10 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[ws_sold_date_sk] +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF9 +--------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] apply RFs: RF11 -----------------------PhysicalProject -------------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) ---------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[web_sales] apply RFs: RF9 +----------------PhysicalProject +------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) +--------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN[MERGE_SORT] --------PhysicalDistribute[DistributionSpecGather] @@ -97,16 +87,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[ss_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 RF13 RF14 +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -120,16 +110,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[cs_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 RF16 RF17 +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject @@ -143,16 +133,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF20 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF19 ss_item_sk->[ws_item_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 RF19 RF20 +------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) --------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF20 +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out index d99f0294700040..709da33d851bff 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query71.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query71.out index 068d0b83b15427..0d26e1f81ccb94 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query71.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query71.out @@ -9,37 +9,28 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF4 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[cs_sold_time_sk,ss_sold_time_sk,ws_sold_time_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] -------------------------PhysicalUnion ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((tmp.sold_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk,ss_item_sk,ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalUnion +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_manager_id = 1)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +--------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2002)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(t_meal_time IN ('breakfast', 'dinner')) ------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query74.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query74.out index 8b171914ebd371..30e95b3fd06a84 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query74.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query74.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((PULL_UP_UNIFIED_OUTPUT_ALIAS = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] +------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] --------PhysicalProject ----------PhysicalUnion ------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out index c81d5d95feb264..473b9fded85715 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out @@ -7,41 +7,34 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------PhysicalProject -----------------------------filter(ss_customer_sk IS NULL) -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalUnion +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------filter(ss_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] apply RFs: RF1 +----------------------------PhysicalProject +------------------------------filter(ws_promo_sk IS NULL) +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------filter(cs_bill_customer_sk IS NULL) +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 ws_item_sk->[i_item_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------filter(ws_promo_sk IS NULL) -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ---------------------------PhysicalProject -----------------------------filter(cs_bill_customer_sk IS NULL) -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/suites/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.groovy b/regression-test/suites/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.groovy new file mode 100644 index 00000000000000..3eee499b2d865f --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.groovy @@ -0,0 +1,429 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("join_pull_up_union") { + sql "set runtime_filter_mode=OFF" + sql """ SET inline_cte_referenced_threshold=0 """ + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "SET disable_join_reorder=true" + + sql """ + -- Create tables + DROP TABLE IF EXISTS table_a; + CREATE TABLE table_a ( + id INT, + name VARCHAR(50), + value VARCHAR(50) + )ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + DROP TABLE IF EXISTS table_b; + CREATE TABLE table_b ( + id INT, + name VARCHAR(50), + value VARCHAR(50) + )ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + DROP TABLE IF EXISTS table_c; + CREATE TABLE table_c ( + id INT, + name VARCHAR(50), + value VARCHAR(50), + value1 VARCHAR(50), + value2 VARCHAR(50) + )ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + DROP TABLE IF EXISTS table_d; + CREATE TABLE table_d ( + id INT, + name VARCHAR(50), + value VARCHAR(50) + )ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """INSERT INTO table_a (id, name, value) VALUES + (1, 'Alice', 'Value_A1'), + (2, 'Bob', 'Value_A2'), + (3, 'Charlie', 'Value_A3'), + (5, 'Eva', 'Value_A5');""" + + sql """INSERT INTO table_b (id, name, value) VALUES + (1, 'Alice', 'Value_B1'), + (2, 'Bob', 'Value_B2'), + (4, 'Daniel', 'Value_B4'), + (6, 'Fiona', 'Value_B6');""" + + sql """INSERT INTO table_c (id, name, value, value1, value2) VALUES + (1, 'Alice', 'Value_C1', 'Extra_C1_1', 'Extra_C1_2'), + (3, 'Charlie', 'Value_C3', 'Extra_C3_1', 'Extra_C3_2'), + (4, 'Daniel', 'Value_C4', 'Extra_C4_1', 'Extra_C4_2'), + (7, 'Grace', 'Value_C7', 'Extra_C7_1', 'Extra_C7_2');""" + + sql """INSERT INTO table_d (id, name, value) VALUES + (1, 'Alice', 'Value_D1'), + (2, 'Bob', 'Value_D2'), + (3, 'Charlie', 'Value_D3'), + (8, 'Henry', 'Value_D8');""" + + + // Simple case with two tables joined in a union + qt_basic_join_union """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Three-way union with common join + qt_three_way_union """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.id = c.id + UNION ALL + SELECT a.id, a.name, d.value FROM table_a a JOIN table_d d ON a.id = d.id) t + """ + + // Union with projections above joins + qt_union_with_projections """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, UPPER(b.value) AS upper_value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, LOWER(c.value) AS lower_value FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Union with constant expressions + qt_union_with_constants """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Union with loss slots + qt_union_with_loss_slots """ + explain shape plan + SELECT t.id FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Union with different join conditions + qt_different_join_conditions """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.name = c.name) t + """ + + // Union with multi-column join conditions + qt_multi_column_join """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id AND a.name = b.name + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.id = c.id AND a.name = c.name) t + """ + + // Union with other joins + qt_left_joins """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a LEFT JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a LEFT JOIN table_c c ON a.id = c.id) t + """ + + // Union with subqueries in join + qt_subquery_join """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN (SELECT id, MAX(value) AS value FROM table_b GROUP BY id) b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN (SELECT id, MAX(value) AS value FROM table_c GROUP BY id) c ON a.id = c.id) t + """ + + // Union with complex expressions in join condition + qt_complex_join_condition1 """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON CAST(a.id AS INT) + 1 = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON CAST(a.id AS INT) + 1 = c.id) t + """ + + // Union with complex expressions in join condition + qt_complex_join_condition2 """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON CAST(a.id AS INT) + 1 = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON CAST(a.id AS DOUBLE) + 1 = c.id) t + """ + + qt_union_filter1 """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id where a.id = 1 + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id where a.id = 1) t + """ + + qt_union_filter2 """ + explain shape plan + SELECT * FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id where a.value = 1 + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id where a.value = 1) t + """ + + + // Simple case with two tables joined in a union + order_qt_basic_join_union_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Three-way union with common join + order_qt_three_way_union_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.id = c.id + UNION ALL + SELECT a.id, a.name, d.value FROM table_a a JOIN table_d d ON a.id = d.id) t + """ + + // Union with projections above joins + order_qt_union_with_projections_res """ + SELECT * FROM + (SELECT a.id, a.name, UPPER(b.value) AS upper_value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, LOWER(c.value) AS lower_value FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Union with constant expressions + order_qt_union_with_constants_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Union with loss slots + order_qt_union_with_loss_slots_res """ + SELECT t.id FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id) t + """ + + // Union with different join conditions + order_qt_different_join_conditions_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.name = c.name) t + """ + + // Union with multi-column join conditions + order_qt_multi_column_join_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON a.id = b.id AND a.name = b.name + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON a.id = c.id AND a.name = c.name) t + """ + + // Union with other joins + order_qt_left_joins_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a LEFT JOIN table_b b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a LEFT JOIN table_c c ON a.id = c.id) t + """ + + // Union with subqueries in join + order_qt_subquery_join_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN (SELECT id, MAX(value) AS value FROM table_b GROUP BY id) b ON a.id = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN (SELECT id, MAX(value) AS value FROM table_c GROUP BY id) c ON a.id = c.id) t + """ + + // Union with complex expressions in join condition + order_qt_complex_join_condition1_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON CAST(a.id AS INT) + 1 = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON CAST(a.id AS INT) + 1 = c.id) t + """ + + // Union with complex expressions in join condition + order_qt_complex_join_condition2_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value FROM table_a a JOIN table_b b ON CAST(a.id AS INT) + 1 = b.id + UNION ALL + SELECT a.id, a.name, c.value FROM table_a a JOIN table_c c ON CAST(a.id AS DOUBLE) + 1 = c.id) t + """ + + order_qt_union_filter1_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id where a.id = 1 + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id where a.id = 1) t + """ + + order_qt_union_filter2_res """ + SELECT * FROM + (SELECT a.id, a.name, b.value, 'B' AS source FROM table_a a JOIN table_b b ON a.id = b.id where a.value = 1 + UNION ALL + SELECT a.id, a.name, c.value, 'C' AS source FROM table_a a JOIN table_c c ON a.id = c.id where a.value = 1) t + """ + + sql """drop table if exists test_like1""" + sql """CREATE TABLE `test_like1` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + sql """drop table if exists test_like2""" + sql """CREATE TABLE `test_like2` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + sql """drop table if exists test_like3""" + sql """CREATE TABLE `test_like3` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + sql "drop table if exists test_like4" + sql """create table test_like4 (a bigint, b varchar(10), c int, d int) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + sql "insert into test_like1 values(100,'d2',3,5),(0,'d2',3,5),(null,null,9,3),(33,'d2',2,5),(null,'d2',3,55),(78,null,9,3),(12,null,9,3);" + sql "insert into test_like2 values(10,'d2',2,2),(0,'d2',2,2),(100,'d2',3,null),(null,null,9,3),(78,'d2',23,5),(33,'d2',23,5);" + sql "insert into test_like3 values(1,'d2',2,2),(33,'d2',99,5),(33,'d2',23,6),(33,'d2',3,5);" + sql "insert into test_like4 values(11,'d2',3,5),(1,'d2',3,5),(79,null,9,3),(33,'d2',2,5),(null,'d2',3,55),(78,null,9,3),(12,null,9,3);" + + qt_expr """select c1,c2 from + (select t2.a+1 c1,t2.c+2 c2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + select t3.a+1,t1.a+2 from test_like1 t1 join test_like3 t3 on t1.a=t3.a) t order by 1,2""" + qt_const """select c1,c2 from + (select t2.a+1 c1,2 c2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + select t3.a+1,3 from test_like1 t1 join test_like3 t3 on t1.a=t3.a) t order by 1,2""" + + qt_multi_condition """select c1,c2 from ( + select t2.a+1 c1,2 c2 from test_like1 t1 join test_like2 t2 on t1.c=t2.d AND t1.c=t2.d union ALL + select t3.a+1,3 from test_like1 t1 join test_like3 t3 on t1.c=t3.d AND t1.c=t3.d) t order by 1,2""" + + qt_multi_condition2 """select c1,c2 from ( + select t2.a+1 c1 ,2 c2 from test_like1 t1 join test_like2 t2 on t1.c=t2.d AND t1.c=t2.c union ALL + select t3.a+1,3 from test_like1 t1 join test_like3 t3 on t1.c=t3.d AND t1.c=t3.c) t order by 1,2""" + + qt_multi_differenct_count_condition """select c1,c2 from ( + select t2.a+1 c1,2 c2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a AND t1.a=t2.c union ALL + select t3.a+1 c3,3 c4 from test_like1 t1 join test_like3 t3 on t1.a=t3.c) t order by 1,2""" + qt_no_common_side_project """select c1,c2 from ( + select t2.a+1 c1,t2.c+2 c2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + select t3.a+1,t3.a+2 from test_like1 t1 join test_like3 t3 on t1.a=t3.a) t order by 1,2""" + + qt_common_slot_differnt """select c1,c2 from ( + select t2.a+1 c1,t2.c+2 c2 from test_like1 t1 join test_like2 t2 on t1.a+1=t2.a union ALL + select t3.a+1,t3.c+2 from test_like1 t1 join test_like3 t3 on t1.a=t3.a) t order by 1,2""" + + qt_other_expr_differnt """select c1,c2 from ( + select t2.a+1 c1,t2.c+2 c2 from test_like1 t1 join test_like2 t2 on t1.a=t2.a union ALL + select t3.a+1,t3.c+100 from test_like1 t1 join test_like3 t3 on t1.a=t3.a) t order by 1,2""" + + qt_2_same_tables """select c1,c2 from ( + select t1.a c1,1 c2 from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a + union ALL + select t1.a,t2.a from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a) t order by 1,2""" + + qt_simple_column """select c1,c2 from ( + select t1.a c1,t2.a c2 from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a + union ALL + select t1.a,t2.a from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a) t order by 1,2""" + + qt_func_column """select c1,c2 from ( + select t1.a+1 c1,length(t2.b) c2 from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a + union ALL + select t1.a+1,length(t2.b)+1 from test_like1 t1 inner join test_like3 t2 on t1.a=t2.a) t order by 1,2""" + + qt_other_join_slot_differnt """select c1 from ( + select t1.a+1 c1 from test_like1 t1 inner join test_like2 t2 on t1.a=t2.c + union ALL + select t1.a+1 from test_like1 t1 inner join test_like3 t2 on t1.a=t2.a) t order by 1""" + + qt_join_common_slot_has_expr """select c1 from ( + select t1.a+1 c1 from test_like1 t1 inner join test_like2 t2 on t1.a+1=t2.a + union ALL + select t1.a+1 from test_like1 t1 inner join test_like2 t2 on t1.a+1=t2.a) t order by 1""" + + qt_can_not_transform """select c1,c2 from ( + select t2.c c1,t1.a c2 from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a + union ALL + select t1.a,t2.c from test_like1 t1 inner join test_like2 t2 on t1.a=t2.a) t order by 1,2""" + + qt_other_side_condition_slot_has_expr_do_transform """ + select c1 from ( + select t1.a+1 c1 from test_like4 t1 inner join test_like2 t2 on t1.a=t2.a+2 + union ALL + select t1.a+1 from test_like4 t1 inner join test_like3 t2 on t1.a=t2.a+1) t order by 1""" +} \ No newline at end of file