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 3b39fb3a76e6af..b080df1df4f18a 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 @@ -73,6 +73,7 @@ import org.apache.doris.nereids.rules.rewrite.EliminateNotNull; import org.apache.doris.nereids.rules.rewrite.EliminateNullAwareLeftAntiJoin; import org.apache.doris.nereids.rules.rewrite.EliminateOrderByConstant; +import org.apache.doris.nereids.rules.rewrite.EliminateOrderByKey; import org.apache.doris.nereids.rules.rewrite.EliminateSemiJoin; import org.apache.doris.nereids.rules.rewrite.EliminateSort; import org.apache.doris.nereids.rules.rewrite.EliminateSortUnderSubqueryOrView; @@ -352,7 +353,8 @@ public class Rewriter extends AbstractBatchJobExecutor { ), // this rule should invoke after ColumnPruning custom(RuleType.ELIMINATE_UNNECESSARY_PROJECT, EliminateUnnecessaryProject::new), - + topic("Eliminate Order By Key", + topDown(new EliminateOrderByKey())), topic("Eliminate GroupBy", topDown(new EliminateGroupBy(), new MergeAggregate(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FuncDeps.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FuncDeps.java index 5eebf08ddd766e..dcd9673b5ef757 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FuncDeps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FuncDeps.java @@ -33,9 +33,10 @@ * Function dependence items. */ public class FuncDeps { - static class FuncDepsItem { - final Set determinants; - final Set dependencies; + /**FuncDepsItem*/ + public static class FuncDepsItem { + public final Set determinants; + public final Set dependencies; public FuncDepsItem(Set determinants, Set dependencies) { this.determinants = ImmutableSet.copyOf(determinants); @@ -64,16 +65,21 @@ public int hashCode() { private final Set items; // determinants -> dependencies private final Map, Set>> edges; + // dependencies -> determinants + private final Map, Set>> redges; public FuncDeps() { items = new HashSet<>(); edges = new HashMap<>(); + redges = new HashMap<>(); } public void addFuncItems(Set determinants, Set dependencies) { items.add(new FuncDepsItem(determinants, dependencies)); edges.computeIfAbsent(determinants, k -> new HashSet<>()); edges.get(determinants).add(dependencies); + redges.computeIfAbsent(dependencies, k -> new HashSet<>()); + redges.get(dependencies).add(determinants); } public int size() { @@ -185,6 +191,14 @@ public Set getItems() { return items; } + public Map, Set>> getEdges() { + return edges; + } + + public Map, Set>> getREdges() { + return redges; + } + /** * find the determinants of dependencies */ 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 a3d76d7480b972..254af1bf1f50f9 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 @@ -269,6 +269,7 @@ public enum RuleType { ELIMINATE_JOIN_BY_FK(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY_KEY(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY_KEY_BY_UNIFORM(RuleTypeClass.REWRITE), + ELIMINATE_ORDER_BY_KEY(RuleTypeClass.REWRITE), ELIMINATE_FILTER_GROUP_BY_KEY(RuleTypeClass.REWRITE), ELIMINATE_DEDUP_JOIN_CONDITION(RuleTypeClass.REWRITE), ELIMINATE_NULL_AWARE_LEFT_ANTI_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByKey.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByKey.java new file mode 100644 index 00000000000000..d4ede4deafe075 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByKey.java @@ -0,0 +1,167 @@ +// 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.catalog.Type; +import org.apache.doris.nereids.annotation.DependsRules; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.DataTrait; +import org.apache.doris.nereids.properties.FuncDeps; +import org.apache.doris.nereids.properties.OrderKey; +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.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * 1.eliminate by duplicate + * select a from t1 order by a, a; + * -> + * select a from t1 order by a; + * 2.eliminate by function dependency + * select a from t1 order by a, a+1; + * select a from t1 order by a, abs(a) ; + * select a from t1 where a=c order by a,c + * -> + * select a from t1 order by a; + * 3.eliminate by uniform + * select a,b,c from test where a=1 order by a; + * -> + * select a,b,c from test where a=1; + * */ +@DependsRules({ + NormalizeSort.class, + ExtractAndNormalizeWindowExpression.class, + CheckAndStandardizeWindowFunctionAndFrame.class}) +public class EliminateOrderByKey implements RewriteRuleFactory { + @Override + public List buildRules() { + return ImmutableList.of( + logicalSort(any()).then(EliminateOrderByKey::eliminateSort).toRule(RuleType.ELIMINATE_ORDER_BY_KEY), + logicalWindow(any()).then(EliminateOrderByKey::eliminateWindow) + .toRule(RuleType.ELIMINATE_ORDER_BY_KEY)); + } + + private static Plan eliminateWindow(LogicalWindow window) { + DataTrait dataTrait = window.child().getLogicalProperties().getTrait(); + List newNamedExpressions = new ArrayList<>(); + boolean changed = false; + for (NamedExpression expr : window.getWindowExpressions()) { + Alias alias = (Alias) expr; + WindowExpression windowExpression = (WindowExpression) alias.child(); + List orderExpressions = windowExpression.getOrderKeys(); + if (orderExpressions.stream().anyMatch(( + orderKey -> orderKey.getDataType().isOnlyMetricType()))) { + throw new AnalysisException(Type.OnlyMetricTypeErrorMsg); + } + List orderKeys = new ArrayList<>(); + for (OrderExpression orderExpression : orderExpressions) { + orderKeys.add(orderExpression.getOrderKey()); + } + List retainExpression = eliminate(dataTrait, orderKeys); + if (retainExpression.size() == orderKeys.size()) { + newNamedExpressions.add(expr); + continue; + } + changed = true; + List newOrderExpressions = new ArrayList<>(); + for (OrderKey orderKey : retainExpression) { + newOrderExpressions.add(new OrderExpression(orderKey)); + } + WindowExpression newWindowExpression = windowExpression.withOrderKeys(newOrderExpressions); + newNamedExpressions.add(alias.withChildren(ImmutableList.of(newWindowExpression))); + } + return changed ? window.withExpressionsAndChild(newNamedExpressions, window.child()) : window; + } + + private static Plan eliminateSort(LogicalSort sort) { + DataTrait dataTrait = sort.child().getLogicalProperties().getTrait(); + List retainExpression = eliminate(dataTrait, sort.getOrderKeys()); + if (retainExpression.isEmpty()) { + return sort.child(); + } else if (retainExpression.size() == sort.getOrderKeys().size()) { + return sort; + } + return sort.withOrderKeys(retainExpression); + } + + private static List eliminate(DataTrait dataTrait, List inputOrderKeys) { + Set validSlots = new HashSet<>(); + for (OrderKey inputOrderKey : inputOrderKeys) { + Expression expr = inputOrderKey.getExpr(); + if (!(expr instanceof Slot)) { + return inputOrderKeys; + } + validSlots.add((Slot) expr); + validSlots.addAll(dataTrait.calEqualSet((Slot) expr)); + } + FuncDeps funcDeps = dataTrait.getAllValidFuncDeps(validSlots); + Map, Set>> redges = funcDeps.getREdges(); + + List retainExpression = new ArrayList<>(); + Set orderExprWithEqualSet = new HashSet<>(); + for (OrderKey inputOrderKey : inputOrderKeys) { + Expression expr = inputOrderKey.getExpr(); + // eliminate by duplicate + if (orderExprWithEqualSet.contains(expr)) { + continue; + } + // eliminate by uniform + if (dataTrait.isUniformAndNotNull((Slot) expr)) { + orderExprWithEqualSet.add(expr); + orderExprWithEqualSet.addAll(dataTrait.calEqualSet((Slot) expr)); + continue; + } + // eliminate by fd + Set set = ImmutableSet.of((Slot) expr); + boolean shouldRetain = true; + if (redges.containsKey(set)) { + Set> dominants = redges.get(set); + for (Set dominant : dominants) { + if (orderExprWithEqualSet.containsAll(dominant)) { + shouldRetain = false; + break; + } + } + } + if (!shouldRetain) { + continue; + } + retainExpression.add(inputOrderKey); + orderExprWithEqualSet.add(expr); + orderExprWithEqualSet.addAll(dataTrait.calEqualSet((Slot) expr)); + } + return retainExpression; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java index 2216e58c4fa3b8..4f0492ec63dc8a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.plans.logical; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; @@ -36,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.algebra.OlapScan; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -536,6 +538,12 @@ AGGREGATE KEY (siteid,citycode,username) builder.addUniqueSlot(originalPlan.getLogicalProperties().getTrait()); builder.replaceUniqueBy(constructReplaceMap(mtmv)); } else if (getTable().getKeysType().isAggregationFamily() && !getTable().isRandomDistribution()) { + // When skipDeleteBitmap is set to true, in the unique model, rows that are replaced due to having the same + // unique key will also be read. As a result, the uniqueness of the unique key cannot be guaranteed. + if (ConnectContext.get().getSessionVariable().skipDeleteBitmap + && getTable().getKeysType() == KeysType.UNIQUE_KEYS) { + return; + } ImmutableSet.Builder uniqSlots = ImmutableSet.builderWithExpectedSize(outputSet.size()); for (Slot slot : outputSet) { if (!(slot instanceof SlotReference)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java index 612231340f16ed..305657f820bb13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java @@ -277,9 +277,14 @@ public void computeEqualSet(DataTrait.Builder builder) { public void computeFd(DataTrait.Builder builder) { builder.addFuncDepsDG(child().getLogicalProperties().getTrait()); for (NamedExpression expr : getProjects()) { - if (!expr.isSlot()) { - builder.addDeps(expr.getInputSlots(), ImmutableSet.of(expr.toSlot())); + if (!(expr instanceof Alias)) { + continue; + } + // a+random(1,10) should continue, otherwise the a(determinant), a+random(1,10) (dependency) will be added. + if (expr.containsNonfoldable()) { + continue; } + builder.addDeps(expr.getInputSlots(), ImmutableSet.of(expr.toSlot())); } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByKeyTest.java new file mode 100644 index 00000000000000..05b3529a7ca635 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByKeyTest.java @@ -0,0 +1,512 @@ +// 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.trees.expressions.WindowExpression; +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; + +public class EliminateOrderByKeyTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + createTable("create table test.eliminate_order_by_constant_t(a int null, b int not null," + + "c varchar(10) null, d date, dt datetime, id int)\n" + + "distributed by hash(a) properties(\"replication_num\"=\"1\");"); + createTable("create table test.eliminate_order_by_constant_t2(a int, b int, c int, d int) " + + "distributed by hash(a) properties(\"replication_num\"=\"1\");"); + createTable("create table test.test_unique_order_by2(a int not null, b int not null, c int, d int) " + + "unique key(a,b) distributed by hash(a) properties('replication_num'='1');"); + connectContext.setDatabase("test"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + void testEliminateByCompositeKeys() { + PlanChecker.from(connectContext) + .analyze("select * from test_unique_order_by2 order by a,'abc',d,b,d,c") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 3)); + } + + @Test + void testEliminateByFd() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from test.eliminate_order_by_constant_t order by a,abs(a),a+1") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByFdAndDup() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a,abs(a),a+1") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByFdTopN() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a+1 limit 5") + .rewrite() + .printlnTree() + .matches(logicalTopN().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByFdAndDupTopN() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a,abs(a),a+1,id limit 5") + .rewrite() + .printlnTree() + .matches(logicalTopN().when(sort -> sort.getOrderKeys().size() == 2)); + } + + @Test + void testEliminateByDup() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,a") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByDupExpr() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a+1,a+1") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByDupTopN() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,a limit 5") + .rewrite() + .printlnTree() + .matches(logicalTopN().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByUniformPredicate() { + PlanChecker.from(connectContext) + .analyze("select 1 as c1,a from eliminate_order_by_constant_t where a=1 order by a") + .rewrite() + .printlnTree() + .nonMatch(logicalSort()); + } + + @Test + void testEliminateByUniformWithAgg() { + PlanChecker.from(connectContext) + .analyze("select 1 as c1,a from eliminate_order_by_constant_t where a=1 group by c1,a order by a") + .rewrite() + .printlnTree() + .nonMatch(logicalSort()); + } + + @Test + void testEliminateByUniformMultiKey() { + PlanChecker.from(connectContext) + .analyze("select 1 as c1,a,b,c from eliminate_order_by_constant_t where a=1 order by a,'abc',b,c1") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEliminateByUniformTopN() { + PlanChecker.from(connectContext) + .analyze("select 1 as c1,a,b,c from eliminate_order_by_constant_t where a=1 order by a,'abc',c1 limit 5") + .rewrite() + .printlnTree() + .nonMatch(logicalTopN()); + } + + @Test + void notEliminateNonDeterministic() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,a+random(1,10)") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + } + + @Test + void testMultiColumnFd() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,b,a+b") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + } + + @Test + void testMultiColumnFdWithOtherInMiddle() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t order by a,c,b,a+b") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 3)); + } + + @Test + void testWindowDup() { + PlanChecker.from(connectContext) + .analyze("select sum(a) over (partition by a order by a,a) from eliminate_order_by_constant_t") + .rewrite() + .printlnTree() + .matches(logicalWindow() + .when(window -> ((WindowExpression) window.getWindowExpressions().get(0).child(0)) + .getOrderKeys().size() == 1)); + } + + @Test + void testWindowFd() { + PlanChecker.from(connectContext) + .analyze("select sum(a) over (partition by a order by a,a+1,abs(a),1-a,b) from eliminate_order_by_constant_t") + .rewrite() + .printlnTree() + .matches(logicalWindow() + .when(window -> ((WindowExpression) window.getWindowExpressions().get(0).child(0)) + .getOrderKeys().size() == 2)); + } + + @Test + void testWindowUniform() { + PlanChecker.from(connectContext) + .analyze("select sum(a) over (partition by a order by b) from eliminate_order_by_constant_t where b=100") + .rewrite() + .printlnTree() + .matches(logicalWindow() + .when(window -> ((WindowExpression) window.getWindowExpressions().get(0).child(0)) + .getOrderKeys().isEmpty())); + } + + @Test + void testWindowMulti() { + PlanChecker.from(connectContext) + .analyze("select sum(a) over (partition by a order by a,a+1,abs(a),1-a,b)" + + ", max(a) over (partition by a order by b,b+1,b,abs(b)) from eliminate_order_by_constant_t") + .rewrite() + .printlnTree() + .matches(logicalWindow() + .when(window -> ((WindowExpression) window.getWindowExpressions().get(0).child(0)) + .getOrderKeys().size() == 2 + && ((WindowExpression) window.getWindowExpressions().get(1).child(0)) + .getOrderKeys().size() == 1)); + } + + @Test + void testWindowMultiDesc() { + PlanChecker.from(connectContext) + .analyze("select sum(a) over (partition by a order by a desc,a+1 asc,abs(a) desc,1-a,b)," + + "max(a) over (partition by a order by b desc,b+1 desc,b asc,abs(b) desc) " + + "from eliminate_order_by_constant_t") + .rewrite() + .printlnTree() + .matches(logicalWindow() + .when(window -> ((WindowExpression) window.getWindowExpressions().get(0).child(0)) + .getOrderKeys().size() == 2 + && ((WindowExpression) window.getWindowExpressions().get(1).child(0)) + .getOrderKeys().size() == 1)); + } + + @Test + void testEqualSet() { + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t where a=id order by a,id") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t where a=id order by id,a") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t where a=id and a=b order by id,a,b") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testEqualSetAndFd() { + PlanChecker.from(connectContext) + .analyze("select a,b,c,d,dt from eliminate_order_by_constant_t where a=b order by a,a+b, b ") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testUniformAndFd() { + PlanChecker.from(connectContext) + .analyze("select a,b from eliminate_order_by_constant_t where a=b and a=1 order by a,b,a+b") + .rewrite() + .printlnTree() + .nonMatch(logicalSort()); + } + + @Test + void testUniformAndFd2() { + PlanChecker.from(connectContext) + .analyze("select a from eliminate_order_by_constant_t where a=1 order by a,b,b,abs(a),a ") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + @Test + void testFdValidSlotsAddEqualSet() { + PlanChecker.from(connectContext) + .analyze("select c,d,a,a+100,b+a+100,b from eliminate_order_by_constant_t where b=a order by c,d,a,a+100,b+a+100") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 3)); + + // TODO After removing b from the projection column, b+a+100 cannot be deleted from sort. + // This is because the equal set after the project does not output a=b because the b projection column does not output it. + PlanChecker.from(connectContext) + .analyze("select c,d,a,a+100,b+a+100 from eliminate_order_by_constant_t where b=a order by c,d,a,a+100,b+a+100") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 4)); + } + + @Test + void testEqualSetUniform() { + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t2 where b=a and a=c and d=1 order by d,a,b,c,c,b,a,d,d") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t2 where b=a and a=d and a=c and d=1 order by d,a,b,c,c,b,a,d,d") + .rewrite() + .printlnTree() + .nonMatch(logicalSort()); + } + + @Test + void testEqualSetUniformFd() { + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t2 where b=a and a=d and a=c and d=1 order by d,a,a+1,b+1,c+1,c,b,a,d,d") + .rewrite() + .printlnTree() + .nonMatch(logicalSort()); + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t2 where d=1 order by d,a,b,c,d+b+a-100,d+b+a,b,a,d,d") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 3)); + } + + @Test + void testEqualSetFd() { + PlanChecker.from(connectContext) + .analyze("select * from eliminate_order_by_constant_t2 where d=b and a=d order by a,c,d+b+a,b,a,d,d+1,abs(d)") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + } + + @Test + void testInnerJoinEqual() { + PlanChecker.from(connectContext) + .analyze("select t1.a, t2.a,t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 " + + "inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a " + + "order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 4)); + } + + @Test + void testLeftJoinEqual() { + // t1.a=t2.a but should not eliminate + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 " + + "left outer join eliminate_order_by_constant_t2 t2 on t1.a = t2.a " + + "order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 5)); + // left is converted to inner join + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 " + + "left outer join eliminate_order_by_constant_t2 t2 on t1.a = t2.a where t2.c=100 " + + "order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 3)); + } + + @Test + void testRightJoinEqual() { + // right join and has uniform + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1" + + " right outer join eliminate_order_by_constant_t2 t2 on t1.a = t2.a where t2.a=1 " + + "order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 4)); + } + + @Test + void testSemiAntiJoin() { + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b from eliminate_order_by_constant_t t1 " + + "left semi join eliminate_order_by_constant_t2 t2 on t1.a = t2.a order by t1.a, t1.b, t1.a+1,t1.a+t1.b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + PlanChecker.from(connectContext) + .analyze("select t2.a, t2.b from eliminate_order_by_constant_t t1 " + + " right semi join eliminate_order_by_constant_t2 t2 on t1.a = t2.a " + + " order by t2.a, t2.b, t2.a+1,t2.a+t2.b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b " + + "from eliminate_order_by_constant_t t1 " + + "left anti join eliminate_order_by_constant_t2 t2 on t1.a = t2.a " + + "order by t1.a, t1.b, t1.a+1,t1.a+t1.b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + PlanChecker.from(connectContext) + .analyze("select t2.a, t2.b" + + " from eliminate_order_by_constant_t t1" + + " right anti join eliminate_order_by_constant_t2 t2 on t1.a = t2.a" + + " order by t2.a, t2.b, t2.a+1,t2.a+t2.b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + } + + @Test + void testAgg() { + PlanChecker.from(connectContext) + .analyze("select a, count(b) as cnt " + + "from eliminate_order_by_constant_t2 " + + "group by a " + + "order by a, cnt, a,a+cnt,a+100;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + + PlanChecker.from(connectContext) + .analyze("select a, b, count(c) as cnt" + + " from eliminate_order_by_constant_t2" + + " group by cube(a, b)" + + " order by a, b, cnt, a, b+1;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 3)); + } + + @Test + void testJoinWindow() { + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b, t2.c, t2.d, t2.a," + + " row_number() over (partition by t1.a order by t1.b) as rn" + + " from eliminate_order_by_constant_t t1" + + " inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a" + + " order by t1.a, t2.a,t2.c, t1.b, t2.d, abs(t1.a), abs(t2.a), t2.c,rn,rn+100;;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 5)); + PlanChecker.from(connectContext) + .analyze("select a, b, count(c) as cnt, " + + " row_number() over (partition by a order by b) as rn " + + " from eliminate_order_by_constant_t2 " + + " group by a, b " + + " order by a, b, cnt, a+100, b, rn, rn+cnt, abs(rn+cnt);") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 4)); + } + + @Test + void testAggWindowJoin() { + PlanChecker.from(connectContext) + .analyze("select t1.a, t1.b, count(t2.c) as cnt," + + " row_number() over (partition by t1.a order by t1.b) as rn" + + " from eliminate_order_by_constant_t t1" + + " inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a" + + " group by t1.a, t1.b ,t2.a" + + " order by t1.a,t2.a,t1.b, cnt, -t1.a, -t1.b-1000,rn, cnt, rn+111;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 4)); + } + + @Test + void testUnionAll() { + PlanChecker.from(connectContext) + .analyze("select * from (" + + " select a, b from eliminate_order_by_constant_t2 " + + " union all " + + " select a, b from eliminate_order_by_constant_t ) t " + + " order by a, b, abs(a),abs(a)+b,a+b,a,b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + PlanChecker.from(connectContext) + .analyze("select * from (" + + " select a, b from eliminate_order_by_constant_t2 " + + " union all " + + " select a, b from eliminate_order_by_constant_t ) t " + + " where a=b order by a, b, abs(a),abs(a)+b,a+b,a,b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 1)); + } + + // TODO LogicalUnion compute uniform can expand support scope when each child has same uniform output + // and corresponding same position + @Test + void testUnionJoin() { + PlanChecker.from(connectContext) + .analyze("select * from (select t1.a, t1.b " + + " from eliminate_order_by_constant_t t1 " + + " inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a " + + " union " + + " select t1.a, t1.b " + + " from eliminate_order_by_constant_t t1 " + + " left join eliminate_order_by_constant_t2 t2 on t1.a = t2.a ) t" + + " where a=1" + + " order by a, b, a+100,abs(a)+b;") + .rewrite() + .printlnTree() + .matches(logicalSort().when(sort -> sort.getOrderKeys().size() == 2)); + } +} diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index 10a01a67cc86f8..73b015ed086737 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -1226,8 +1226,8 @@ SyntaxError: -- !shape -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalTopN[LOCAL_SORT] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((dwd_tracking_sensor_init_tmp_ymd.dt = dw_user_b2c_tracking_info_tmp_ymd.dt) and (dwd_tracking_sensor_init_tmp_ymd.guid = dw_user_b2c_tracking_info_tmp_ymd.guid)) otherCondition=((dwd_tracking_sensor_init_tmp_ymd.dt >= substring(first_visit_time, 1, 10))) diff --git a/regression-test/data/nereids_rules_p0/eliminate_order_by_key/eliminate_order_by_key.out b/regression-test/data/nereids_rules_p0/eliminate_order_by_key/eliminate_order_by_key.out new file mode 100644 index 00000000000000..5ba1fcce5523d6 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eliminate_order_by_key/eliminate_order_by_key.out @@ -0,0 +1,402 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !predicate -- +1 1 +1 1 + +-- !predicate_order_by_two -- +1 1 +1 1 + +-- !with_group_by -- +1 1 + +-- !predicate_multi_other -- +1 1 100 apple +1 1 100 apple + +-- !with_group_by_shape -- +PhysicalResultSink +--hashAgg[LOCAL] +----filter((eliminate_order_by_constant_t.a = 1)) +------PhysicalOlapScan[eliminate_order_by_constant_t] + +-- !fd -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !fd_duplicate -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !fd_topn -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 + +-- !fd_duplicate_topn -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 + +-- !fd_multi_column -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !fd_desc -- +6 106 fig 2023-01-07 2023-01-07T16:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 + +-- !fd_multi_column_desc -- +\N 107 grape 2023-01-08 2023-01-08T17:00 +\N 103 date 2023-01-04 2023-01-04T13:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !fd_multi_column_desc_with_other_in_middle -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !equalset_fd -- + +-- !uniform_fd -- + +-- !fd_valid_slot_add_equalset -- + +-- !dup_shape -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !dup_expr_shape -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !dup_window -- +\N +\N +2 +2 +2 +4 +6 +6 +6 +10 +10 + +-- !fd_window -- +\N +\N +2 +2 +2 +4 +6 +6 +6 +10 +10 + +-- !uniform_window -- +2 +2 + +-- !uniform_window -- +apple +apple + +-- !multi_window -- +\N \N +\N \N +2 1 +2 1 +2 2 +4 4 +6 3 +6 3 +6 6 +10 5 +10 5 + +-- !multi_window_desc -- +\N \N +\N \N +2 1 +2 1 +2 2 +4 4 +6 3 +6 3 +6 6 +10 5 +10 5 + +-- !equal_set_uniform -- + +-- !equal_set_uniform2 -- + +-- !equal_set_uniform_fd -- + +-- !fd_uniform -- + +-- !equalset_fd -- + +-- !join_inner_order_by -- +1 100 3 4 +1 100 3 4 +2 101 3 5 +3 102 5 6 +3 102 5 6 +4 104 6 7 +5 105 7 8 +5 105 7 8 +6 106 8 9 + +-- !join_left_outer_order_by -- +\N 103 \N \N +\N 107 \N \N +1 100 3 4 +1 100 3 4 +2 101 3 5 +3 102 5 6 +3 102 5 6 +4 104 6 7 +5 105 7 8 +5 105 7 8 +6 106 8 9 + +-- !join_right_outer_order_by -- +\N \N 9 10 +\N \N 10 11 +\N \N 11 12 +\N \N 12 13 +1 100 3 4 +1 100 3 4 +2 101 3 5 +3 102 5 6 +3 102 5 6 +4 104 6 7 +5 105 7 8 +5 105 7 8 +6 106 8 9 + +-- !join_right_outer_order_by_predicate -- +1 100 3 4 +1 100 3 4 + +-- !join_left_semi_order_by -- +1 100 +1 100 +2 101 +3 102 +3 102 +4 104 +5 105 +5 105 +6 106 + +-- !join_right_semi_order_by -- +1 2 +2 3 +3 4 +4 5 +5 6 +6 7 + +-- !join_left_anti_order_by -- +\N 103 +\N 107 + +-- !join_right_anti_order_by -- +7 8 +8 9 +9 10 +10 11 + +-- !agg_order_by -- +1 1 +2 1 +3 1 +4 1 +5 1 +6 1 +7 1 +8 1 +9 1 +10 1 + +-- !agg_grouping_order_by -- +\N \N 10 +\N 2 1 +\N 3 1 +\N 4 1 +\N 5 1 +\N 6 1 +\N 7 1 +\N 8 1 +\N 9 1 +\N 10 1 +\N 11 1 +1 \N 1 +1 2 1 +2 \N 1 +2 3 1 +3 \N 1 +3 4 1 +4 \N 1 +4 5 1 +5 \N 1 +5 6 1 +6 \N 1 +6 7 1 +7 \N 1 +7 8 1 +8 \N 1 +8 9 1 +9 \N 1 +9 10 1 +10 \N 1 +10 11 1 + +-- !join_window_order_by -- +1 100 3 4 1 1 +1 100 3 4 1 2 +2 101 3 5 2 1 +3 102 5 6 3 1 +3 102 5 6 3 2 +4 104 6 7 4 1 +5 105 7 8 5 1 +5 105 7 8 5 2 +6 106 8 9 6 1 + +-- !agg_window_order_by -- +1 2 1 1 +2 3 1 1 +3 4 1 1 +4 5 1 1 +5 6 1 1 +6 7 1 1 +7 8 1 1 +8 9 1 1 +9 10 1 1 +10 11 1 1 + +-- !join_agg_window_order_by -- +1 100 2 1 +2 101 1 1 +3 102 2 1 +4 104 1 1 +5 105 2 1 +6 106 1 1 + +-- !union_all_order_by -- +\N 103 +\N 107 +1 2 +1 100 +1 100 +2 3 +2 101 +3 4 +3 102 +3 102 +4 5 +4 104 +5 6 +5 105 +5 105 +6 7 +6 106 +7 8 +8 9 +9 10 +10 11 + +-- !union_join_order_by -- +1 100 + +-- !composite_key -- +1 2 3 4 +2 3 3 5 +3 4 5 6 +4 5 6 7 +5 6 7 8 +6 7 8 9 +7 8 9 10 +8 9 10 11 +9 10 11 12 +10 11 12 13 + diff --git a/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out b/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out index 980ad322519564..409fe3c15667f0 100644 --- a/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out +++ b/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out @@ -195,18 +195,18 @@ PhysicalResultSink ----------PhysicalOlapScan[t1] -- !limit_sort_filter -- -PhysicalDeferMaterializeResultSink ---PhysicalDeferMaterializeTopN -----PhysicalDeferMaterializeTopN +PhysicalResultSink +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] ------filter((t1.id = 1)) ---------PhysicalDeferMaterializeOlapScan[t1] +--------PhysicalOlapScan[t1] -- !limit_offset_sort_filter -- -PhysicalDeferMaterializeResultSink ---PhysicalDeferMaterializeTopN -----PhysicalDeferMaterializeTopN +PhysicalResultSink +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] ------filter((t1.id = 1)) ---------PhysicalDeferMaterializeOlapScan[t1] +--------PhysicalOlapScan[t1] -- !limit_subquery_order_by_inside_limit_outside -- PhysicalResultSink @@ -275,18 +275,18 @@ PhysicalResultSink ----------PhysicalOlapScan[t1] -- !limit_filter -- -PhysicalDeferMaterializeResultSink ---PhysicalDeferMaterializeTopN -----PhysicalDeferMaterializeTopN +PhysicalResultSink +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] ------filter((t1.id = 1)) ---------PhysicalDeferMaterializeOlapScan[t1] +--------PhysicalOlapScan[t1] -- !limit_offset_filter -- -PhysicalDeferMaterializeResultSink ---PhysicalDeferMaterializeTopN -----PhysicalDeferMaterializeTopN +PhysicalResultSink +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] ------filter((t1.id = 1)) ---------PhysicalDeferMaterializeOlapScan[t1] +--------PhysicalOlapScan[t1] -- !limit_project_filter -- PhysicalDeferMaterializeResultSink diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query16.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query16.out index c6e88456a7e402..02559f87cb35c3 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query16.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query16.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[cs_ship_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] ---------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[cs_ship_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF0 cs_order_number->[cs_order_number] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_returns] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF0 cs_order_number->[cs_order_number] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'WV')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) -----------------------PhysicalOlapScan[call_center] +------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'WV')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query94.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query94.out index 75a20cb4e35006..d26128a78f3b6a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query94.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] ---------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF0 ws_order_number->[ws_order_number] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF0 ws_order_number->[ws_order_number] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_returns] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'OK')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((web_site.web_company_name = 'pri')) -----------------------PhysicalOlapScan[web_site] +------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'OK')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((web_site.web_company_name = 'pri')) +------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query95.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query95.out index fce09b1b6046e9..4435e9fb23dcd5 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query95.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF7 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF6 web_site_sk->[ws_web_site_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[ws_ship_addr_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[ws_order_number] -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF2 ws_order_number->[wr_order_number];RF7 ws_order_number->[ws_order_number,ws_order_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() -------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 RF6 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((customer_address.ca_state = 'NC')) -----------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------filter((web_site.web_company_name = 'pri')) -------------------------PhysicalOlapScan[web_site] +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF6 web_site_sk->[ws_web_site_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[ws_ship_addr_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[ws_order_number] +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 +--------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF2 ws_order_number->[wr_order_number];RF7 ws_order_number->[ws_order_number,ws_order_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 RF6 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((customer_address.ca_state = 'NC')) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------filter((web_site.web_company_name = 'pri')) +--------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query96.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query96.out index 11217d6de3e01b..9028fb34a8d1d3 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query96.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query96.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ss_sold_time_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((household_demographics.hd_dep_count = 3)) ---------------------------PhysicalOlapScan[household_demographics] -------------------PhysicalProject ---------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) -----------------------PhysicalOlapScan[time_dim] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ss_sold_time_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((household_demographics.hd_dep_count = 3)) +----------------------------PhysicalOlapScan[household_demographics] +--------------------PhysicalProject +----------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) +------------------------PhysicalOlapScan[time_dim] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query16.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query16.out index c6e88456a7e402..02559f87cb35c3 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query16.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query16.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[cs_ship_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] ---------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[cs_ship_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF0 cs_order_number->[cs_order_number] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_returns] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF0 cs_order_number->[cs_order_number] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'WV')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) -----------------------PhysicalOlapScan[call_center] +------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'WV')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query94.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query94.out index 75a20cb4e35006..d26128a78f3b6a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query94.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] ---------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF0 ws_order_number->[ws_order_number] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF0 ws_order_number->[ws_order_number] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_returns] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'OK')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((web_site.web_company_name = 'pri')) -----------------------PhysicalOlapScan[web_site] +------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'OK')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((web_site.web_company_name = 'pri')) +------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query95.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query95.out index aaaba243279489..253f14febe60d7 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query95.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF14 RF15 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF12 web_site_sk->[ws_web_site_sk];RF13 web_site_sk->[ws_web_site_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF10 ca_address_sk->[ws_ship_addr_sk];RF11 ca_address_sk->[ws_ship_addr_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_ship_date_sk];RF9 d_date_sk->[ws_ship_date_sk] -----------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[ws_order_number];RF7 ws_order_number->[ws_order_number] -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 -------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF4 ws_order_number->[wr_order_number];RF5 ws_order_number->[wr_order_number];RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF2 wr_order_number->[ws_order_number];RF3 wr_order_number->[ws_order_number] -------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 RF10 RF11 RF12 RF13 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((customer_address.ca_state = 'NC')) -----------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------filter((web_site.web_company_name = 'pri')) -------------------------PhysicalOlapScan[web_site] +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF12 web_site_sk->[ws_web_site_sk];RF13 web_site_sk->[ws_web_site_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF10 ca_address_sk->[ws_ship_addr_sk];RF11 ca_address_sk->[ws_ship_addr_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ws_ship_date_sk];RF9 d_date_sk->[ws_ship_date_sk] +------------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[ws_order_number];RF7 ws_order_number->[ws_order_number] +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 +--------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF4 ws_order_number->[wr_order_number];RF5 ws_order_number->[wr_order_number];RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF2 wr_order_number->[ws_order_number];RF3 wr_order_number->[ws_order_number] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF4 RF5 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 RF10 RF11 RF12 RF13 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((customer_address.ca_state = 'NC')) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------filter((web_site.web_company_name = 'pri')) +--------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query96.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query96.out index 11217d6de3e01b..9028fb34a8d1d3 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query96.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query96.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ss_sold_time_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((household_demographics.hd_dep_count = 3)) ---------------------------PhysicalOlapScan[household_demographics] -------------------PhysicalProject ---------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) -----------------------PhysicalOlapScan[time_dim] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ss_sold_time_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((household_demographics.hd_dep_count = 3)) +----------------------------PhysicalOlapScan[household_demographics] +--------------------PhysicalProject +----------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) +------------------------PhysicalOlapScan[time_dim] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query16.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query16.out index 1733e793f1f9b0..b1b5037f23d714 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query16.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query16.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF3 cs_order_number->[cs_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF3 cs_order_number->[cs_order_number] +--------------------PhysicalProject +----------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'WV')) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) ---------------------------PhysicalOlapScan[call_center] +----------------------------------filter((customer_address.ca_state = 'WV')) +------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +----------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query94.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query94.out index 6a25137bf51fcf..df854654271e34 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query94.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF3 ws_order_number->[ws_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF3 ws_order_number->[ws_order_number] +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_returns] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_returns] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'OK')) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((web_site.web_company_name = 'pri')) ---------------------------PhysicalOlapScan[web_site] +----------------------------------filter((customer_address.ca_state = 'OK')) +------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((web_site.web_company_name = 'pri')) +----------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query95.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query95.out index ad87360cc8a6ed..623fe7152eccbb 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query95.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF7 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'NC')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) ---------------------------------PhysicalOlapScan[date_dim] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 ------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +--------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'NC')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query96.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query96.out index c50ffa373c8150..a7d441a3e3d0ee 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query96.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query96.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalProject ---------------------filter((household_demographics.hd_dep_count = 3)) -----------------------PhysicalOlapScan[household_demographics] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) +----------------------------PhysicalOlapScan[time_dim] +--------------------PhysicalProject +----------------------filter((household_demographics.hd_dep_count = 3)) +------------------------PhysicalOlapScan[household_demographics] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query16.out b/regression-test/data/shape_check/tpcds_sf100/shape/query16.out index 1733e793f1f9b0..b1b5037f23d714 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query16.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query16.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF3 cs_order_number->[cs_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF3 cs_order_number->[cs_order_number] +--------------------PhysicalProject +----------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'WV')) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) ---------------------------PhysicalOlapScan[call_center] +----------------------------------filter((customer_address.ca_state = 'WV')) +------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) +----------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query94.out b/regression-test/data/shape_check/tpcds_sf100/shape/query94.out index 6a25137bf51fcf..df854654271e34 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query94.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF3 ws_order_number->[ws_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF3 ws_order_number->[ws_order_number] +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_returns] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_returns] -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_state = 'OK')) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((web_site.web_company_name = 'pri')) ---------------------------PhysicalOlapScan[web_site] +----------------------------------filter((customer_address.ca_state = 'OK')) +------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2000-04-01') and (date_dim.d_date >= '2000-02-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((web_site.web_company_name = 'pri')) +----------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query95.out b/regression-test/data/shape_check/tpcds_sf100/shape/query95.out index 269d330c090c56..2e4d4278eaaed8 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query95.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF14 RF15 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF12 ws_order_number->[wr_order_number,ws_order_number];RF13 ws_order_number->[wr_order_number,ws_order_number] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF10 wr_order_number->[ws_order_number];RF11 wr_order_number->[ws_order_number] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 RF11 RF12 RF13 -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_returns] apply RFs: RF12 RF13 -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF12 ws_order_number->[wr_order_number,ws_order_number];RF13 ws_order_number->[wr_order_number,ws_order_number] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF6 web_site_sk->[ws_web_site_sk];RF7 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_ship_date_sk];RF5 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk];RF3 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'NC')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) ---------------------------------PhysicalOlapScan[date_dim] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF10 wr_order_number->[ws_order_number];RF11 wr_order_number->[ws_order_number] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 RF11 RF12 RF13 ------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +--------------------------PhysicalOlapScan[web_returns] apply RFs: RF12 RF13 +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF14 ws_order_number->[ws_order_number,ws_order_number];RF15 ws_order_number->[ws_order_number,ws_order_number] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF6 web_site_sk->[ws_web_site_sk];RF7 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_ship_date_sk];RF5 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[ws_ship_addr_sk];RF3 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'NC')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '1999-04-02') and (date_dim.d_date >= '1999-02-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query96.out b/regression-test/data/shape_check/tpcds_sf100/shape/query96.out index c50ffa373c8150..a7d441a3e3d0ee 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query96.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query96.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalProject ---------------------filter((household_demographics.hd_dep_count = 3)) -----------------------PhysicalOlapScan[household_demographics] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) +----------------------------PhysicalOlapScan[time_dim] +--------------------PhysicalProject +----------------------filter((household_demographics.hd_dep_count = 3)) +------------------------PhysicalOlapScan[household_demographics] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query95.out b/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query95.out index 21c6fa60d37b75..84bdd6cf3d8f4a 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query95.out +++ b/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF7 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 wr_order_number->[ws_order_number] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'VA')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) ---------------------------------PhysicalOlapScan[date_dim] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 wr_order_number->[ws_order_number] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 ------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +--------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'VA')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query16.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query16.out index 247f0f8c77738f..391b7d7cc41ba7 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query16.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query16.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF4 cs_order_number->[cs_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 -------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF3 cs_order_number->[cr_order_number] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF4 cs_order_number->[cs_order_number] --------------------PhysicalProject -----------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'PA')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((call_center.cc_county = 'Williamson County')) -----------------------------PhysicalOlapScan[call_center] +----------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF3 cs_order_number->[cr_order_number] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'PA')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((call_center.cc_county = 'Williamson County')) +------------------------------PhysicalOlapScan[call_center] Hint log: Used: leading(catalog_sales { cs1 customer_address date_dim call_center } ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query94.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query94.out index 0f35f2dc29e44e..bd89dd58effa9b 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query94.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF4 ws_order_number->[ws_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF4 ws_order_number->[ws_order_number] --------------------PhysicalProject -----------------------PhysicalOlapScan[web_returns] apply RFs: RF3 ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2002-06-30') and (date_dim.d_date >= '2002-05-01')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +----------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2002-06-30') and (date_dim.d_date >= '2002-05-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query95.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query95.out index 21c6fa60d37b75..84bdd6cf3d8f4a 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query95.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF7 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 wr_order_number->[ws_order_number] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'VA')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) ---------------------------------PhysicalOlapScan[date_dim] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 wr_order_number->[ws_order_number] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 ------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +--------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'VA')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query96.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query96.out index 188b18e2bc0799..465d33cd6537d2 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query96.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query96.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalProject ---------------------filter((household_demographics.hd_dep_count = 0)) -----------------------PhysicalOlapScan[household_demographics] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) +----------------------------PhysicalOlapScan[time_dim] +--------------------PhysicalProject +----------------------filter((household_demographics.hd_dep_count = 0)) +------------------------PhysicalOlapScan[household_demographics] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] Hint log: Used: leading(store_sales time_dim household_demographics store ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query16.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query16.out index db5bf9b39a05ec..21dcd5cf6df0ae 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query16.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query16.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF4 cs_order_number->[cs_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 -------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF3 cs_order_number->[cr_order_number] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF4 cs_order_number->[cs_order_number] --------------------PhysicalProject -----------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'PA')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((call_center.cc_county = 'Williamson County')) -----------------------------PhysicalOlapScan[call_center] +----------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() build RFs:RF3 cs_order_number->[cr_order_number] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[cs_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'PA')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((call_center.cc_county = 'Williamson County')) +------------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query94.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query94.out index 0f35f2dc29e44e..bd89dd58effa9b 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query94.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF4 ws_order_number->[ws_order_number] -------------------PhysicalProject ---------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF4 ws_order_number->[ws_order_number] --------------------PhysicalProject -----------------------PhysicalOlapScan[web_returns] apply RFs: RF3 ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2002-06-30') and (date_dim.d_date >= '2002-05-01')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +----------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() build RFs:RF3 ws_order_number->[wr_order_number] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_returns] apply RFs: RF3 +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2002-06-30') and (date_dim.d_date >= '2002-05-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query95.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query95.out index 21c6fa60d37b75..84bdd6cf3d8f4a 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query95.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF7 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 wr_order_number->[ws_order_number] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 -------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------hashJoin[RIGHT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 ws_order_number->[wr_order_number,ws_order_number] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_state = 'VA')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) ---------------------------------PhysicalOlapScan[date_dim] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 wr_order_number->[ws_order_number] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 ------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] +--------------------------PhysicalOlapScan[web_returns] apply RFs: RF6 +--------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 ws_order_number->[ws_order_number,ws_order_number] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF3 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_state = 'VA')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= '2001-05-31') and (date_dim.d_date >= '2001-04-01')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query96.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query96.out index b4b739a9bf444f..7e3d08cf2617a5 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query96.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query96.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalProject ---------------------filter((household_demographics.hd_dep_count = 0)) -----------------------PhysicalOlapScan[household_demographics] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF0 t_time_sk->[ss_sold_time_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) +----------------------------PhysicalOlapScan[time_dim] +--------------------PhysicalProject +----------------------filter((household_demographics.hd_dep_count = 0)) +------------------------PhysicalOlapScan[household_demographics] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query16.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query16.out index e9d35f9e131c3e..ff30193149f0de 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query16.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query16.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[cs_ship_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] ---------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF3 cc_call_center_sk->[cs_call_center_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[cs_ship_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_ship_date_sk] +----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((cs1.cs_order_number = cr1.cr_order_number)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF0 cs_order_number->[cs_order_number] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_returns] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((cs1.cs_order_number = cs2.cs_order_number)) otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) build RFs:RF0 cs_order_number->[cs_order_number] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1999-05-31') and (date_dim.d_date >= '1999-04-01')) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IL')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter(cc_county IN ('Bronx County', 'Maverick County', 'Mesa County', 'Raleigh County', 'Richland County')) -----------------------PhysicalOlapScan[call_center] +------------------------------filter((date_dim.d_date <= '1999-05-31') and (date_dim.d_date >= '1999-04-01')) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'IL')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter(cc_county IN ('Bronx County', 'Maverick County', 'Mesa County', 'Raleigh County', 'Richland County')) +------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query31.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query31.out index 177a039758745c..f6b0f237d63c68 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query31.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query31.out @@ -33,33 +33,30 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((ws.d_year = 1999) and d_qoy IN (1, 2, 3)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalQuickSort[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalQuickSort[LOCAL_SORT] -------------PhysicalProject ---------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ws1.ca_county = ws3.ca_county)) otherCondition=((if((web_sales > 0.00), (cast(web_sales as DECIMALV3(38, 8)) / web_sales), NULL) > if((store_sales > 0.00), (cast(store_sales as DECIMALV3(38, 8)) / store_sales), NULL))) build RFs:RF8 ca_county->[ca_county,ca_county,ca_county,ca_county] +------PhysicalProject +--------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ws1.ca_county = ws3.ca_county)) otherCondition=((if((web_sales > 0.00), (cast(web_sales as DECIMALV3(38, 8)) / web_sales), NULL) > if((store_sales > 0.00), (cast(store_sales as DECIMALV3(38, 8)) / store_sales), NULL))) build RFs:RF8 ca_county->[ca_county,ca_county,ca_county,ca_county] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ws1.ca_county = ws2.ca_county)) otherCondition=((if((web_sales > 0.00), (cast(web_sales as DECIMALV3(38, 8)) / web_sales), NULL) > if((store_sales > 0.00), (cast(store_sales as DECIMALV3(38, 8)) / store_sales), NULL))) build RFs:RF7 ca_county->[ca_county,ca_county,ca_county] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ss1.ca_county = ws1.ca_county)) otherCondition=() build RFs:RF6 ca_county->[ca_county,ca_county] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ws1.ca_county = ws2.ca_county)) otherCondition=((if((web_sales > 0.00), (cast(web_sales as DECIMALV3(38, 8)) / web_sales), NULL) > if((store_sales > 0.00), (cast(store_sales as DECIMALV3(38, 8)) / store_sales), NULL))) build RFs:RF7 ca_county->[ca_county,ca_county,ca_county] ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ss1.ca_county = ws1.ca_county)) otherCondition=() build RFs:RF6 ca_county->[ca_county,ca_county] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ss2.ca_county = ss3.ca_county)) otherCondition=() build RFs:RF5 ca_county->[ca_county,ca_county] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ss1.ca_county = ss2.ca_county)) otherCondition=() build RFs:RF4 ca_county->[ca_county] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ss2.ca_county = ss3.ca_county)) otherCondition=() build RFs:RF5 ca_county->[ca_county,ca_county] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ss1.ca_county = ss2.ca_county)) otherCondition=() build RFs:RF4 ca_county->[ca_county] -----------------------------PhysicalProject -------------------------------filter((ss1.d_qoy = 1) and (ss1.d_year = 1999)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 RF7 RF8 -----------------------------PhysicalProject -------------------------------filter((ss2.d_qoy = 2) and (ss2.d_year = 1999)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 RF7 RF8 ---------------------------PhysicalProject -----------------------------filter((ss3.d_qoy = 3) and (ss3.d_year = 1999)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------filter((ss1.d_qoy = 1) and (ss1.d_year = 1999)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 RF7 RF8 ----------------------PhysicalProject -------------------------filter((ws1.d_qoy = 1) and (ws1.d_year = 1999)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF7 RF8 +------------------------filter((ss2.d_qoy = 2) and (ss2.d_year = 1999)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 RF7 RF8 --------------------PhysicalProject -----------------------filter((ws2.d_qoy = 2) and (ws2.d_year = 1999)) -------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF8 +----------------------filter((ss3.d_qoy = 3) and (ss3.d_year = 1999)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalProject -------------------filter((ws3.d_qoy = 3) and (ws3.d_year = 1999)) ---------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------filter((ws1.d_qoy = 1) and (ws1.d_year = 1999)) +--------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF7 RF8 +--------------PhysicalProject +----------------filter((ws2.d_qoy = 2) and (ws2.d_year = 1999)) +------------------PhysicalCteConsumer ( cteId=CTEId#1 ) apply RFs: RF8 +----------PhysicalProject +------------filter((ws3.d_qoy = 3) and (ws3.d_year = 1999)) +--------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query94.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query94.out index 792f183626c468..2f12dce9e26230 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query94.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query94.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[DISTINCT_GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF3 ws_order_number->[ws_order_number] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '1999-05-31') and (date_dim.d_date >= '1999-04-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_state = 'NE')) ---------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] ---------------------PhysicalProject -----------------------PhysicalOlapScan[web_sales] -----------------PhysicalProject -------------------PhysicalOlapScan[web_returns] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((ws1.ws_order_number = wr1.wr_order_number)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws2.ws_order_number)) otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) build RFs:RF3 ws_order_number->[ws_order_number] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF2 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[ws_ship_addr_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_ship_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_date <= '1999-05-31') and (date_dim.d_date >= '1999-04-01')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_state = 'NE')) +----------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] +------------------PhysicalProject +--------------------PhysicalOlapScan[web_returns] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query95.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query95.out index 4146fd70b83ac6..140ec5cb678d55 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query95.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query95.out @@ -9,35 +9,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalProject ----------PhysicalOlapScan[web_sales] apply RFs: RF7 --PhysicalResultSink -----PhysicalTopN[GATHER_SORT] -------hashAgg[DISTINCT_GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------hashJoin[LEFT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 wr_order_number->[ws_order_number,ws_order_number] -------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 ws_order_number->[ws_order_number] +----PhysicalLimit[GLOBAL] +------PhysicalLimit[LOCAL] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------hashJoin[LEFT_SEMI_JOIN colocated] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF6 wr_order_number->[ws_order_number,ws_order_number] +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF5 ws_order_number->[ws_order_number] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF4 web_site_sk->[ws_web_site_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ws_ship_addr_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_state = 'AL')) +----------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((web_site.web_company_name = 'pri')) +------------------------------PhysicalOlapScan[web_site] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk)) otherCondition=() build RFs:RF4 web_site_sk->[ws_web_site_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ws_ship_addr_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_ship_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= '2002-05-31') and (date_dim.d_date >= '2002-04-01')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_state = 'AL')) ---------------------------------PhysicalOlapScan[customer_address] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 wr_order_number->[ws_order_number,ws_order_number] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalProject ---------------------------filter((web_site.web_company_name = 'pri')) -----------------------------PhysicalOlapScan[web_site] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number)) otherCondition=() build RFs:RF7 wr_order_number->[ws_order_number,ws_order_number] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_returns] +--------------------------PhysicalOlapScan[web_returns] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query96.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query96.out index 5b0549e0eb3739..297341a8714d27 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query96.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query96.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- PhysicalResultSink ---PhysicalTopN[GATHER_SORT] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecGather] ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ss_sold_time_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------PhysicalProject -------------------------filter((household_demographics.hd_dep_count = 6)) ---------------------------PhysicalOlapScan[household_demographics] -------------------PhysicalProject ---------------------filter((time_dim.t_hour = 16) and (time_dim.t_minute >= 30)) -----------------------PhysicalOlapScan[time_dim] ---------------PhysicalProject -----------------filter((store.s_store_name = 'ese')) -------------------PhysicalOlapScan[store] +--PhysicalLimit[GLOBAL] +----PhysicalLimit[LOCAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ss_sold_time_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------PhysicalProject +--------------------------filter((household_demographics.hd_dep_count = 6)) +----------------------------PhysicalOlapScan[household_demographics] +--------------------PhysicalProject +----------------------filter((time_dim.t_hour = 16) and (time_dim.t_minute >= 30)) +------------------------PhysicalOlapScan[time_dim] +----------------PhysicalProject +------------------filter((store.s_store_name = 'ese')) +--------------------PhysicalOlapScan[store] diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index 65bd291f67e6db..a50dcbd83dec4e 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -23,6 +23,7 @@ suite("push_down_count_through_join_one_side") { sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + sql "set topn_opt_limit_threshold=1024" sql """ DROP TABLE IF EXISTS count_t_one_side; """ diff --git a/regression-test/suites/nereids_rules_p0/eliminate_order_by_key/eliminate_order_by_key.groovy b/regression-test/suites/nereids_rules_p0/eliminate_order_by_key/eliminate_order_by_key.groovy new file mode 100644 index 00000000000000..15a9ad86f8c829 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eliminate_order_by_key/eliminate_order_by_key.groovy @@ -0,0 +1,193 @@ +// 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("eliminate_order_by_key") { + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "drop table if exists eliminate_order_by_constant_t" + sql """create table eliminate_order_by_constant_t(a int null, b int not null, c varchar(10) null, d date, dt datetime, id int) + distributed by hash(a) properties("replication_num"="1"); + """ +// sql "set disable_nereids_rules='eliminate_order_by_key'" + sql """ + INSERT INTO eliminate_order_by_constant_t (a, b, c, d, dt,id) VALUES + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00',1), + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00',2), + (2, 101, 'banana', '2023-01-02', '2023-01-02 11:00:00',3), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00',4), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00',5), + (NULL, 103, 'date', '2023-01-04', '2023-01-04 13:00:00',6), + (4, 104, 'elderberry', '2023-01-05', '2023-01-05 14:00:00',7), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00',8), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00',9), + (6, 106, 'fig', '2023-01-07', '2023-01-07 16:00:00',10), + (NULL, 107, 'grape', '2023-01-08', '2023-01-08 17:00:00',11); + """ + qt_predicate "select 1 as c1,a from eliminate_order_by_constant_t where a=1 order by a" + qt_predicate_order_by_two "select 1 as c1,a from eliminate_order_by_constant_t where a=1 order by a,c1" + qt_with_group_by """select 1 as c1,a from eliminate_order_by_constant_t where a=1 group by c1,a order by a""" + qt_predicate_multi_other """select 1 as c1,a,b,c from eliminate_order_by_constant_t where a=1 order by a,'abc',b,c""" + qt_with_group_by_shape """explain shape plan select 1 as c1,a from eliminate_order_by_constant_t where a=1 group by c1,a order by a""" + + // fd + qt_fd "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a+1,id" + qt_fd_duplicate "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a,abs(a),a+1,id" + qt_fd_topn "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a+1,id limit 5" + qt_fd_duplicate_topn "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,abs(a),a,abs(a),a+1,id limit 5" + qt_fd_multi_column "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,b,a+b" + qt_fd_desc "select a,b,c,d,dt from eliminate_order_by_constant_t order by a desc,abs(a) asc,a+1 desc,id" + qt_fd_multi_column_desc "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,b desc,a+b" + qt_fd_multi_column_desc_with_other_in_middle "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,c,b desc,a+b asc" + qt_equalset_fd "select a,b,c,d,dt from eliminate_order_by_constant_t where a=b order by a,a+b, b" + qt_uniform_fd "select a,b from eliminate_order_by_constant_t where a=b and a=1 order by a,b,a+b" + qt_fd_valid_slot_add_equalset "select c,d,a,a+100,b+a+100,b from eliminate_order_by_constant_t where b=a order by c,d,a,a+100,b+a+100" + + // duplicate + qt_dup_shape "select a,b,c,d,dt from eliminate_order_by_constant_t order by a,a,id" + qt_dup_expr_shape "select a,b,c,d,dt from eliminate_order_by_constant_t order by a+1,a+1,id" + + // window + qt_dup_window "select sum(a) over (partition by a order by a,a) from eliminate_order_by_constant_t order by 1" + qt_fd_window "select sum(a) over (partition by a order by a,a+1,abs(a),1-a,b) from eliminate_order_by_constant_t order by 1" + qt_uniform_window "select sum(a) over (partition by a order by b) from eliminate_order_by_constant_t where b=100 order by 1" + qt_uniform_window "select first_value(c) over (partition by a order by b) from eliminate_order_by_constant_t where b=100 order by 1" + qt_multi_window """select sum(a) over (partition by a order by a,a+1,abs(a),1-a,b), max(a) over (partition by a order by b,b+1,b,abs(b)) + from eliminate_order_by_constant_t order by 1,2""" + qt_multi_window_desc """select sum(a) over (partition by a order by a desc,a+1 asc,abs(a) desc,1-a,b), max(a) over (partition by a order by b desc,b+1 desc,b asc,abs(b) desc) + from eliminate_order_by_constant_t order by 1,2""" + + + sql "drop table if exists eliminate_order_by_constant_t2" + sql """create table eliminate_order_by_constant_t2(a int, b int, c int, d int) distributed by hash(a) properties("replication_num"="1");""" + sql """INSERT INTO eliminate_order_by_constant_t2 (a, b, c, d) + VALUES(1, 2, 3, 4),(2, 3, 3, 5),(3, 4, 5, 6),(4, 5, 6, 7),(5, 6, 7, 8),(6, 7, 8, 9),(7, 8, 9, 10),(8, 9, 10, 11),(9, 10, 11, 12),(10, 11, 12, 13);""" + qt_equal_set_uniform """select * from eliminate_order_by_constant_t2 where b=a and a=c and d=1 order by d,a,b,c,c,b,a,d,d""" + qt_equal_set_uniform2 """select * from eliminate_order_by_constant_t2 where b=a and a=d and a=c and d=1 order by d,a,b,c,c,b,a,d,d""" + qt_equal_set_uniform_fd """select * from eliminate_order_by_constant_t2 where b=a and a=d and a=c and d=1 order by d,a,a+1,b+1,c+1,c,b,a,d,d""" + qt_fd_uniform """select * from eliminate_order_by_constant_t2 where d=1 order by d,a,b,c,d+b+a-100,d+b+a,b,a,d,d""" + qt_equalset_fd "select * from eliminate_order_by_constant_t2 where d=b and a=d order by a,c,d+b+a,b,a,d,d+1,abs(d)" + + // other operator + // join + qt_join_inner_order_by """ + select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 + inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c; + """ + + qt_join_left_outer_order_by """ + select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 + left outer join eliminate_order_by_constant_t2 t2 on t1.a = t2.a order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c; + """ + + qt_join_right_outer_order_by """ + select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 + right outer join eliminate_order_by_constant_t2 t2 on t1.a = t2.a order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c; + """ + + qt_join_right_outer_order_by_predicate """select t1.a, t1.b, t2.c, t2.d from eliminate_order_by_constant_t t1 + right outer join eliminate_order_by_constant_t2 t2 on t1.a = t2.a where t2.a=1 order by t1.a, t2.a, t2.c, t1.a+t2.a, t1.b, t2.d, t1.a+1, t2.c;""" + + qt_join_left_semi_order_by """ + select t1.a, t1.b from eliminate_order_by_constant_t t1 + left semi join eliminate_order_by_constant_t2 t2 on t1.a = t2.a order by t1.a, t1.b, t1.a+1,t1.a+t1.b; + """ + + qt_join_right_semi_order_by """ + select t2.a, t2.b + from eliminate_order_by_constant_t t1 + right semi join eliminate_order_by_constant_t2 t2 on t1.a = t2.a + order by t2.a, t2.b, t2.a+1,t2.a+t2.b; + """ + + qt_join_left_anti_order_by """ + select t1.a, t1.b + from eliminate_order_by_constant_t t1 + left anti join eliminate_order_by_constant_t2 t2 on t1.a = t2.a + order by t1.a, t1.b, t1.a+1,t1.a+t1.b; + """ + + qt_join_right_anti_order_by """ + select t2.a, t2.b + from eliminate_order_by_constant_t t1 + right anti join eliminate_order_by_constant_t2 t2 on t1.a = t2.a + order by t2.a, t2.b, t2.a+1,t2.a+t2.b; + """ + // agg + qt_agg_order_by """ + select a, count(b) as cnt + from eliminate_order_by_constant_t2 + group by a + order by a, cnt, a,a+cnt,a+100; + """ + // agg+grouping + qt_agg_grouping_order_by """ + select a, b, count(c) as cnt + from eliminate_order_by_constant_t2 + group by cube(a, b) + order by a, b, cnt, a, b+1; + """ + // join+window + qt_join_window_order_by """ + select t1.a, t1.b, t2.c, t2.d, t2.a, + row_number() over (partition by t1.a order by t1.b) as rn + from eliminate_order_by_constant_t t1 + inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a + order by t1.a, t2.c, t1.b, t2.d, abs(t1.a), abs(t2.a), t2.c,rn,rn+100; + """ + // agg+window + qt_agg_window_order_by """ + select a, b, count(c) as cnt, + row_number() over (partition by a order by b) as rn + from eliminate_order_by_constant_t2 + group by a, b + order by a, b, cnt, a+100, b, rn, rn+cnt, abs(rn+cnt); + """ + // join + agg+ window + qt_join_agg_window_order_by """ + select t1.a, t1.b, count(t2.c) as cnt, + row_number() over (partition by t1.a order by t1.b) as rn + from eliminate_order_by_constant_t t1 + inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a + group by t1.a, t1.b ,t2.a + order by t1.a,t2.a,t1.b, cnt, -t1.a, -t1.b-1000,rn, cnt, rn+111; + """ + // union all + order by + qt_union_all_order_by """ + select * from ( + select a, b from eliminate_order_by_constant_t2 + union all + select a, b from eliminate_order_by_constant_t ) t + order by a, b, abs(a),abs(a)+b,a+b,a,b; + """ + // union + join + order by + qt_union_join_order_by """ + select * from (select t1.a, t1.b + from eliminate_order_by_constant_t t1 + inner join eliminate_order_by_constant_t2 t2 on t1.a = t2.a + union + select t1.a, t1.b + from eliminate_order_by_constant_t t1 + left join eliminate_order_by_constant_t2 t2 on t1.a = t2.a ) t + where a=1 + order by a, b, a+100,abs(a)+b; + """ + + // test composite key + sql """create table test_unique_order_by2(a int not null, b int not null, c int, d int) unique key(a,b) distributed by hash(a) properties('replication_num'='1');""" + sql """INSERT INTO test_unique_order_by2 (a, b, c, d) + VALUES(1, 2, 3, 4),(2, 3, 3, 5),(3, 4, 5, 6),(4, 5, 6, 7),(5, 6, 7, 8),(6, 7, 8, 9),(7, 8, 9, 10),(8, 9, 10, 11),(9, 10, 11, 12),(10, 11, 12, 13);""" + qt_composite_key """select * from test_unique_order_by2 order by a,'abc',d,b,d,c;""" +} \ No newline at end of file