diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index e02b18ce8490cb..5fbe354ffa0af9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -294,7 +294,8 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d && context.getFirstAggregateInFragment(inputFragment) == child) { PhysicalHashAggregate hashAggregate = (PhysicalHashAggregate) child; if (hashAggregate.getAggPhase() == AggPhase.LOCAL - && hashAggregate.getAggMode() == AggMode.INPUT_TO_BUFFER) { + && hashAggregate.getAggMode() == AggMode.INPUT_TO_BUFFER + && hashAggregate.getTopnPushInfo() == null) { AggregationNode aggregationNode = (AggregationNode) inputFragment.getPlanRoot(); aggregationNode.setUseStreamingPreagg(hashAggregate.isMaybeUsingStream()); } @@ -1058,6 +1059,23 @@ public PlanFragment visitPhysicalHashAggregate( // local exchanger will be used. aggregationNode.setColocate(true); } + if (aggregate.getTopnPushInfo() != null) { + List orderingExprs = Lists.newArrayList(); + List ascOrders = Lists.newArrayList(); + List nullsFirstParams = Lists.newArrayList(); + aggregate.getTopnPushInfo().orderkeys.forEach(k -> { + orderingExprs.add(ExpressionTranslator.translate(k.getExpr(), context)); + ascOrders.add(k.isAsc()); + nullsFirstParams.add(k.isNullFirst()); + }); + SortInfo sortInfo = new SortInfo(orderingExprs, ascOrders, nullsFirstParams, outputTupleDesc); + aggregationNode.setSortByGroupKey(sortInfo); + if (aggregationNode.getLimit() == -1) { + aggregationNode.setLimit(aggregate.getTopnPushInfo().limit); + } + } else { + aggregationNode.setSortByGroupKey(null); + } setPlanRoot(inputPlanFragment, aggregationNode, aggregate); if (aggregate.getStats() != null) { aggregationNode.setCardinality((long) aggregate.getStats().getRowCount()); 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 9f3ea9bc378d4c..cc8a3d3fd71cb0 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 @@ -84,6 +84,7 @@ import org.apache.doris.nereids.rules.rewrite.InferPredicates; import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.InlineLogicalView; +import org.apache.doris.nereids.rules.rewrite.LimitAggToTopNAgg; import org.apache.doris.nereids.rules.rewrite.LimitSortToTopN; import org.apache.doris.nereids.rules.rewrite.LogicalResultSinkToShortCircuitPointQuery; import org.apache.doris.nereids.rules.rewrite.MergeAggregate; @@ -366,6 +367,7 @@ public class Rewriter extends AbstractBatchJobExecutor { // generate one PhysicalLimit if current distribution is gather or two // PhysicalLimits with gather exchange topDown(new LimitSortToTopN()), + topDown(new LimitAggToTopNAgg()), topDown(new MergeTopNs()), topDown(new SplitLimit()), topDown( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java index 59410115793ceb..9239070d3ce6db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PlanPostProcessors.java @@ -65,6 +65,9 @@ public List getProcessors() { builder.add(new AddOffsetIntoDistribute()); builder.add(new CommonSubExpressionOpt()); // DO NOT replace PLAN NODE from here + if (cascadesContext.getConnectContext().getSessionVariable().pushTopnToAgg) { + builder.add(new PushTopnToAgg()); + } builder.add(new TopNScanOpt()); builder.add(new FragmentProcessor()); if (!cascadesContext.getConnectContext().getSessionVariable().getRuntimeFilterMode() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PushTopnToAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PushTopnToAgg.java new file mode 100644 index 00000000000000..aca3f21a7d175f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/PushTopnToAgg.java @@ -0,0 +1,177 @@ +// 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. +// This file is copied from +// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/AggregationNode.java +// and modified by Doris + +package org.apache.doris.nereids.processor.post; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.properties.DistributionSpecGather; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.AggMode; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate.TopnPushInfo; +import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; +import org.apache.doris.qe.ConnectContext; + +import org.apache.hadoop.util.Lists; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Add SortInfo to Agg. This SortInfo is used as boundary, not used to sort elements. + * example + * sql: select count(*) from orders group by o_clerk order by o_clerk limit 1; + * plan: topn(1) -> aggGlobal -> shuffle -> aggLocal -> scan + * optimization: aggLocal and aggGlobal only need to generate the smallest row with respect to o_clerk. + * + * TODO: the following case is not covered: + * sql: select sum(o_shippriority) from orders group by o_clerk limit 1; + * plan: limit -> aggGlobal -> shuffle -> aggLocal -> scan + * aggGlobal may receive partial aggregate results, and hence is not supported now + * instance1: input (key=2, v=1) => localAgg => (2, 1) => aggGlobal inst1 => (2, 1) + * instance2: input (key=1, v=1), (key=2, v=2) => localAgg inst2 => (1, 1) + * (2,1),(1,1) => limit => may output (2, 1), which is not complete, missing (2, 2) in instance2 + * + *TOPN: + * Precondition: topn orderkeys are the prefix of group keys + * TODO: topnKeys could be subset of groupKeys. This will be implemented in future + * Pattern 2-phase agg: + * topn -> aggGlobal -> distribute -> aggLocal + * => + * topn(n) -> aggGlobal(topn=n) -> distribute -> aggLocal(topn=n) + * Pattern 1-phase agg: + * topn->agg->Any(not agg) -> topn -> agg(topn=n) -> any + * + * LIMIT: + * Pattern 1: limit->agg(1phase)->any + * Pattern 2: limit->agg(global)->gather->agg(local) + */ +public class PushTopnToAgg extends PlanPostProcessor { + @Override + public Plan visitPhysicalTopN(PhysicalTopN topN, CascadesContext ctx) { + topN.child().accept(this, ctx); + if (ConnectContext.get().getSessionVariable().topnOptLimitThreshold <= topN.getLimit() + topN.getOffset()) { + return topN; + } + Plan topnChild = topN.child(); + if (topnChild instanceof PhysicalProject) { + topnChild = topnChild.child(0); + } + if (topnChild instanceof PhysicalHashAggregate) { + PhysicalHashAggregate upperAgg = (PhysicalHashAggregate) topnChild; + List orderKeys = tryGenerateOrderKeyByGroupKeyAndTopnKey(topN, upperAgg); + if (!orderKeys.isEmpty()) { + + if (upperAgg.getAggPhase().isGlobal() && upperAgg.getAggMode() == AggMode.BUFFER_TO_RESULT) { + upperAgg.setTopnPushInfo(new TopnPushInfo( + orderKeys, + topN.getLimit() + topN.getOffset())); + if (upperAgg.child() instanceof PhysicalDistribute + && upperAgg.child().child(0) instanceof PhysicalHashAggregate) { + PhysicalHashAggregate bottomAgg = + (PhysicalHashAggregate) upperAgg.child().child(0); + bottomAgg.setTopnPushInfo(new TopnPushInfo( + orderKeys, + topN.getLimit() + topN.getOffset())); + } + } else if (upperAgg.getAggPhase().isLocal() && upperAgg.getAggMode() == AggMode.INPUT_TO_RESULT) { + // one phase agg + upperAgg.setTopnPushInfo(new TopnPushInfo( + orderKeys, + topN.getLimit() + topN.getOffset())); + } + } + } + return topN; + } + + /** + return true, if topn order-key is prefix of agg group-key, ignore asc/desc and null_first + TODO order-key can be subset of group-key. BE does not support now. + */ + private List tryGenerateOrderKeyByGroupKeyAndTopnKey(PhysicalTopN topN, + PhysicalHashAggregate agg) { + List orderKeys = Lists.newArrayListWithCapacity(agg.getGroupByExpressions().size()); + if (topN.getOrderKeys().size() > agg.getGroupByExpressions().size()) { + return orderKeys; + } + List topnKeys = topN.getOrderKeys().stream() + .map(OrderKey::getExpr).collect(Collectors.toList()); + for (int i = 0; i < topN.getOrderKeys().size(); i++) { + // prefix check + if (!topnKeys.get(i).equals(agg.getGroupByExpressions().get(i))) { + return Lists.newArrayList(); + } + orderKeys.add(topN.getOrderKeys().get(i)); + } + for (int i = topN.getOrderKeys().size(); i < agg.getGroupByExpressions().size(); i++) { + orderKeys.add(new OrderKey(agg.getGroupByExpressions().get(i), true, false)); + } + return orderKeys; + } + + @Override + public Plan visitPhysicalLimit(PhysicalLimit limit, CascadesContext ctx) { + limit.child().accept(this, ctx); + if (ConnectContext.get().getSessionVariable().topnOptLimitThreshold <= limit.getLimit() + limit.getOffset()) { + return limit; + } + Plan limitChild = limit.child(); + if (limitChild instanceof PhysicalProject) { + limitChild = limitChild.child(0); + } + if (limitChild instanceof PhysicalHashAggregate) { + PhysicalHashAggregate upperAgg = (PhysicalHashAggregate) limitChild; + if (upperAgg.getAggPhase().isGlobal() && upperAgg.getAggMode() == AggMode.BUFFER_TO_RESULT) { + Plan child = upperAgg.child(); + Plan grandChild = child.child(0); + if (child instanceof PhysicalDistribute + && ((PhysicalDistribute) child).getDistributionSpec() instanceof DistributionSpecGather + && grandChild instanceof PhysicalHashAggregate) { + upperAgg.setTopnPushInfo(new TopnPushInfo( + generateOrderKeyByGroupKey(upperAgg), + limit.getLimit() + limit.getOffset())); + PhysicalHashAggregate bottomAgg = + (PhysicalHashAggregate) grandChild; + bottomAgg.setTopnPushInfo(new TopnPushInfo( + generateOrderKeyByGroupKey(bottomAgg), + limit.getLimit() + limit.getOffset())); + } + } else if (upperAgg.getAggMode() == AggMode.INPUT_TO_RESULT) { + // 1-phase agg + upperAgg.setTopnPushInfo(new TopnPushInfo( + generateOrderKeyByGroupKey(upperAgg), + limit.getLimit() + limit.getOffset())); + } + } + return limit; + } + + private List generateOrderKeyByGroupKey(PhysicalHashAggregate agg) { + return agg.getGroupByExpressions().stream() + .map(key -> new OrderKey(key, true, false)) + .collect(Collectors.toList()); + } +} 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 6d13a2f7784daf..2ef3da2eb3bf1b 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 @@ -304,6 +304,7 @@ public enum RuleType { PUSH_LIMIT_THROUGH_UNION(RuleTypeClass.REWRITE), PUSH_LIMIT_THROUGH_WINDOW(RuleTypeClass.REWRITE), LIMIT_SORT_TO_TOP_N(RuleTypeClass.REWRITE), + LIMIT_AGG_TO_TOPN_AGG(RuleTypeClass.REWRITE), // topN push down PUSH_DOWN_TOP_N_THROUGH_JOIN(RuleTypeClass.REWRITE), PUSH_DOWN_TOP_N_THROUGH_PROJECT_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java new file mode 100644 index 00000000000000..040bd9c10f3f3e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LimitAggToTopNAgg.java @@ -0,0 +1,119 @@ +// 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.properties.OrderKey; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * convert limit->agg to topn->agg + * if all group keys are in limit.output + * to enable + * 1. topn-filter + * 2. push limit to local agg + */ +public class LimitAggToTopNAgg implements RewriteRuleFactory { + @Override + public List buildRules() { + return ImmutableList.of( + // limit -> agg to topn->agg + logicalLimit(logicalAggregate()) + .when(limit -> ConnectContext.get() != null + && ConnectContext.get().getSessionVariable().pushTopnToAgg + && ConnectContext.get().getSessionVariable().topnOptLimitThreshold + >= limit.getLimit() + limit.getOffset()) + .then(limit -> { + LogicalAggregate agg = limit.child(); + List orderKeys = generateOrderKeyByGroupKey(agg); + return new LogicalTopN<>(orderKeys, limit.getLimit(), limit.getOffset(), agg); + }).toRule(RuleType.LIMIT_AGG_TO_TOPN_AGG), + //limit->project->agg to topn->project->agg + logicalLimit(logicalProject(logicalAggregate())) + .when(limit -> ConnectContext.get() != null + && ConnectContext.get().getSessionVariable().pushTopnToAgg + && ConnectContext.get().getSessionVariable().topnOptLimitThreshold + >= limit.getLimit() + limit.getOffset()) + .when(limit -> outputAllGroupKeys(limit, limit.child().child())) + .then(limit -> { + LogicalProject project = limit.child(); + LogicalAggregate agg = (LogicalAggregate) project.child(); + List orderKeys = generateOrderKeyByGroupKey(agg); + return new LogicalTopN<>(orderKeys, limit.getLimit(), limit.getOffset(), project); + }).toRule(RuleType.LIMIT_AGG_TO_TOPN_AGG), + // topn -> agg: add all group key to sort key, if sort key is prefix of group key + logicalTopN(logicalAggregate()) + .when(topn -> ConnectContext.get() != null + && ConnectContext.get().getSessionVariable().pushTopnToAgg + && ConnectContext.get().getSessionVariable().topnOptLimitThreshold + >= topn.getLimit() + topn.getOffset()) + .then(topn -> { + LogicalAggregate agg = (LogicalAggregate) topn.child(); + List newOrders = tryGenerateOrderKeyByGroupKeyAndTopnKey(topn, agg); + if (newOrders.isEmpty()) { + return topn; + } else { + return topn.withOrderKeys(newOrders); + } + }).toRule(RuleType.LIMIT_AGG_TO_TOPN_AGG)); + } + + private List tryGenerateOrderKeyByGroupKeyAndTopnKey(LogicalTopN topN, + LogicalAggregate agg) { + List orderKeys = Lists.newArrayListWithCapacity(agg.getGroupByExpressions().size()); + if (topN.getOrderKeys().size() > agg.getGroupByExpressions().size()) { + return orderKeys; + } + List topnKeys = topN.getOrderKeys().stream() + .map(OrderKey::getExpr).collect(Collectors.toList()); + for (int i = 0; i < topN.getOrderKeys().size(); i++) { + // prefix check + if (!topnKeys.get(i).equals(agg.getGroupByExpressions().get(i))) { + return Lists.newArrayList(); + } + orderKeys.add(topN.getOrderKeys().get(i)); + } + for (int i = topN.getOrderKeys().size(); i < agg.getGroupByExpressions().size(); i++) { + orderKeys.add(new OrderKey(agg.getGroupByExpressions().get(i), true, false)); + } + return orderKeys; + } + + private boolean outputAllGroupKeys(LogicalLimit limit, LogicalAggregate agg) { + return limit.getOutputSet().containsAll(agg.getGroupByExpressions()); + } + + private List generateOrderKeyByGroupKey(LogicalAggregate agg) { + return agg.getGroupByExpressions().stream() + .map(key -> new OrderKey(key, true, false)) + .collect(Collectors.toList()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java index a79953dc71ee81..3e6f40cc0d44b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.properties.RequireProperties; import org.apache.doris.nereids.properties.RequirePropertiesSupplier; @@ -32,6 +33,7 @@ import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.algebra.Aggregate; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.MutableState; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; @@ -188,6 +190,8 @@ public List getExpressions() { @Override public String toString() { + TopnPushInfo topnPushInfo = (TopnPushInfo) getMutableState( + MutableState.KEY_PUSH_TOPN_TO_AGG).orElseGet(() -> null); return Utils.toSqlString("PhysicalHashAggregate[" + id.asInt() + "]" + getGroupIdWithPrefix(), "aggPhase", aggregateParam.aggPhase, "aggMode", aggregateParam.aggMode, @@ -196,6 +200,7 @@ public String toString() { "outputExpr", outputExpressions, "partitionExpr", partitionExpressions, "requireProperties", requireProperties, + "topnOpt", topnPushInfo != null, "stats", statistics ); } @@ -299,4 +304,30 @@ public PhysicalHashAggregate resetLogicalProperties() { requireProperties, physicalProperties, statistics, child()); } + + /** + * used to push limit down to localAgg + */ + public static class TopnPushInfo { + public List orderkeys; + public long limit; + + public TopnPushInfo(List orderkeys, long limit) { + this.orderkeys = ImmutableList.copyOf(orderkeys); + this.limit = limit; + } + } + + public TopnPushInfo getTopnPushInfo() { + Optional obj = getMutableState(MutableState.KEY_PUSH_TOPN_TO_AGG); + if (obj.isPresent() && obj.get() instanceof TopnPushInfo) { + return (TopnPushInfo) obj.get(); + } + return null; + } + + public PhysicalHashAggregate setTopnPushInfo(TopnPushInfo topnPushInfo) { + setMutableState(MutableState.KEY_PUSH_TOPN_TO_AGG, topnPushInfo); + return this; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/MutableState.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/MutableState.java index 6f03de77fa57b1..cdd0896e0c8809 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/MutableState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/MutableState.java @@ -27,7 +27,7 @@ public interface MutableState { String KEY_FRAGMENT = "fragment"; String KEY_PARENT = "parent"; String KEY_RF_JUMP = "rf-jump"; - + String KEY_PUSH_TOPN_TO_AGG = "pushTopnToAgg"; Optional get(String key); MutableState set(String key, Object value); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java index c1beae813bf211..e6ea3e9f263175 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java @@ -26,6 +26,7 @@ import org.apache.doris.analysis.FunctionCallExpr; import org.apache.doris.analysis.SlotDescriptor; import org.apache.doris.analysis.SlotId; +import org.apache.doris.analysis.SortInfo; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.common.NotImplementedException; import org.apache.doris.common.UserException; @@ -65,6 +66,8 @@ public class AggregationNode extends PlanNode { // If true, use streaming preaggregation algorithm. Not valid if this is a merge agg. private boolean useStreamingPreagg; + private SortInfo sortByGroupKey; + /** * Create an agg node that is not an intermediate node. * isIntermediate is true if it is a slave node in a 2-part agg plan. @@ -288,6 +291,9 @@ protected void toThrift(TPlanNode msg) { msg.agg_node.setUseStreamingPreaggregation(useStreamingPreagg); msg.agg_node.setIsFirstPhase(aggInfo.isFirstPhase()); msg.agg_node.setIsColocate(isColocate); + if (sortByGroupKey != null) { + msg.agg_node.setAggSortInfoByGroupKey(sortByGroupKey.toThrift()); + } List groupingExprs = aggInfo.getGroupingExprs(); if (groupingExprs != null) { msg.agg_node.setGroupingExprs(Expr.treesToThrift(groupingExprs)); @@ -333,6 +339,7 @@ public String getNodeExplainString(String detailPrefix, TExplainLevel detailLeve if (!conjuncts.isEmpty()) { output.append(detailPrefix).append("having: ").append(getExplainString(conjuncts)).append("\n"); } + output.append(detailPrefix).append("sortByGroupKey:").append(sortByGroupKey != null).append("\n"); output.append(detailPrefix).append(String.format( "cardinality=%,d", cardinality)).append("\n"); return output.toString(); @@ -411,4 +418,13 @@ public void finalize(Analyzer analyzer) throws UserException { public void setColocate(boolean colocate) { isColocate = colocate; } + + + public boolean isSortByGroupKey() { + return sortByGroupKey != null; + } + + public void setSortByGroupKey(SortInfo sortByGroupKey) { + this.sortByGroupKey = sortByGroupKey; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 62fdb83ded8bce..8beb9c05b79b25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1269,6 +1269,9 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = REWRITE_OR_TO_IN_PREDICATE_THRESHOLD, fuzzy = true) private int rewriteOrToInPredicateThreshold = 2; + @VariableMgr.VarAttr(name = "push_topn_to_agg", fuzzy = false, needForward = true) + public boolean pushTopnToAgg = true; + @VariableMgr.VarAttr(name = NEREIDS_CBO_PENALTY_FACTOR, needForward = true) private double nereidsCboPenaltyFactor = 0.7; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java index 5787229c5ba27f..ac1ffa97ef9e7d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateSortTest.java @@ -122,6 +122,7 @@ void testEliminateSortUnderTableSink() { @Test void testEliminateSortInUnion() { PlanChecker.from(connectContext) + .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("SELECT * FROM (SELECT * FROM student UNION SELECT * FROM student ORDER BY id) u LIMIT 1") .rewrite() .nonMatch(logicalSort()); @@ -130,10 +131,12 @@ void testEliminateSortInUnion() { @Test void testEliminateSortInSubquery() { PlanChecker.from(connectContext) + .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from (select * from student order by id) t") .rewrite() .nonMatch(logicalSort()); PlanChecker.from(connectContext) + .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select \n" + " id, \n" + " name \n" @@ -159,24 +162,30 @@ void testEliminateSortInSubquery() { @Test void testSortLimit() { - PlanChecker.from(connectContext) + PlanChecker.from(connectContext).disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from (select * from student order by id) t limit 1") .rewrite() - .nonMatch(logicalTopN()); + // there is no topn below agg + .matches(logicalTopN(logicalAggregate(logicalProject(logicalOlapScan())))); PlanChecker.from(connectContext) + .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from (select * from student order by id limit 1) t") .rewrite() .matches(logicalTopN()); PlanChecker.from(connectContext) + .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from " + "(select * from student order by id limit 1) t1 left join student t2 on t1.id = t2.id") .rewrite() .matches(logicalTopN()); PlanChecker.from(connectContext) + .disableNereidsRules("PRUNE_EMPTY_PARTITION") .analyze("select count(*) from " + "(select * from student order by id) t1 left join student t2 on t1.id = t2.id limit 1") .rewrite() - .nonMatch(logicalTopN()); + .matches(logicalTopN(logicalAggregate(logicalProject(logicalJoin( + logicalProject(logicalOlapScan()), + logicalProject(logicalOlapScan())))))); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoinTest.java index bce9a2cd207a66..910e4ce669ab7d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoinTest.java @@ -134,7 +134,7 @@ void testJoinSql() { .rewrite() .matches( logicalProject(logicalJoin( - logicalLimit(logicalAggregate(logicalProject(logicalOlapScan()))) + logicalTopN(logicalAggregate(logicalProject(logicalOlapScan()))) .when(l -> l.getLimit() == 10), logicalProject(logicalOlapScan()) )) diff --git a/regression-test/data/nereids_clickbench_shape_p0/query18.out b/regression-test/data/nereids_clickbench_shape_p0/query18.out index 0a03abdf84f4d9..6af4a027d886c3 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query18.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query18.out @@ -1,9 +1,9 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ckbench_shape_18 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] +--PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] -------PhysicalLimit[LOCAL] +------PhysicalTopN[LOCAL_SORT] --------hashAgg[LOCAL] ----------PhysicalProject ------------PhysicalOlapScan[hits] diff --git a/regression-test/data/nereids_rules_p0/limit_push_down/limit_push_down.out b/regression-test/data/nereids_rules_p0/limit_push_down/limit_push_down.out index 54b8a7a6b6c6b5..6fc51c5c00bafb 100644 --- a/regression-test/data/nereids_rules_p0/limit_push_down/limit_push_down.out +++ b/regression-test/data/nereids_rules_p0/limit_push_down/limit_push_down.out @@ -97,8 +97,8 @@ PhysicalResultSink -- !limit_distinct -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] ------hashAgg[LOCAL] --------PhysicalOlapScan[t1] @@ -111,46 +111,52 @@ PhysicalResultSink -- !limit_distinct -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[LOCAL] ---------NestedLoopJoin[CROSS_JOIN] -----------PhysicalLimit[LOCAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[t1] -----------PhysicalOlapScan[t2] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------NestedLoopJoin[CROSS_JOIN] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[t1] +------------PhysicalOlapScan[t2] -- !limit_distinct -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[LOCAL] ---------NestedLoopJoin[CROSS_JOIN] -----------PhysicalLimit[LOCAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[t1] -----------PhysicalOlapScan[t2] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------NestedLoopJoin[CROSS_JOIN] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[t1] +------------PhysicalOlapScan[t2] -- !limit_distinct -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] ------hashAgg[LOCAL] --------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalLimit[LOCAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[t1] +----------PhysicalTopN[MERGE_SORT] +------------PhysicalTopN[LOCAL_SORT] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t1] ----------PhysicalOlapScan[t2] -- !limit_distinct -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] ------hashAgg[LOCAL] --------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalLimit[LOCAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[t1] +----------PhysicalTopN[MERGE_SORT] +------------PhysicalTopN[LOCAL_SORT] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t1] ----------PhysicalOlapScan[t2] -- !limit_offset_agg -- @@ -162,17 +168,19 @@ PhysicalResultSink -- !limit_set_operation -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------PhysicalUnion -------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[t1] -------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[t2] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[t1] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[t2] -- !limit_offset_set_operation -- PhysicalResultSink @@ -351,21 +359,23 @@ PhysicalResultSink -- !limit_subquery_union_join -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------PhysicalUnion -------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] -----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------------PhysicalOlapScan[t1] -------------------PhysicalOlapScan[t2] -------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] -----------------hashJoin[INNER_JOIN] hashCondition=((t3.id = t4.id)) otherCondition=() -------------------PhysicalOlapScan[t3] -------------------PhysicalOlapScan[t4] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------------------PhysicalOlapScan[t1] +--------------------PhysicalOlapScan[t2] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------hashJoin[INNER_JOIN] hashCondition=((t3.id = t4.id)) otherCondition=() +--------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t4] -- !limit_subquery_union_window -- PhysicalResultSink @@ -510,8 +520,8 @@ PhysicalResultSink -- !limit_subquery_distinct -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] ------hashAgg[LOCAL] --------PhysicalOlapScan[t1] diff --git a/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.out b/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.out deleted file mode 100644 index 9ffe9520387501..00000000000000 --- a/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.out +++ /dev/null @@ -1,15 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !basic -- -PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------PhysicalUnion -------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[t] -------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[t] - diff --git a/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out new file mode 100644 index 00000000000000..714ce630f16351 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out @@ -0,0 +1,17 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !basic -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[t] +------------PhysicalTopN[MERGE_SORT] +--------------PhysicalTopN[LOCAL_SORT] +----------------hashAgg[LOCAL] +------------------PhysicalOlapScan[t] + diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out index 66a845c6290a8d..fa66824d3b83c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query23.out @@ -46,36 +46,35 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[item_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF6 ws_item_sk->[item_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalTopN[GATHER_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[item_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF6 ws_item_sk->[item_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 7) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out index c413940debc6e7..0389bfeea82673 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query32.out @@ -1,26 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) -----------------PhysicalWindow -------------------PhysicalQuickSort[LOCAL_SORT] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((item.i_manufact_id = 722)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +--------------PhysicalWindow +----------------PhysicalQuickSort[LOCAL_SORT] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((item.i_manufact_id = 722)) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2001-06-07') and (date_dim.d_date >= '2001-03-09')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out index 83fb52e056d5c3..7410248c2e22ea 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query38.out @@ -1,50 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalIntersect -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------PhysicalIntersect +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1200) and (date_dim.d_month_seq >= 1189)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out index d3a845763241f7..ca6f63d5e6bdf0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query97.out @@ -1,35 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------filter(( not ss_sold_date_sk IS NULL)) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) ---------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------filter(( not cs_sold_date_sk IS NULL)) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) ---------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------filter(( not ss_sold_date_sk IS NULL)) +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------filter(( not cs_sold_date_sk IS NULL)) +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1210) and (date_dim.d_month_seq >= 1199)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out index b748b3f800a60a..bdf35640afaef5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/constraints/query23.out @@ -44,36 +44,35 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------PhysicalProject ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------PhysicalTopN[GATHER_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out index f82ff43c6a8173..c5bc092c1008b1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.out @@ -46,36 +46,35 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------PhysicalOlapScan[date_dim] +------PhysicalTopN[GATHER_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out index 7992f57d1c87b3..15393efb582c73 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.out @@ -1,26 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) -----------------PhysicalWindow -------------------PhysicalQuickSort[LOCAL_SORT] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((item.i_manufact_id = 29)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +--------------PhysicalWindow +----------------PhysicalQuickSort[LOCAL_SORT] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((item.i_manufact_id = 29)) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out index e385d6e6157a32..ff996b20091063 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.out @@ -1,50 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalIntersect -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------PhysicalIntersect +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out index 4ebfd5abc0eb1c..404a9e7e9e92a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.out @@ -1,33 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out index ec93cf24cd4801..7551bb14d40e2b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.out @@ -46,36 +46,35 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------PhysicalProject -------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ---------------------------PhysicalOlapScan[date_dim] +------PhysicalTopN[GATHER_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +----------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out index 7992f57d1c87b3..15393efb582c73 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.out @@ -1,26 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) -----------------PhysicalWindow -------------------PhysicalQuickSort[LOCAL_SORT] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((item.i_manufact_id = 29)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +--------------PhysicalWindow +----------------PhysicalQuickSort[LOCAL_SORT] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((item.i_manufact_id = 29)) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out index d0200f9eaf0313..7710d09cf178ee 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.out @@ -1,50 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalIntersect -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------PhysicalIntersect +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out index 4ebfd5abc0eb1c..404a9e7e9e92a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.out @@ -1,33 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out index 7dfe8db4eb43a4..1bf8480c75247d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query23.out @@ -46,36 +46,35 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[item_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF6 ws_item_sk->[item_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalTopN[GATHER_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[item_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF6 ws_item_sk->[item_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out index 7992f57d1c87b3..15393efb582c73 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query32.out @@ -1,26 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) -----------------PhysicalWindow -------------------PhysicalQuickSort[LOCAL_SORT] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((item.i_manufact_id = 29)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +--------------PhysicalWindow +----------------PhysicalQuickSort[LOCAL_SORT] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((item.i_manufact_id = 29)) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out index e385d6e6157a32..ff996b20091063 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query38.out @@ -1,50 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalIntersect -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------PhysicalIntersect +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out index 4ebfd5abc0eb1c..404a9e7e9e92a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query97.out @@ -1,33 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out index 0c058225465a80..28e49b0a70c28d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out @@ -46,36 +46,35 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------filter(d_year IN (2000, 2001, 2002, 2003)) ----------------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[item_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF6 ws_item_sk->[item_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------PhysicalTopN[GATHER_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[item_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk)) otherCondition=() build RFs:RF6 ws_item_sk->[item_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out index 7992f57d1c87b3..15393efb582c73 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out @@ -1,26 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) -----------------PhysicalWindow -------------------PhysicalQuickSort[LOCAL_SORT] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((item.i_manufact_id = 29)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) -------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +--------------PhysicalWindow +----------------PhysicalQuickSort[LOCAL_SORT] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((item.i_manufact_id = 29)) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1999-04-07') and (date_dim.d_date >= '1999-01-07')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out index d0200f9eaf0313..7710d09cf178ee 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out @@ -1,50 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalIntersect -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalOlapScan[customer] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------PhysicalIntersect +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) +--------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out index 4ebfd5abc0eb1c..404a9e7e9e92a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out @@ -1,33 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- PhysicalResultSink ---PhysicalLimit[GLOBAL] -----PhysicalLimit[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecGather] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------PhysicalOlapScan[date_dim] +--PhysicalTopN[GATHER_SORT] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecGather] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[FULL_OUTER_JOIN colocated] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk)) otherCondition=() +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpch_p0/tpch/push_topn_to_agg.out b/regression-test/data/nereids_tpch_p0/tpch/push_topn_to_agg.out new file mode 100644 index 00000000000000..af01d68ffc6a31 --- /dev/null +++ b/regression-test/data/nereids_tpch_p0/tpch/push_topn_to_agg.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !stable_1 -- +0 +1 + +-- !stable_2 -- +0 +1 + +-- !stable_3 -- +0 +1 + diff --git a/regression-test/data/query_p0/cache/sql_cache.out b/regression-test/data/query_p0/cache/sql_cache.out index 1b3729294d123e..75f28261bc3432 100644 --- a/regression-test/data/query_p0/cache/sql_cache.out +++ b/regression-test/data/query_p0/cache/sql_cache.out @@ -1,50 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !sql_cache -- +-- !sql_cache1 -- 2022-05-28 0 2022-05-29 0 2022-05-30 0 2022-06-01 0 2022-06-02 0 --- !sql_cache -- +-- !sql_cache2 -- 2022-05-28 0 2022-05-29 0 2022-05-30 0 2022-06-01 0 2022-06-02 0 --- !sql_cache -- +-- !sql_cache3 -- 2022-05-28 0 2022-05-29 0 2022-05-30 0 2022-06-01 0 2022-06-02 0 --- !sql_cache -- +-- !sql_cache4 -- 2022-05-28 0 2022-05-28 0 --- !sql_cache -- +-- !sql_cache5 -- 2022-05-28 0 2022-05-28 0 --- !sql_cache -- +-- !sql_cache6 -- 2022-05-28 0 2022-05-29 0 2022-05-30 0 2022-06-01 0 2022-06-02 0 --- !sql_cache -- +-- !sql_cache7 -- 2022-05-28 0 2022-05-29 0 2022-05-30 0 2022-06-01 0 2022-06-02 0 --- !sql_cache -- +-- !sql_cache8 -- 2022-05-28 0 --- !sql_cache -- +-- !sql_cache9 -- 2022-05-28 0 diff --git a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy similarity index 81% rename from regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy rename to regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy index 779a2762e9bdb1..f53b100c29aff2 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.groovy @@ -16,11 +16,16 @@ // under the License. suite("push_down_limit_distinct_through_join") { - sql "SET enable_nereids_planner=true" - sql "set runtime_filter_mode=OFF" - sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "SET disable_join_reorder=true" + multi_sql """ + SET enable_nereids_planner=true; + SET enable_fallback_to_original_planner=false; + set runtime_filter_mode=OFF; + SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'; + SET ENABLE_NEREIDS_RULES=push_down_limit_distinct_through_join; + set push_topn_to_agg = true; + SET disable_join_reorder=true; + set topn_opt_limit_threshold=1024; + """ sql """ DROP TABLE IF EXISTS t; diff --git a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query23.groovy index e80a44e506443b..dbdafcc237395c 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query23.groovy @@ -19,19 +19,24 @@ suite("query23") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql 'set dump_nereids_memo=false' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """with frequent_ss_items as (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt diff --git a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query32.groovy b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query32.groovy index 52987f25301be8..5f46ae8b71566a 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query32.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query32.groovy @@ -19,19 +19,24 @@ suite("query32") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql 'set dump_nereids_memo=false' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """select sum(cs_ext_discount_amt) as "excess discount amount" from diff --git a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query38.groovy b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query38.groovy index f6baa01e589a17..d76741c55f7c75 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query38.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query38.groovy @@ -19,19 +19,24 @@ suite("query38") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql 'set dump_nereids_memo=false' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """select count(*) from ( select distinct c_last_name, c_first_name, d_date diff --git a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query97.groovy b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query97.groovy index 25cdb4a08c7cb5..0403b092dd14a5 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query97.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf1000_p0/shape/query97.groovy @@ -19,19 +19,24 @@ suite("query97") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql 'set dump_nereids_memo=false' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """with ssci as ( select ss_customer_sk customer_sk diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy index b3acaa39d1270b..307ac32cae9c6a 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/constraints/query23.groovy @@ -19,19 +19,25 @@ suite("query23") { String db = context.config.getDbNameByFile(context.file) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql "set enable_nereids_rules='ELIMINATE_GROUP_BY_KEY'" - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + set enable_nereids_rules='ELIMINATE_GROUP_BY_KEY'; + """ def ds = """with frequent_ss_items as (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.groovy index 2a5015733b5deb..e9142e8d28bd64 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query23.groovy @@ -19,20 +19,24 @@ suite("query23") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=true' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_23 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.groovy index 7cfedc0bd928d4..40a9e61e41e916 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query32.groovy @@ -19,20 +19,24 @@ suite("query32") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=true' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_32 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.groovy index f6e1a8d31556c5..aa4ec80803d85b 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query38.groovy @@ -19,20 +19,24 @@ suite("query38") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=true' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_38 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.groovy index 85c560b85458ef..940944738a3ccd 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query97.groovy @@ -19,20 +19,24 @@ suite("query97") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=true' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_97 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.groovy index 752068cc9e6e05..4333cebd4c8e41 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query23.groovy @@ -19,20 +19,24 @@ suite("query23") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=false' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_23 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.groovy index 95ea92340716a3..043d9bfcece5ec 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query32.groovy @@ -19,20 +19,24 @@ suite("query32") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=false' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_32 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.groovy index 9e848042cc806a..8f94be0dafdf8b 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query38.groovy @@ -19,20 +19,24 @@ suite("query38") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=false' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_38 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.groovy index d10559588f2bae..8361a61db55d7d 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query97.groovy @@ -19,20 +19,24 @@ suite("query97") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' -sql 'set enable_runtime_filter_prune=false' - sql 'set parallel_pipeline_task_num=8' - sql 'set forbid_unknown_col_stats=false' - sql 'set enable_stats=false' - sql "set runtime_filter_type=8" - sql 'set broadcast_row_count_limit = 30000000' - sql 'set enable_nereids_timeout = false' - sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=false; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ qt_ds_shape_97 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query23.groovy index e4936149328fc3..ed702a8727806a 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query23.groovy @@ -19,18 +19,24 @@ suite("query23") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set runtime_filter_type=8' - sql 'set enable_runtime_filter_prune=true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """with frequent_ss_items as diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query32.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query32.groovy index 76df2cfd5934dd..ea40d98abd5df2 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query32.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query32.groovy @@ -19,18 +19,24 @@ suite("query32") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set runtime_filter_type=8' - sql 'set enable_runtime_filter_prune=true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """select sum(cs_ext_discount_amt) as "excess discount amount" diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query38.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query38.groovy index f82bd500586d89..b2cfe113e7ad58 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query38.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query38.groovy @@ -19,18 +19,24 @@ suite("query38") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set runtime_filter_type=8' - sql 'set enable_runtime_filter_prune=true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """select count(*) from ( select distinct c_last_name, c_first_name, d_date diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query97.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query97.groovy index d1eca3690e51de..29c68a33c871cf 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query97.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf_prune/query97.groovy @@ -19,18 +19,24 @@ suite("query97") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set runtime_filter_type=8' - sql 'set enable_runtime_filter_prune=true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=true; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """with ssci as ( select ss_customer_sk customer_sk diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query23.groovy index 58f2d4de7fc1d3..82b9d4d9d5429a 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query23.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query23.groovy @@ -19,18 +19,24 @@ suite("query23") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """with frequent_ss_items as (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query32.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query32.groovy index a53ed716caf2ba..20a5cc1dd76ece 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query32.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query32.groovy @@ -19,18 +19,24 @@ suite("query32") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """select sum(cs_ext_discount_amt) as "excess discount amount" from diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query38.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query38.groovy index d0f30bfc495dd0..b08b923b6c9096 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query38.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query38.groovy @@ -19,18 +19,24 @@ suite("query38") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """select count(*) from ( select distinct c_last_name, c_first_name, d_date diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query97.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query97.groovy index 60ca854add06fc..74ba09b5696852 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query97.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/shape/query97.groovy @@ -19,18 +19,24 @@ suite("query97") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'set be_number_for_test=3' - sql 'set parallel_fragment_exec_instance_num=8; ' - sql 'set parallel_pipeline_task_num=8; ' - sql 'set forbid_unknown_col_stats=true' - sql 'set enable_nereids_timeout = false' - sql 'set enable_runtime_filter_prune=false' - sql 'set runtime_filter_type=8' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set exec_mem_limit=21G; + set be_number_for_test=3; + set enable_runtime_filter_prune=false; + set parallel_pipeline_task_num=8; + set forbid_unknown_col_stats=false; + set enable_stats=true; + set runtime_filter_type=8; + set broadcast_row_count_limit = 30000000; + set enable_nereids_timeout = false; + set enable_pipeline_engine = true; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set push_topn_to_agg = true; + set topn_opt_limit_threshold=1024; + """ def ds = """with ssci as ( select ss_customer_sk customer_sk diff --git a/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy b/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy new file mode 100644 index 00000000000000..36460f06e32d56 --- /dev/null +++ b/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy @@ -0,0 +1,169 @@ +/* + * 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("push_topn_to_agg") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql "set topn_opt_limit_threshold=1024" + // limit -> agg + // verify switch + sql "set push_topn_to_agg=false" + explain{ + sql "select o_custkey, sum(o_shippriority) from orders group by o_custkey limit 4;" + multiContains ("sortByGroupKey:false", 2) + } + sql "set push_topn_to_agg=true" + explain{ + sql "select o_custkey, sum(o_shippriority) from orders group by o_custkey limit 4;" + multiContains ("sortByGroupKey:true", 2) + notContains("STREAMING") + } + + // when apply this opt, trun off STREAMING + // limit -> proj -> agg, + explain{ + sql "select sum(c_custkey), c_name from customer group by c_name limit 6;" + multiContains ("sortByGroupKey:true", 2) + notContains("STREAMING") + } + + // topn -> agg + explain{ + sql "select o_custkey, sum(o_shippriority) from orders group by o_custkey order by o_custkey limit 8;" + multiContains ("sortByGroupKey:true", 2) + notContains("STREAMING") + } + + // order key should be prefix of group key + explain{ + sql "select o_custkey, sum(o_shippriority), o_clerk from orders group by o_custkey, o_clerk order by o_clerk, o_custkey limit 11;" + multiContains("sortByGroupKey:false", 2) + } + + // order key should be prefix of group key + explain{ + sql "select o_custkey, o_clerk, sum(o_shippriority) as x from orders group by o_custkey, o_clerk order by o_custkey, x limit 12;" + multiContains("sortByGroupKey:false", 2) + } + + // one phase agg is optimized + explain { + sql "select sum(o_shippriority) from orders group by o_orderkey limit 13; " + contains("sortByGroupKey:true") + } + + // group key is not output of limit, deny opt + explain { + sql "select sum(o_shippriority) from orders group by o_clerk limit 14; " + contains("sortByGroupKey:false") + } + + // group key is part of output of limit, apply opt + explain { + sql "select sum(o_shippriority), o_clerk from orders group by o_clerk limit 15; " + contains("sortByGroupKey:true") + } + + // order key is not prefix of group key + explain { + sql "select o_custkey, sum(o_shippriority) from orders group by o_custkey order by o_custkey+1 limit 16; " + contains("sortByGroupKey:false") + } + + // order key is not prefix of group key + explain { + sql "select o_custkey, sum(o_shippriority) from orders group by o_custkey order by o_custkey+1 limit 17; " + contains("sortByGroupKey:false") + } + + // topn + one phase agg + explain { + sql "select sum(ps_availqty), ps_partkey, ps_suppkey from partsupp group by ps_partkey, ps_suppkey order by ps_partkey, ps_suppkey limit 18;" + contains("sortByGroupKey:true") + } + + // sort key is prefix of group key, make all group key to sort key(ps_suppkey) and then apply push-topn-agg rule + explain { + sql "select sum(ps_availqty), ps_partkey, ps_suppkey from partsupp group by ps_partkey, ps_suppkey order by ps_partkey limit 19;" + contains("sortByGroupKey:true") + } + + explain { + sql "select sum(ps_availqty), ps_suppkey, ps_availqty from partsupp group by ps_suppkey, ps_availqty order by ps_suppkey limit 19;" + contains("sortByGroupKey:true") + } + + // sort key is not prefix of group key, deny + explain { + sql "select sum(ps_availqty), ps_partkey, ps_suppkey from partsupp group by ps_partkey, ps_suppkey order by ps_suppkey limit 20;" + contains("sortByGroupKey:false") + } + + multi_sql """ + drop table if exists t1; + CREATE TABLE IF NOT EXISTS t1 + ( + k1 TINYINT + ) + ENGINE=olap + AGGREGATE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + + insert into t1 values (0),(1); + + drop table if exists t2; + CREATE TABLE IF NOT EXISTS t2 + ( + k1 TINYINT + ) + ENGINE=olap + AGGREGATE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + insert into t2 values(5),(6); + """ + + // the result of following sql may be unstable, run 3 times + qt_stable_1 """ + select * from ( + select k1 from t1 + UNION + select k1 from t2 + ) as b order by k1 limit 2; + """ + qt_stable_2 """ + select * from ( + select k1 from t1 + UNION + select k1 from t2 + ) as b order by k1 limit 2; + """ + qt_stable_3 """ + select * from ( + select k1 from t1 + UNION + select k1 from t2 + ) as b order by k1 limit 2; + """ +} \ No newline at end of file diff --git a/regression-test/suites/nereids_tpch_p0/tpch/topn-filter.groovy b/regression-test/suites/nereids_tpch_p0/tpch/topn-filter.groovy index 9bee2140098a06..39dbf185ddacfc 100644 --- a/regression-test/suites/nereids_tpch_p0/tpch/topn-filter.groovy +++ b/regression-test/suites/nereids_tpch_p0/tpch/topn-filter.groovy @@ -19,11 +19,14 @@ suite("topn-filter") { String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set disable_join_reorder=true;' - sql 'set topn_opt_limit_threshold=1024' + multi_sql """ + use ${db}; + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + set disable_join_reorder=true; + set topn_opt_limit_threshold=1024; + set push_topn_to_agg=false; + """ def String simpleTopn = """ select * from orders diff --git a/regression-test/suites/query_p0/cache/sql_cache.groovy b/regression-test/suites/query_p0/cache/sql_cache.groovy index d4b16cb5438fd2..c38c0800740e2e 100644 --- a/regression-test/suites/query_p0/cache/sql_cache.groovy +++ b/regression-test/suites/query_p0/cache/sql_cache.groovy @@ -55,7 +55,7 @@ suite("sql_cache") { ("2022-06-02",0) """ - qt_sql_cache """ + qt_sql_cache1 """ select k1, sum(k2) as total_pv @@ -71,7 +71,7 @@ suite("sql_cache") { sql "set enable_sql_cache=true " - qt_sql_cache """ + qt_sql_cache2 """ select k1, sum(k2) as total_pv @@ -84,7 +84,7 @@ suite("sql_cache") { order by k1; """ - qt_sql_cache """ + qt_sql_cache3 """ select k1, sum(k2) as total_pv @@ -98,7 +98,7 @@ suite("sql_cache") { k1; """ - qt_sql_cache """ + qt_sql_cache4 """ select k1, sum(k2) as total_pv @@ -124,7 +124,7 @@ suite("sql_cache") { k1; """ - qt_sql_cache """ + qt_sql_cache5 """ select k1, sum(k2) as total_pv @@ -153,7 +153,7 @@ suite("sql_cache") { sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" - qt_sql_cache """ + qt_sql_cache6 """ select k1, sum(k2) as total_pv @@ -166,7 +166,7 @@ suite("sql_cache") { order by k1; """ - qt_sql_cache """ + qt_sql_cache7 """ select k1, sum(k2) as total_pv @@ -183,7 +183,7 @@ suite("sql_cache") { sql 'set default_order_by_limit = 2' sql 'set sql_select_limit = 1' - qt_sql_cache """ + qt_sql_cache8 """ select k1, sum(k2) as total_pv @@ -197,7 +197,7 @@ suite("sql_cache") { k1; """ - qt_sql_cache """ + qt_sql_cache9 """ select k1, sum(k2) as total_pv