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 ed0fa310228a70..9e3186c6cec7b2 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 @@ -60,6 +60,7 @@ import org.apache.doris.nereids.rules.rewrite.CountDistinctRewrite; import org.apache.doris.nereids.rules.rewrite.CountLiteralRewrite; import org.apache.doris.nereids.rules.rewrite.CreatePartitionTopNFromWindow; +import org.apache.doris.nereids.rules.rewrite.DecomposeRepeatWithPreAggregation; import org.apache.doris.nereids.rules.rewrite.DecoupleEncodeDecode; import org.apache.doris.nereids.rules.rewrite.DeferMaterializeTopNResult; import org.apache.doris.nereids.rules.rewrite.DistinctAggStrategySelector; @@ -902,7 +903,8 @@ private static List getWholeTreeRewriteJobs( rewriteJobs.addAll(jobs(topic("or expansion", custom(RuleType.OR_EXPANSION, () -> OrExpansion.INSTANCE)))); } - + rewriteJobs.add(topic("repeat rewrite", + custom(RuleType.DECOMPOSE_REPEAT, () -> DecomposeRepeatWithPreAggregation.INSTANCE))); rewriteJobs.addAll(jobs(topic("split multi distinct", custom(RuleType.DISTINCT_AGG_STRATEGY_SELECTOR, () -> DistinctAggStrategySelector.INSTANCE)))); 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 52fd3c0cc3aab5..21f1ae4bdbf46b 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 @@ -414,7 +414,7 @@ public enum RuleType { SHOR_CIRCUIT_POINT_QUERY(RuleTypeClass.REWRITE), // skew rewrtie SALT_JOIN(RuleTypeClass.REWRITE), - + DECOMPOSE_REPEAT(RuleTypeClass.REWRITE), DISTINCT_AGGREGATE_SPLIT(RuleTypeClass.REWRITE), PROCESS_SCALAR_AGG_MUST_USE_MULTI_DISTINCT(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DecomposeRepeatWithPreAggregation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DecomposeRepeatWithPreAggregation.java new file mode 100644 index 00000000000000..6f6a7f373a5634 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DecomposeRepeatWithPreAggregation.java @@ -0,0 +1,513 @@ +// 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.jobs.JobContext; +import org.apache.doris.nereids.rules.rewrite.DistinctAggStrategySelector.DistinctSelectorContext; +import org.apache.doris.nereids.trees.copier.DeepCopierContext; +import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Min; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum0; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * This rule will rewrite grouping sets. eg: + * select a, b, c, d, e sum(f) from t group by rollup(a, b, c, d, e); + * rewrite to: + * with cte1 as (select a, b, c, d, e, sum(f) x from t group by rollup(a, b, c, d, e)) + * select * fom cte1 + * union all + * select a, b, c, d, null, sum(x) x from t group by rollup(a, b, c, d) + * + * LogicalAggregate(gby: a,b,c,d,e,grouping_id output:a,b,c,d,e,grouping_id,sum(f)) + * +--LogicalRepeat(grouping sets: (a,b,c,d,e),(a,b,c,d),(a,b,c),(a,b),(a),()) + * -> + * LogicalCTEAnchor + * +--LogicalCTEProducer(cte) + * +--LogicalAggregate(gby: a,b,c,d,e; aggFunc: sum(f) as x) + * +--LogicalUnionAll + * +--LogicalProject(a,b,c,d, null as e, sum(x)) + * +--LogicalAggregate(gby:a,b,c,d,grouping_id; aggFunc: sum(x)) + * +--LogicalRepeat(grouping sets: (a,b,c,d),(a,b,c),(a,b),(a),()) + * +--LogicalCTEConsumer(aggregateConsumer) + * +--LogicalCTEConsumer(directConsumer) + */ +public class DecomposeRepeatWithPreAggregation extends DefaultPlanRewriter + implements CustomRewriter { + public static final DecomposeRepeatWithPreAggregation INSTANCE = new DecomposeRepeatWithPreAggregation(); + private static final Set> SUPPORT_AGG_FUNCTIONS = + ImmutableSet.of(Sum.class, Sum0.class, Min.class, Max.class, AnyValue.class, Count.class); + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + DistinctSelectorContext ctx = new DistinctSelectorContext(jobContext.getCascadesContext().getStatementContext(), + jobContext.getCascadesContext()); + plan = plan.accept(this, ctx); + for (int i = ctx.cteProducerList.size() - 1; i >= 0; i--) { + LogicalCTEProducer producer = ctx.cteProducerList.get(i); + plan = new LogicalCTEAnchor<>(producer.getCteId(), producer, plan); + } + return plan; + } + + @Override + public Plan visitLogicalCTEAnchor( + LogicalCTEAnchor anchor, DistinctSelectorContext ctx) { + Plan child1 = anchor.child(0).accept(this, ctx); + DistinctSelectorContext consumerContext = + new DistinctSelectorContext(ctx.statementContext, ctx.cascadesContext); + Plan child2 = anchor.child(1).accept(this, consumerContext); + for (int i = consumerContext.cteProducerList.size() - 1; i >= 0; i--) { + LogicalCTEProducer producer = consumerContext.cteProducerList.get(i); + child2 = new LogicalCTEAnchor<>(producer.getCteId(), producer, child2); + } + return anchor.withChildren(ImmutableList.of(child1, child2)); + } + + @Override + public Plan visitLogicalAggregate(LogicalAggregate aggregate, DistinctSelectorContext ctx) { + aggregate = visitChildren(this, aggregate, ctx); + int maxGroupIndex = canOptimize(aggregate); + if (maxGroupIndex < 0) { + return aggregate; + } + Map preToProducerSlotMap = new HashMap<>(); + LogicalCTEProducer> producer = constructProducer(aggregate, maxGroupIndex, ctx, + preToProducerSlotMap); + LogicalCTEConsumer aggregateConsumer = new LogicalCTEConsumer(ctx.statementContext.getNextRelationId(), + producer.getCteId(), "", producer); + LogicalCTEConsumer directConsumer = new LogicalCTEConsumer(ctx.statementContext.getNextRelationId(), + producer.getCteId(), "", producer); + + // build map : origin slot to consumer slot + Map producerToConsumerMap = new HashMap<>(); + for (Map.Entry entry : aggregateConsumer.getProducerToConsumerOutputMap().entries()) { + producerToConsumerMap.put(entry.getKey(), entry.getValue()); + } + Map originToConsumerMap = new HashMap<>(); + for (Map.Entry entry : preToProducerSlotMap.entrySet()) { + originToConsumerMap.put(entry.getKey(), producerToConsumerMap.get(entry.getValue())); + } + + LogicalRepeat repeat = (LogicalRepeat) aggregate.child(); + List> newGroupingSets = new ArrayList<>(); + for (int i = 0; i < repeat.getGroupingSets().size(); ++i) { + if (i == maxGroupIndex) { + continue; + } + newGroupingSets.add(repeat.getGroupingSets().get(i)); + } + List groupingFunctionSlots = new ArrayList<>(); + LogicalRepeat newRepeat = constructRepeat(repeat, aggregateConsumer, newGroupingSets, + originToConsumerMap, groupingFunctionSlots); + Set needRemovedExprSet = getNeedAddNullExpressions(repeat, newGroupingSets, maxGroupIndex); + Map aggFuncToSlot = new HashMap<>(); + LogicalAggregate topAgg = constructAgg(aggregate, originToConsumerMap, newRepeat, groupingFunctionSlots, + aggFuncToSlot); + LogicalProject project = constructProject(aggregate, originToConsumerMap, needRemovedExprSet, + groupingFunctionSlots, topAgg, aggFuncToSlot); + LogicalPlan directChild = getDirectChild(directConsumer, groupingFunctionSlots); + return constructUnion(project, directChild, aggregate); + } + + /** + * Get the direct child plan for the union operation. + * If there are grouping function slots, wrap the consumer with a project that adds + * zero literals for each grouping function slot to match the output schema. + * + * @param directConsumer the CTE consumer for the direct path + * @param groupingFunctionSlots the list of grouping function slots to handle + * @return the direct child plan, possibly wrapped with a project + */ + private LogicalPlan getDirectChild(LogicalCTEConsumer directConsumer, List groupingFunctionSlots) { + LogicalPlan directChild = directConsumer; + if (!groupingFunctionSlots.isEmpty()) { + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(directConsumer.getOutput()); + for (int i = 0; i < groupingFunctionSlots.size(); ++i) { + builder.add(new Alias(new BigIntLiteral(0))); + } + directChild = new LogicalProject(builder.build(), directConsumer); + } + return directChild; + } + + /** + * Build a map from aggregate function to its corresponding slot. + * + * @param outputExpressions the output expressions containing aggregate functions + * @param pToc the map from producer slot to consumer slot + * @return a map from aggregate function to its corresponding slot in consumer outputs + */ + private Map getAggFuncSlotMap(List outputExpressions, + Map pToc) { + // build map : aggFunc to Slot + Map aggFuncSlotMap = new HashMap<>(); + for (NamedExpression expr : outputExpressions) { + if (expr instanceof Alias) { + Optional aggFunc = expr.child(0).collectFirst(e -> e instanceof AggregateFunction); + aggFunc.ifPresent( + func -> aggFuncSlotMap.put((AggregateFunction) func, pToc.get(expr.toSlot()))); + } + } + return aggFuncSlotMap; + } + + /** + * Get the set of expressions that need to be replaced with null in the new grouping sets. + * These are expressions that exist in the maximum grouping set but not in other grouping sets. + * + * @param repeat the original LogicalRepeat plan + * @param newGroupingSets the new grouping sets after removing the maximum grouping set + * @param maxGroupIndex the index of the maximum grouping set + * @return the set of expressions that need to be replaced with null + */ + private Set getNeedAddNullExpressions(LogicalRepeat repeat, + List> newGroupingSets, int maxGroupIndex) { + Set otherGroupExprSet = new HashSet<>(); + for (List groupingSet : newGroupingSets) { + otherGroupExprSet.addAll(groupingSet); + } + List maxGroupByList = repeat.getGroupingSets().get(maxGroupIndex); + Set needRemovedExprSet = new HashSet<>(maxGroupByList); + needRemovedExprSet.removeAll(otherGroupExprSet); + return needRemovedExprSet; + } + + /** + * Construct a LogicalAggregate for the decomposed repeat. + * + * @param aggregate the original aggregate plan + * @param originToConsumerMap the map from original slots to consumer slots + * @param newRepeat the new LogicalRepeat plan with reduced grouping sets + * @param groupingFunctionSlots the list of new grouping function slots + * @param aggFuncToSlot output parameter: map from original aggregate functions to their slots in the new aggregate + * @return a LogicalAggregate for the decomposed repeat + */ + private LogicalAggregate constructAgg(LogicalAggregate aggregate, + Map originToConsumerMap, LogicalRepeat newRepeat, + List groupingFunctionSlots, Map aggFuncToSlot) { + Map aggFuncSlotMap = getAggFuncSlotMap(aggregate.getOutputExpressions(), + originToConsumerMap); + Set groupingSetsUsedSlot = ImmutableSet.copyOf( + ExpressionUtils.flatExpressions((List) newRepeat.getGroupingSets())); + List topAggGby = new ArrayList<>(groupingSetsUsedSlot); + topAggGby.add(newRepeat.getGroupingId().get()); + topAggGby.addAll(groupingFunctionSlots); + List topAggOutput = new ArrayList<>((List) topAggGby); + for (NamedExpression expr : aggregate.getOutputExpressions()) { + if (expr instanceof Alias && expr.containsType(AggregateFunction.class)) { + NamedExpression aggFuncAfterRewrite = (NamedExpression) expr.rewriteDownShortCircuit(e -> { + if (e instanceof AggregateFunction) { + if (e instanceof Count) { + return new Sum(aggFuncSlotMap.get(e)); + } else { + return e.withChildren(aggFuncSlotMap.get(e)); + } + } else { + return e; + } + }); + aggFuncAfterRewrite = ((Alias) aggFuncAfterRewrite) + .withExprId(StatementScopeIdGenerator.newExprId()); + NamedExpression replacedExpr = (NamedExpression) aggFuncAfterRewrite.rewriteDownShortCircuit( + e -> { + if (originToConsumerMap.containsKey(e)) { + return originToConsumerMap.get(e); + } else { + return e; + } + } + ); + topAggOutput.add(replacedExpr); + aggFuncToSlot.put((AggregateFunction) expr.collectFirst(e -> e instanceof AggregateFunction).get(), + replacedExpr.toSlot()); + } + } + return new LogicalAggregate<>(topAggGby, topAggOutput, Optional.of(newRepeat), newRepeat); + } + + /** + * Construct a LogicalProject that wraps the aggregate and handles output expressions. + * This method replaces removed expressions with null literals, and output the grouping scalar functions + * at the end of the projections. + * + * @param aggregate the original aggregate plan + * @param originToConsumerMap the map from original slots to consumer slots + * @param needRemovedExprSet the set of expressions that need to be replaced with null + * @param groupingFunctionSlots the list of grouping function slots to add to the project + * @param topAgg the aggregate plan to wrap + * @param aggFuncToSlot the map from aggregate functions to their slots + * @return a LogicalProject wrapping the aggregate with proper output expressions + */ + private LogicalProject constructProject(LogicalAggregate aggregate, + Map originToConsumerMap, Set needRemovedExprSet, + List groupingFunctionSlots, LogicalAggregate topAgg, + Map aggFuncToSlot) { + LogicalRepeat repeat = (LogicalRepeat) aggregate.child(0); + Set originGroupingFunctionId = new HashSet<>(); + for (NamedExpression namedExpression : repeat.getGroupingScalarFunctionAlias()) { + originGroupingFunctionId.add(namedExpression.getExprId()); + } + ImmutableList.Builder projects = ImmutableList.builder(); + for (NamedExpression expr : aggregate.getOutputExpressions()) { + if (needRemovedExprSet.contains(expr)) { + projects.add(new Alias(new NullLiteral(expr.getDataType()), expr.getName())); + } else if (expr instanceof Alias && expr.containsType(AggregateFunction.class)) { + AggregateFunction aggregateFunction = (AggregateFunction) expr.collectFirst( + e -> e instanceof AggregateFunction).get(); + projects.add(aggFuncToSlot.get(aggregateFunction)); + } else if (expr.getExprId().equals(repeat.getGroupingId().get().getExprId()) + || originGroupingFunctionId.contains(expr.getExprId())) { + continue; + } else { + NamedExpression replacedExpr = (NamedExpression) expr.rewriteDownShortCircuit( + e -> { + if (originToConsumerMap.containsKey(e)) { + return originToConsumerMap.get(e); + } else { + return e; + } + } + ); + projects.add(replacedExpr.toSlot()); + } + } + projects.addAll(groupingFunctionSlots); + return new LogicalProject<>(projects.build(), topAgg); + } + + /** + * Construct a LogicalUnion that combines the results from the decomposed repeat + * and the CTE consumer. + * + * @param aggregateProject the first child plan (project with aggregate) + * @param directConsumer the second child plan (CTE consumer) + * @param aggregate the original aggregate plan for output reference + * @return a LogicalUnion combining the two children + */ + private LogicalUnion constructUnion(LogicalPlan aggregateProject, LogicalPlan directConsumer, + LogicalAggregate aggregate) { + LogicalRepeat repeat = (LogicalRepeat) aggregate.child(); + List unionOutputs = new ArrayList<>(); + List> childrenOutputs = new ArrayList<>(); + childrenOutputs.add((List) aggregateProject.getOutput()); + childrenOutputs.add((List) directConsumer.getOutput()); + Set groupingFunctionId = new HashSet<>(); + for (NamedExpression alias : repeat.getGroupingScalarFunctionAlias()) { + groupingFunctionId.add(alias.getExprId()); + } + List groupingFunctionSlots = new ArrayList<>(); + for (NamedExpression expr : aggregate.getOutputExpressions()) { + if (expr.getExprId().equals(repeat.getGroupingId().get().getExprId())) { + continue; + } + if (groupingFunctionId.contains(expr.getExprId())) { + groupingFunctionSlots.add(expr.toSlot()); + continue; + } + unionOutputs.add(expr.toSlot()); + } + unionOutputs.addAll(groupingFunctionSlots); + return new LogicalUnion(Qualifier.ALL, unionOutputs, childrenOutputs, ImmutableList.of(), + false, ImmutableList.of(aggregateProject, directConsumer)); + } + + /** + * Determine if optimization is possible; if so, return the index of the largest group. + * The optimization requires: + * 1. The aggregate's child must be a LogicalRepeat + * 2. All aggregate functions must be Sum, Min, or Max (non-distinct) + * 3. No GroupingScalarFunction in repeat output + * 4. More than 3 grouping sets + * 5. There exists a grouping set that contains all other grouping sets + * + * @param aggregate the aggregate plan to check + * @return value -1 means can not be optimized, values other than -1 + * represent the index of the set that contains all other sets + */ + private int canOptimize(LogicalAggregate aggregate) { + Plan aggChild = aggregate.child(); + if (!(aggChild instanceof LogicalRepeat)) { + return -1; + } + // check agg func + Set aggFunctions = aggregate.getAggregateFunctions(); + for (AggregateFunction aggFunction : aggFunctions) { + if (!SUPPORT_AGG_FUNCTIONS.contains(aggFunction.getClass())) { + return -1; + } + if (aggFunction.isDistinct()) { + return -1; + } + } + LogicalRepeat repeat = (LogicalRepeat) aggChild; + List> groupingSets = repeat.getGroupingSets(); + // This is an empirical threshold: when there are too few grouping sets, + // the overhead of creating CTE and union may outweigh the benefits. + // The value 3 is chosen heuristically based on practical experience. + if (groupingSets.size() <= 3) { + return -1; + } + return findMaxGroupingSetIndex(groupingSets); + } + + /** + * Find the index of the grouping set that contains all other grouping sets. + * First pass: find the grouping set with the maximum size (if multiple have the same size, take the first one). + * Second pass: verify that this max-size grouping set contains all other grouping sets. + * A grouping set A contains grouping set B if A contains all expressions in B. + * + * @param groupingSets the list of grouping sets to search + * @return the index of the grouping set that contains all others, or -1 if no such set exists + */ + private int findMaxGroupingSetIndex(List> groupingSets) { + if (groupingSets.isEmpty()) { + return -1; + } + // First pass: find the grouping set with maximum size + int maxSize = groupingSets.get(0).size(); + int maxGroupIndex = 0; + for (int i = 1; i < groupingSets.size(); ++i) { + if (groupingSets.get(i).size() > maxSize) { + maxSize = groupingSets.get(i).size(); + maxGroupIndex = i; + } + } + // Second pass: verify that the max-size grouping set contains all other grouping sets + ImmutableSet maxGroup = ImmutableSet.copyOf(groupingSets.get(maxGroupIndex)); + for (int i = 0; i < groupingSets.size(); ++i) { + if (i == maxGroupIndex) { + continue; + } + if (!maxGroup.containsAll(groupingSets.get(i))) { + return -1; + } + } + return maxGroupIndex; + } + + /** + * Construct a LogicalCTEProducer that pre-aggregates data using the maximum grouping set. + * This producer will be used by consumers to avoid recomputing the same aggregation. + * + * @param aggregate the original aggregate plan + * @param maxGroupIndex the index of the maximum grouping set + * @param ctx context + * @param preToCloneSlotMap output parameter: map from pre-aggregate slots to cloned slots + * @return a LogicalCTEProducer containing the pre-aggregation + */ + private LogicalCTEProducer> constructProducer(LogicalAggregate aggregate, + int maxGroupIndex, DistinctSelectorContext ctx, Map preToCloneSlotMap) { + LogicalRepeat repeat = (LogicalRepeat) aggregate.child(); + List maxGroupByList = repeat.getGroupingSets().get(maxGroupIndex); + List originAggOutputs = aggregate.getOutputExpressions(); + Set preAggOutputSet = new HashSet((List) maxGroupByList); + for (NamedExpression aggOutput : originAggOutputs) { + if (aggOutput.containsType(AggregateFunction.class)) { + preAggOutputSet.add(aggOutput); + } + } + List orderedAggOutputs = new ArrayList<>(); + // keep order + for (NamedExpression aggOutput : originAggOutputs) { + if (preAggOutputSet.contains(aggOutput)) { + orderedAggOutputs.add(aggOutput); + } + } + + LogicalAggregate preAgg = new LogicalAggregate<>(maxGroupByList, orderedAggOutputs, repeat.child()); + LogicalAggregate preAggClone = (LogicalAggregate) LogicalPlanDeepCopier.INSTANCE + .deepCopy(preAgg, new DeepCopierContext()); + for (int i = 0; i < preAgg.getOutputExpressions().size(); ++i) { + preToCloneSlotMap.put(preAgg.getOutput().get(i), preAggClone.getOutput().get(i)); + } + LogicalCTEProducer> producer = + new LogicalCTEProducer<>(ctx.statementContext.getNextCTEId(), preAggClone); + ctx.cteProducerList.add(producer); + return producer; + } + + /** + * Construct a new LogicalRepeat with reduced grouping sets and replaced expressions. + * The grouping sets and output expressions are replaced using the slot mapping from producer to consumer. + * + * @param repeat the original LogicalRepeat plan + * @param child the child plan (usually a CTE consumer) + * @param newGroupingSets the new grouping sets after removing the maximum grouping set + * @param producerToDirectConsumerSlotMap the map from producer slots to consumer slots + * @return a new LogicalRepeat with replaced expressions + */ + private LogicalRepeat constructRepeat(LogicalRepeat repeat, LogicalPlan child, + List> newGroupingSets, Map producerToDirectConsumerSlotMap, + List groupingFunctionSlots) { + List> replacedNewGroupingSets = new ArrayList<>(); + for (List groupingSet : newGroupingSets) { + replacedNewGroupingSets.add(ExpressionUtils.replace(groupingSet, producerToDirectConsumerSlotMap)); + } + List replacedRepeatOutputs = new ArrayList<>(child.getOutput()); + List newGroupingFunctions = new ArrayList<>(); + for (NamedExpression groupingFunction : repeat.getGroupingScalarFunctionAlias()) { + newGroupingFunctions.add(new Alias(groupingFunction.child(0), groupingFunction.getName())); + } + replacedRepeatOutputs.addAll(ExpressionUtils.replace((List) newGroupingFunctions, + producerToDirectConsumerSlotMap)); + for (NamedExpression groupingFunction : newGroupingFunctions) { + groupingFunctionSlots.add(groupingFunction.toSlot()); + } + return repeat.withNormalizedExpr(replacedNewGroupingSets, replacedRepeatOutputs, + repeat.getGroupingId().get(), child); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Alias.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Alias.java index 0650a7cbf86f05..cc0ed1d2447cae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Alias.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Alias.java @@ -171,6 +171,10 @@ public Alias withChildren(List children) { return new Alias(exprId, children, name, qualifier, nameFromChild); } + public Alias withExprId(ExprId exprId) { + return new Alias(exprId, children, name, qualifier, nameFromChild); + } + public R accept(ExpressionVisitor visitor, C context) { return visitor.visitAlias(this, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java index 5a099f5a4ad0b3..a7ab86de4fbd44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java @@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.scalar.GroupingScalarFunction; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.algebra.Repeat; @@ -34,6 +35,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Objects; @@ -125,6 +127,19 @@ public List getOutputs() { return outputExpressions; } + /** + * get groupingScalarFunction with Alias + */ + public List getGroupingScalarFunctionAlias() { + List functionList = new ArrayList<>(); + for (NamedExpression outputExpression : outputExpressions) { + if (outputExpression.containsType(GroupingScalarFunction.class)) { + functionList.add(outputExpression); + } + } + return functionList; + } + @Override public String toString() { return Utils.toSqlString("LogicalRepeat", diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRepeat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRepeat.java index 7b841ac1fc6e4e..1160db440da5f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRepeat.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRepeat.java @@ -39,6 +39,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.stream.Stream; /** * PhysicalRepeat. @@ -117,7 +118,7 @@ public String toString() { @Override public List computeOutput() { - return outputExpressions.stream() + return Stream.concat(outputExpressions.stream(), Stream.of(groupingId)) .map(NamedExpression::toSlot) .collect(ImmutableList.toImmutableList()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PreMaterializedViewRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PreMaterializedViewRewriterTest.java index 11fc0a27de8b9f..214f5d9a7f6421 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PreMaterializedViewRewriterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PreMaterializedViewRewriterTest.java @@ -2977,6 +2977,7 @@ private void checkIfEquals(String originalSql, List equivalentSqlList) { private CascadesContext initOriginal(String sql) { CascadesContext cascadesContext = createCascadesContext(sql, connectContext); + connectContext.setThreadLocalInfo(); PlanChecker.from(cascadesContext).analyze().rewrite(); return cascadesContext; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/DecomposeRepeatWithPreAggregationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/DecomposeRepeatWithPreAggregationTest.java new file mode 100644 index 00000000000000..c78394ce9ef141 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/DecomposeRepeatWithPreAggregationTest.java @@ -0,0 +1,485 @@ +// 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.rules.rewrite.DistinctAggStrategySelector.DistinctSelectorContext; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +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.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * UT for {@link DecomposeRepeatWithPreAggregation}. + */ +public class DecomposeRepeatWithPreAggregationTest extends TestWithFeService implements MemoPatternMatchSupported { + private DecomposeRepeatWithPreAggregation rule; + private DistinctSelectorContext ctx; + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("decompose_repeat_with_preagg"); + createTable( + "create table decompose_repeat_with_preagg.t1 (\n" + + "a int, b int, c int, d int\n" + + ")\n" + + "distributed by hash(a) buckets 1\n" + + "properties('replication_num' = '1');" + ); + connectContext.setDatabase("default_cluster:decompose_repeat_with_preagg"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + rule = DecomposeRepeatWithPreAggregation.INSTANCE; + ctx = new DistinctSelectorContext( + MemoTestUtils.createCascadesContext( + new LogicalEmptyRelation(org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of())).getStatementContext(), + MemoTestUtils.createCascadesContext( + new LogicalEmptyRelation(org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of()))); + } + + @Test + void rewriteRollupSumShouldGenerateCteAndUnion() { + String sql = "select a,b,c,sum(d) from t1 group by rollup(a,b,c);"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalCTEAnchor()); + } + + @Test + void noRewriteWhenGroupingSetsSizeLe3() { + String sql = "select a,b,sum(d) from t1 group by rollup(a,b);"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .nonMatch(logicalCTEAnchor()); + } + + @Test + void noRewriteWhenDistinctAgg() { + String sql = "select a,b,c,sum(distinct d) from t1 group by rollup(a,b,c);"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .nonMatch(logicalCTEAnchor()); + } + + @Test + void noRewriteWhenUnsupportedAgg() { + String sql = "select a,b,c,avg(d) from t1 group by rollup(a,b,c);"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .nonMatch(logicalCTEAnchor()); + + } + + @Test + void noRewriteWhenHasGroupingScalarFunction() { + String sql = "select a,b,c,sum(d),grouping_id(a) from t1 group by rollup(a,b,c);"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalCTEAnchor()); + } + + @Test + void rewriteWhenMaxGroupingSetNotFirst() { + String sql = "select a,b,c,sum(d) from t1 group by grouping sets((a),(a,b,c),(a,b),());"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalCTEAnchor()); + } + + @Test + void rewriteWhenMaxGroupingSetFindMaxGroup() { + String sql = "select a,b,c,sum(d) from t1 group by grouping sets((a,b),(c,d),(a,b,c,d),());"; + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalCTEAnchor()); + } + + @Test + public void testFindMaxGroupingSetIndex() throws Exception { + Method method = rule.getClass().getDeclaredMethod("findMaxGroupingSetIndex", List.class); + method.setAccessible(true); + + // Test case 1: Empty list + List> emptyList = new ArrayList<>(); + int result = (int) method.invoke(rule, emptyList); + Assertions.assertEquals(-1, result); + + // Test case 2: Single grouping set + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + SlotReference c = new SlotReference("c", IntegerType.INSTANCE); + List> singleSet = ImmutableList.of(ImmutableList.of(a, b, c)); + result = (int) method.invoke(rule, singleSet); + Assertions.assertEquals(0, result); + + // Test case 3: Max grouping set contains all others (rollup scenario) + List> rollupSets = ImmutableList.of( + ImmutableList.of(a, b, c), // index 0 - max + ImmutableList.of(a, b), // index 1 + ImmutableList.of(a), // index 2 + ImmutableList.of() // index 3 + ); + result = (int) method.invoke(rule, rollupSets); + Assertions.assertEquals(0, result); + + // Test case 4: Max grouping set not at first position + List> mixedSets = ImmutableList.of( + ImmutableList.of(a), // index 0 + ImmutableList.of(a, b), // index 1 + ImmutableList.of(a, b, c), // index 2 - max + ImmutableList.of() // index 3 + ); + result = (int) method.invoke(rule, mixedSets); + Assertions.assertEquals(2, result); + + // Test case 5: No grouping set contains all others + SlotReference d = new SlotReference("d", IntegerType.INSTANCE); + List> disjointSets = ImmutableList.of( + ImmutableList.of(a, b), // index 0 + ImmutableList.of(c, d) // index 1 + ); + result = (int) method.invoke(rule, disjointSets); + Assertions.assertEquals(-1, result); + + // Test case 6: Multiple sets with same max size, should take first one + List> sameSizeSets = ImmutableList.of( + ImmutableList.of(a, b, c), // index 0 - first max + ImmutableList.of(a, b, d), // index 1 - same size + ImmutableList.of(a, b) // index 2 + ); + result = (int) method.invoke(rule, sameSizeSets); + // Should return 0 if it contains all others, otherwise -1 + // In this case, (a,b,c) doesn't contain (a,b,d), so should return -1 + Assertions.assertEquals(-1, result); + } + + @Test + public void testGetAggFuncSlotMap() throws Exception { + Method method = rule.getClass().getDeclaredMethod("getAggFuncSlotMap", List.class, Map.class); + method.setAccessible(true); + + SlotReference slot1 = new SlotReference("slot1", IntegerType.INSTANCE); + SlotReference slot2 = new SlotReference("slot2", IntegerType.INSTANCE); + SlotReference consumerSlot1 = new SlotReference("consumer_slot1", IntegerType.INSTANCE); + SlotReference consumerSlot2 = new SlotReference("consumer_slot2", IntegerType.INSTANCE); + + Sum sumFunc = new Sum(slot1); + Max maxFunc = new Max(slot2); + Alias sumAlias = new Alias(sumFunc, "sum_alias"); + Alias maxAlias = new Alias(maxFunc, "max_alias"); + + List outputExpressions = ImmutableList.of(sumAlias, maxAlias); + Map pToc = new HashMap<>(); + pToc.put(sumAlias.toSlot(), consumerSlot1); + pToc.put(maxAlias.toSlot(), consumerSlot2); + + @SuppressWarnings("unchecked") + Map result = (Map) method.invoke(rule, outputExpressions, pToc); + + Assertions.assertEquals(2, result.size()); + Assertions.assertEquals(consumerSlot1, result.get(sumFunc)); + Assertions.assertEquals(consumerSlot2, result.get(maxFunc)); + } + + @Test + public void testGetNeedAddNullExpressions() throws Exception { + Method method = rule.getClass().getDeclaredMethod("getNeedAddNullExpressions", + LogicalRepeat.class, List.class, int.class); + method.setAccessible(true); + + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + SlotReference c = new SlotReference("c", IntegerType.INSTANCE); + + List> groupingSets = ImmutableList.of( + ImmutableList.of(a, b, c), // index 0 - max + ImmutableList.of(a, b), // index 1 + ImmutableList.of(a) // index 2 + ); + + LogicalEmptyRelation emptyRelation = new LogicalEmptyRelation( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of()); + LogicalRepeat repeat = new LogicalRepeat<>( + groupingSets, + (List) ImmutableList.of(a, b, c), + null, + emptyRelation); + + List> newGroupingSets = ImmutableList.of( + ImmutableList.of(a, b), + ImmutableList.of(a) + ); + + @SuppressWarnings("unchecked") + Set result = (Set) method.invoke(rule, repeat, newGroupingSets, 0); + + // c should be in the result since it's in max group but not in other groups + Assertions.assertEquals(1, result.size()); + Assertions.assertTrue(result.contains(c)); + } + + @Test + public void testCanOptimize() throws Exception { + Method method = rule.getClass().getDeclaredMethod("canOptimize", LogicalAggregate.class); + method.setAccessible(true); + + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + SlotReference c = new SlotReference("c", IntegerType.INSTANCE); + SlotReference d = new SlotReference("d", IntegerType.INSTANCE); + + // Test case 1: Valid rollup with Sum + List> groupingSets = ImmutableList.of( + ImmutableList.of(a, b, c, d), + ImmutableList.of(a, b, c), + ImmutableList.of(a, b), + ImmutableList.of(a), + ImmutableList.of() + ); + LogicalEmptyRelation emptyRelation = new LogicalEmptyRelation( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of()); + LogicalRepeat repeat = new LogicalRepeat<>( + groupingSets, + (List) ImmutableList.of(a, b, c, d), + null, + emptyRelation); + Sum sumFunc = new Sum(d); + Alias sumAlias = new Alias(sumFunc, "sum_d"); + LogicalAggregate aggregate = new LogicalAggregate<>( + ImmutableList.of(a, b, c, d), + ImmutableList.of(a, b, c, d, sumAlias), + repeat); + + int result = (int) method.invoke(rule, aggregate); + Assertions.assertEquals(0, result); + + // Test case 2: Child is not LogicalRepeat + LogicalAggregate aggregateWithNonRepeat = new LogicalAggregate<>( + ImmutableList.of(a), + ImmutableList.of(a, sumAlias), + emptyRelation); + result = (int) method.invoke(rule, aggregateWithNonRepeat); + Assertions.assertEquals(-1, result); + + // Test case 3: Unsupported aggregate function (Avg) + org.apache.doris.nereids.trees.expressions.functions.agg.Avg avgFunc = + new org.apache.doris.nereids.trees.expressions.functions.agg.Avg(d); + Alias avgAlias = new Alias(avgFunc, "avg_d"); + LogicalAggregate aggregateWithCount = new LogicalAggregate<>( + ImmutableList.of(a, b, c, d), + ImmutableList.of(a, b, c, d, avgAlias), + repeat); + result = (int) method.invoke(rule, aggregateWithCount); + Assertions.assertEquals(-1, result); + + // Test case 4: Grouping sets size <= 3 + List> smallGroupingSets = ImmutableList.of( + ImmutableList.of(a, b), + ImmutableList.of(a), + ImmutableList.of() + ); + LogicalRepeat smallRepeat = new LogicalRepeat<>( + smallGroupingSets, + (List) ImmutableList.of(a, b), + null, + emptyRelation); + LogicalAggregate aggregateWithSmallRepeat = new LogicalAggregate<>( + ImmutableList.of(a, b), + ImmutableList.of(a, b, sumAlias), + smallRepeat); + result = (int) method.invoke(rule, aggregateWithSmallRepeat); + Assertions.assertEquals(-1, result); + } + + @Test + public void testConstructUnion() throws Exception { + Method method = rule.getClass().getDeclaredMethod("constructUnion", + org.apache.doris.nereids.trees.plans.logical.LogicalPlan.class, + org.apache.doris.nereids.trees.plans.logical.LogicalPlan.class, + LogicalAggregate.class); + method.setAccessible(true); + + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + Sum sumFunc = new Sum(b); + Alias sumAlias = new Alias(sumFunc, "sum_b"); + + LogicalEmptyRelation emptyRelation = new LogicalEmptyRelation( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of()); + List> groupingSets = ImmutableList.of( + ImmutableList.of(a, b), + ImmutableList.of(a), + ImmutableList.of() + ); + LogicalRepeat repeat = new LogicalRepeat<>( + groupingSets, + (List) ImmutableList.of(a, b), + new SlotReference("grouping_id", IntegerType.INSTANCE), + emptyRelation); + LogicalAggregate aggregate = new LogicalAggregate<>( + ImmutableList.of(a, b), + ImmutableList.of(a, b, sumAlias), + repeat); + + LogicalProject project = new LogicalProject<>( + ImmutableList.of(a, b, sumAlias.toSlot()), + aggregate); + LogicalCTEConsumer consumer = new LogicalCTEConsumer( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + new CTEId(1), "", new LogicalCTEProducer<>(new CTEId(1), emptyRelation)); + + LogicalUnion result = (LogicalUnion) method.invoke(rule, project, consumer, aggregate); + Assertions.assertNotNull(result); + Assertions.assertEquals(2, result.children().size()); + Assertions.assertTrue(aggregate.getOutputSet().containsAll(result.getOutputSet())); + } + + @Test + public void testConstructProducer() throws Exception { + Method method = rule.getClass().getDeclaredMethod("constructProducer", + LogicalAggregate.class, int.class, DistinctSelectorContext.class, Map.class); + method.setAccessible(true); + + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + SlotReference c = new SlotReference("c", IntegerType.INSTANCE); + SlotReference d = new SlotReference("d", IntegerType.INSTANCE); + + List> groupingSets = ImmutableList.of( + ImmutableList.of(a, b, c, d), // index 0 - max + ImmutableList.of(a, b, c), + ImmutableList.of(a, b), + ImmutableList.of(a), + ImmutableList.of() + ); + LogicalEmptyRelation emptyRelation = new LogicalEmptyRelation( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of()); + LogicalRepeat repeat = new LogicalRepeat<>( + groupingSets, + (List) ImmutableList.of(a, b, c, d), + null, + emptyRelation); + Sum sumFunc = new Sum(d); + Alias sumAlias = new Alias(sumFunc, "sum_d"); + LogicalAggregate aggregate = new LogicalAggregate<>( + ImmutableList.of(a, b, c, d), + ImmutableList.of(a, b, c, d, sumAlias), + repeat); + + Map preToCloneSlotMap = new HashMap<>(); + LogicalCTEProducer> result = (LogicalCTEProducer>) + method.invoke(rule, aggregate, 0, ctx, preToCloneSlotMap); + + Assertions.assertNotNull(result); + Assertions.assertNotNull(result.child()); + Assertions.assertInstanceOf(LogicalAggregate.class, result.child()); + } + + @Test + public void testConstructRepeat() throws Exception { + Method method = rule.getClass().getDeclaredMethod("constructRepeat", + LogicalRepeat.class, + org.apache.doris.nereids.trees.plans.logical.LogicalPlan.class, + List.class, + Map.class, + List.class); + method.setAccessible(true); + + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + SlotReference b = new SlotReference("b", IntegerType.INSTANCE); + SlotReference c = new SlotReference("c", IntegerType.INSTANCE); + SlotReference consumerA = new SlotReference("consumer_a", IntegerType.INSTANCE); + SlotReference consumerB = new SlotReference("consumer_b", IntegerType.INSTANCE); + SlotReference consumerC = new SlotReference("consumer_c", IntegerType.INSTANCE); + + List> originalGroupingSets = ImmutableList.of( + ImmutableList.of(a, b, c), + ImmutableList.of(a, b), + ImmutableList.of(a) + ); + LogicalEmptyRelation emptyRelation = new LogicalEmptyRelation( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + ImmutableList.of()); + LogicalRepeat originalRepeat = new LogicalRepeat<>( + originalGroupingSets, + (List) ImmutableList.of(a, b, c), + new SlotReference("grouping_id", IntegerType.INSTANCE), + emptyRelation); + + List> newGroupingSets = ImmutableList.of( + ImmutableList.of(a, b), + ImmutableList.of(a) + ); + + Map producerToConsumerSlotMap = new HashMap<>(); + producerToConsumerSlotMap.put(a, consumerA); + producerToConsumerSlotMap.put(b, consumerB); + producerToConsumerSlotMap.put(c, consumerC); + + LogicalCTEConsumer consumer = new LogicalCTEConsumer( + org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator.newRelationId(), + new CTEId(1), "", new LogicalCTEProducer<>(new CTEId(1), emptyRelation)); + List groupingFunctionSlots = new ArrayList<>(); + LogicalRepeat result = (LogicalRepeat) method.invoke(rule, + originalRepeat, consumer, newGroupingSets, producerToConsumerSlotMap, groupingFunctionSlots); + + Assertions.assertNotNull(result); + Assertions.assertEquals(2, result.getGroupingSets().size()); + Assertions.assertTrue(groupingFunctionSlots.isEmpty()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UniqueFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UniqueFunctionTest.java index 574fe85861df77..fd40a7701649a5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UniqueFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/functions/scalar/UniqueFunctionTest.java @@ -924,7 +924,7 @@ void testRepeat1() { String sql = "select random(), a + random(), sum(random()), sum(a + random()), max(random()) over(), max(a + random()) over()" + " from t" + " group by grouping sets((), (random()), (random(), random()), (random(), a + random()))"; - + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION, DECOMPOSE_REPEAT"); Plan root = PlanChecker.from(connectContext) .analyze(sql) .rewrite() diff --git a/regression-test/data/nereids_rules_p0/decompose_repeat/decompose_repeat.out b/regression-test/data/nereids_rules_p0/decompose_repeat/decompose_repeat.out new file mode 100644 index 00000000000000..919738109c1efd --- /dev/null +++ b/regression-test/data/nereids_rules_p0/decompose_repeat/decompose_repeat.out @@ -0,0 +1,371 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sum -- +\N \N \N 10 +1 \N \N 10 +1 2 \N 8 +1 2 1 1 +1 2 3 7 +1 3 \N 2 +1 3 2 2 + +-- !agg_func_gby_key_same_col -- +\N \N \N \N 10 +1 \N \N \N 10 +1 2 \N \N 8 +1 2 1 \N 1 +1 2 1 1 1 +1 2 3 \N 7 +1 2 3 3 3 +1 2 3 4 4 +1 3 \N \N 2 +1 3 2 \N 2 +1 3 2 2 2 + +-- !multi_agg_func -- +\N \N \N 10 9 1 +1 \N \N 10 9 1 +1 2 \N 8 7 1 +1 2 1 1 1 1 +1 2 1 1 1 1 +1 2 3 3 3 1 +1 2 3 4 3 1 +1 2 3 7 6 1 +1 3 \N 2 2 1 +1 3 2 2 2 1 +1 3 2 2 2 1 + +-- !nest_rewrite -- +\N \N \N \N +1 \N \N \N +1 \N \N \N +1 \N \N \N +1 \N \N 10 +1 2 \N \N +1 2 1 \N +1 2 1 1 +1 2 3 \N +1 2 3 3 +1 2 3 4 +1 2 3 7 +1 3 \N \N +1 3 2 \N +1 3 2 2 + +-- !upper_ref -- +11 1 2 1 +12 1 3 \N +12 1 3 2 +17 1 2 3 +18 1 2 \N +20 \N \N \N +20 1 \N \N + +-- !another_cte -- +\N +1 +1 +1 +1 +1 +2 +2 +2 +2 +2 + +-- !choose_max_group -- +3 +3 +3 +3 +4 +4 + +-- !only_output_grouping_id -- +\N +\N +\N +1 + +-- !sum0_count -- +\N \N \N 1 1 1 +\N \N \N 2 2 1 +\N \N \N 3 3 1 +\N \N \N 4 4 1 +1 \N \N 1 1 1 +1 \N \N 2 2 1 +1 \N \N 3 3 1 +1 \N \N 4 4 1 +1 2 1 \N 1 1 +1 2 1 1 1 1 +1 2 3 \N 7 2 +1 2 3 3 3 1 +1 2 3 4 4 1 +1 3 2 \N 2 1 +1 3 2 2 2 1 + +-- !choose_max_group -- +\N \N \N 1 1 +\N \N \N 2 2 +\N \N \N 3 3 +\N \N \N 4 4 +1 \N \N 1 1 +1 \N \N 2 2 +1 \N \N 3 3 +1 \N \N 4 4 +1 2 1 \N 1 +1 2 1 1 1 +1 2 3 \N 7 +1 2 3 3 3 +1 2 3 4 4 +1 3 2 \N 2 +1 3 2 2 2 + +-- !multi_grouping_func -- +\N \N \N 1 1 0 1 +\N \N \N 2 1 0 1 +\N \N \N 3 1 0 1 +\N \N \N 4 1 0 1 +1 \N \N 1 1 0 1 +1 \N \N 2 1 0 1 +1 \N \N 3 1 0 1 +1 \N \N 4 1 0 1 +1 2 1 \N 1 1 0 +1 2 1 1 1 0 0 +1 2 3 \N 2 1 0 +1 2 3 3 1 0 0 +1 2 3 4 1 0 0 +1 3 2 \N 1 1 0 +1 3 2 2 1 0 0 + +-- !grouping_func -- +1 \N \N \N 10 0 +1 2 1 \N 1 0 +1 2 1 \N 1 0 +1 2 1 1 1 0 +1 2 3 \N 7 0 +1 2 3 \N 7 0 +1 2 3 3 3 0 +1 2 3 4 4 0 +1 3 2 \N 2 0 +1 3 2 \N 2 0 +1 3 2 2 2 0 + +-- !avg -- +1 \N \N \N 2.5 +1 2 1 \N 1 +1 2 1 \N 1 +1 2 1 1 1 +1 2 3 \N 3.5 +1 2 3 \N 3.5 +1 2 3 3 3 +1 2 3 4 4 +1 3 2 \N 2 +1 3 2 \N 2 +1 3 2 2 2 + +-- !distinct -- +1 \N \N \N 10 +1 2 1 \N 1 +1 2 1 \N 1 +1 2 1 1 1 +1 2 3 \N 7 +1 2 3 \N 7 +1 2 3 3 3 +1 2 3 4 4 +1 3 2 \N 2 +1 3 2 \N 2 +1 3 2 2 2 + +-- !less_equal_than_3 -- +\N \N \N \N 10 +1 2 1 \N 1 +1 2 1 1 1 +1 2 3 \N 7 +1 2 3 3 3 +1 2 3 4 4 +1 3 2 \N 2 +1 3 2 2 2 + +-- !guard -- +1 \N \N \N 10 0 +1 2 1 \N 1 0 +1 2 1 \N 1 0 +1 2 1 1 1 0 +1 2 3 \N 7 0 +1 2 3 \N 7 0 +1 2 3 3 3 0 +1 2 3 4 4 0 +1 3 2 \N 2 0 +1 3 2 \N 2 0 +1 3 2 2 2 0 + +-- !rollup -- +\N \N \N \N 10 1 +1 \N \N \N 10 0 +1 2 \N \N 8 0 +1 2 1 \N 1 0 +1 2 1 1 1 0 +1 2 3 \N 7 0 +1 2 3 3 3 0 +1 2 3 4 4 0 +1 3 \N \N 2 0 +1 3 2 \N 2 0 +1 3 2 2 2 0 + +-- !cube -- +\N \N \N \N 10 1 +\N \N \N 1 1 1 +\N \N \N 2 2 1 +\N \N \N 3 3 1 +\N \N \N 4 4 1 +\N \N 1 \N 1 1 +\N \N 1 1 1 1 +\N \N 2 \N 2 1 +\N \N 2 2 2 1 +\N \N 3 \N 7 1 +\N \N 3 3 3 1 +\N \N 3 4 4 1 +\N 2 \N \N 8 1 +\N 2 \N 1 1 1 +\N 2 \N 3 3 1 +\N 2 \N 4 4 1 +\N 2 1 \N 1 1 +\N 2 1 1 1 1 +\N 2 3 \N 7 1 +\N 2 3 3 3 1 +\N 2 3 4 4 1 +\N 3 \N \N 2 1 +\N 3 \N 2 2 1 +\N 3 2 \N 2 1 +\N 3 2 2 2 1 +1 \N \N \N 10 0 +1 \N \N 1 1 0 +1 \N \N 2 2 0 +1 \N \N 3 3 0 +1 \N \N 4 4 0 +1 \N 1 \N 1 0 +1 \N 1 1 1 0 +1 \N 2 \N 2 0 +1 \N 2 2 2 0 +1 \N 3 \N 7 0 +1 \N 3 3 3 0 +1 \N 3 4 4 0 +1 2 \N \N 8 0 +1 2 \N 1 1 0 +1 2 \N 3 3 0 +1 2 \N 4 4 0 +1 2 1 \N 1 0 +1 2 1 1 1 0 +1 2 3 \N 7 0 +1 2 3 3 3 0 +1 2 3 4 4 0 +1 3 \N \N 2 0 +1 3 \N 2 2 0 +1 3 2 \N 2 0 +1 3 2 2 2 0 + +-- !cube_add -- +\N \N \N \N 111 +\N \N \N 1 102 +\N \N \N 2 103 +\N \N \N 3 104 +\N \N \N 4 105 +\N \N 1 \N 102 +\N \N 1 1 102 +\N \N 2 \N 103 +\N \N 2 2 103 +\N \N 3 \N 108 +\N \N 3 3 104 +\N \N 3 4 105 +\N 2 \N \N 109 +\N 2 \N 1 102 +\N 2 \N 3 104 +\N 2 \N 4 105 +\N 2 1 \N 102 +\N 2 1 1 102 +\N 2 3 \N 108 +\N 2 3 3 104 +\N 2 3 4 105 +\N 3 \N \N 103 +\N 3 \N 2 103 +\N 3 2 \N 103 +\N 3 2 2 103 +1 \N \N \N 110 +1 \N \N 1 101 +1 \N \N 2 102 +1 \N \N 3 103 +1 \N \N 4 104 +1 \N 1 \N 101 +1 \N 1 1 101 +1 \N 2 \N 102 +1 \N 2 2 102 +1 \N 3 \N 107 +1 \N 3 3 103 +1 \N 3 4 104 +1 2 \N \N 108 +1 2 \N 1 101 +1 2 \N 3 103 +1 2 \N 4 104 +1 2 1 \N 101 +1 2 1 1 101 +1 2 3 \N 107 +1 2 3 3 103 +1 2 3 4 104 +1 3 \N \N 102 +1 3 \N 2 102 +1 3 2 \N 102 +1 3 2 2 102 + +-- !cube_sum_parm_add -- +\N \N \N \N 8 1 +\N \N \N 1 2 1 +\N \N \N 2 2 1 +\N \N \N 3 2 1 +\N \N \N 4 2 1 +\N \N 1 \N 2 1 +\N \N 1 1 2 1 +\N \N 2 \N 2 1 +\N \N 2 2 2 1 +\N \N 3 \N 4 1 +\N \N 3 3 2 1 +\N \N 3 4 2 1 +\N 2 \N \N 6 1 +\N 2 \N 1 2 1 +\N 2 \N 3 2 1 +\N 2 \N 4 2 1 +\N 2 1 \N 2 1 +\N 2 1 1 2 1 +\N 2 3 \N 4 1 +\N 2 3 3 2 1 +\N 2 3 4 2 1 +\N 3 \N \N 2 1 +\N 3 \N 2 2 1 +\N 3 2 \N 2 1 +\N 3 2 2 2 1 +1 \N \N \N 8 0 +1 \N \N 1 2 0 +1 \N \N 2 2 0 +1 \N \N 3 2 0 +1 \N \N 4 2 0 +1 \N 1 \N 2 0 +1 \N 1 1 2 0 +1 \N 2 \N 2 0 +1 \N 2 2 2 0 +1 \N 3 \N 4 0 +1 \N 3 3 2 0 +1 \N 3 4 2 0 +1 2 \N \N 6 0 +1 2 \N 1 2 0 +1 2 \N 3 2 0 +1 2 \N 4 2 0 +1 2 1 \N 2 0 +1 2 1 1 2 0 +1 2 3 \N 4 0 +1 2 3 3 2 0 +1 2 3 4 2 0 +1 3 \N \N 2 0 +1 3 \N 2 2 0 +1 3 2 \N 2 0 +1 3 2 2 2 0 + diff --git a/regression-test/data/nereids_rules_p0/unique_function/agg_with_unique_function.out b/regression-test/data/nereids_rules_p0/unique_function/agg_with_unique_function.out index 5bd030d2e39e0e..dfe9a28c4bb4ad 100644 --- a/regression-test/data/nereids_rules_p0/unique_function/agg_with_unique_function.out +++ b/regression-test/data/nereids_rules_p0/unique_function/agg_with_unique_function.out @@ -253,16 +253,20 @@ PhysicalResultSink ------------------PhysicalOlapScan[tbl_unique_function_with_one_row] -- !check_equal_repeat1_shape -- -PhysicalResultSink ---PhysicalProject[a + random(), a + random() + 0, abs(a + random()), sum(a + random() + 0), sum(a + random())] -----PhysicalQuickSort[MERGE_SORT] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject[(a + random() + 1.0) AS `(a + random() + 1.0)`, a + random(), a + random() + 0, abs(a + random()) AS `abs(a + random())`, sum(a + random() + 0), sum(a + random())] -----------filter((a + random() + 0 > 0.01)) -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat -------------------PhysicalProject[a + random(), a + random() + 0, a + random() + 0 AS `a + random() + 0`, a + random() AS `a + random()`] ---------------------PhysicalProject[((a + random()) + 0.0) AS `a + random() + 0`, (a + random()) AS `a + random()`] -----------------------PhysicalOlapScan[tbl_unique_function_with_one_row] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalProject[a + random(), a + random() + 0, a + random() + 0 AS `a + random() + 0`, a + random() AS `a + random()`] +----------PhysicalProject[((a + random()) + 0.0) AS `a + random() + 0`, (a + random()) AS `a + random()`] +------------PhysicalOlapScan[tbl_unique_function_with_one_row] +--PhysicalResultSink +----PhysicalProject[a + random(), a + random() + 0, abs(a + random()), sum(a + random() + 0), sum(a + random())] +------PhysicalQuickSort[MERGE_SORT] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject[(a + random() + 1.0) AS `(a + random() + 1.0)`, a + random(), a + random() + 0, abs(a + random()) AS `abs(a + random())`, sum(a + random() + 0), sum(a + random())] +------------PhysicalUnion +--------------PhysicalEmptyRelation +--------------filter((.a + random() + 0 > 0.01)) +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query14.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query14.out index 92357ad5661397..fa6d49b020e925 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query14.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query14.out @@ -69,80 +69,90 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query67.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query67.out index 32aca703b091c1..d07dbaab87e846 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query67.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query67.out @@ -1,32 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------PhysicalOlapScan[item] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalQuickSort[LOCAL_SORT] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query14.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query14.out index 1d3f313010e823..75231a5f7d34ad 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query14.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query14.out @@ -69,80 +69,90 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query67.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query67.out index b3a41aa353eb81..a729c9d2be119a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query67.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query67.out @@ -1,32 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------PhysicalOlapScan[item] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalQuickSort[LOCAL_SORT] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query14.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query14.out index b63fb4c178870d..9c4d0b19325a8a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query14.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query14.out @@ -69,80 +69,90 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query67.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query67.out index ca0256aeea797e..c0e4655f9f1c98 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query67.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query67.out @@ -1,32 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalPartitionTopN ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +--------------PhysicalProject +----------------PhysicalOlapScan[store] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalPartitionTopN +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query14.out b/regression-test/data/shape_check/tpcds_sf100/shape/query14.out index 4583e0e246556d..cbcaf12a80eefc 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query14.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query14.out @@ -69,80 +69,90 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query67.out b/regression-test/data/shape_check/tpcds_sf100/shape/query67.out index 5d1c063b4cb653..b6605fe094027f 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query67.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query67.out @@ -1,32 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalPartitionTopN ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +--------------PhysicalProject +----------------PhysicalOlapScan[store] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalPartitionTopN +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query14.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query14.out index 926b275c276b6e..81cae62c393eba 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query14.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query14.out @@ -69,82 +69,92 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) Hint log: Used: diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query67.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query67.out index deb36956c4f449..a99815bf676d13 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query67.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query67.out @@ -1,34 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalPartitionTopN ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1228) and (date_dim.d_month_seq >= 1217)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1228) and (date_dim.d_month_seq >= 1217)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------PhysicalOlapScan[item] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalPartitionTopN +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) Hint log: Used: leading(store_sales broadcast date_dim broadcast store broadcast item ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query14.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query14.out index 6793b0f43678d8..a69d5958d677f2 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query14.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query14.out @@ -69,80 +69,90 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query67.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query67.out index b5f6486b18a89a..7913aec7a93a82 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query67.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query67.out @@ -1,32 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalPartitionTopN ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1228) and (date_dim.d_month_seq >= 1217)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1228) and (date_dim.d_month_seq >= 1217)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------PhysicalOlapScan[item] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalPartitionTopN +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query14.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query14.out index 11c0239c69a982..d87f212b4943fd 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query14.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query14.out @@ -69,80 +69,90 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] -----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +----PhysicalCteAnchor ( cteId=CTEId#4 ) +------PhysicalCteProducer ( cteId=CTEId#4 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------PhysicalProject +------------------NestedLoopJoin[INNER_JOIN](cast(sales as DECIMALV3(38, 4)) > avg_sales.average_sales) +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalAssertNumRows +----------------------PhysicalDistribute[DistributionSpecGather] +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------PhysicalResultSink +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalRepeat +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#4 ) +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#4 ) diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query67.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query67.out index c7cf13027a3e80..486fa44ef7a8a1 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query67.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query67.out @@ -1,32 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------filter((dw2.rk <= 100)) -----------PhysicalWindow -------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalPartitionTopN +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1196) and (date_dim.d_month_seq >= 1185)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------PhysicalProject +------------------------filter((date_dim.d_month_seq <= 1196) and (date_dim.d_month_seq >= 1185)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------PhysicalOlapScan[item] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------filter((dw2.rk <= 100)) +------------PhysicalWindow +--------------PhysicalQuickSort[LOCAL_SORT] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalPartitionTopN +--------------------PhysicalUnion +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalRepeat +--------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/suites/nereids_rules_p0/decompose_repeat/decompose_repeat.groovy b/regression-test/suites/nereids_rules_p0/decompose_repeat/decompose_repeat.groovy new file mode 100644 index 00000000000000..338517afbc4f46 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/decompose_repeat/decompose_repeat.groovy @@ -0,0 +1,74 @@ +// 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("decompose_repeat") { +// sql "set disable_nereids_rules='DECOMPOSE_REPEAT';" + sql "drop table if exists t1;" + sql "create table t1(a int, b int, c int, d int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into t1 values(1,2,3,4),(1,2,3,3),(1,2,1,1),(1,3,2,2);" + order_qt_sum "select a,b,c,sum(d) from t1 group by rollup(a,b,c);" + order_qt_agg_func_gby_key_same_col "select a,b,c,d,sum(d) from t1 group by rollup(a,b,c,d);" + order_qt_multi_agg_func "select a,b,c,sum(d),sum(c),max(a) from t1 group by rollup(a,b,c,d);" + order_qt_nest_rewrite """ + select a,b,c,c1 from ( + select a,b,c,d,sum(d) c1 from t1 group by grouping sets((a,b,c),(a,b,c,d),(a),(a,b,c,c)) + ) t group by rollup(a,b,c,c1); + """ + order_qt_upper_ref """ + select c1+10,a,b,c from (select a,b,c,sum(d) c1 from t1 group by rollup(a,b,c)) t group by c1+10,a,b,c; + """ + order_qt_another_cte """ + with cte1 as (select 1 as c1 union all select 2) + select c1 from ( + select c1,1 c2, 2 c3 from cte1 union select c1, 2,3 from cte1 + ) t + group by rollup(c1,c2,c3); + """ + order_qt_choose_max_group """ + select min(a+b) from t1 group by grouping sets((a,b),(b,c),(a)); + """ + order_qt_only_output_grouping_id """ + select a from t1 group by grouping sets ((),(),(),(a)) order by a; + """ + order_qt_sum0_count "select a,b,c,d,sum0(d) c1, count(d) c3 from t1 group by grouping sets((a,b,c),(d),(d,a),(a,b,c,d));" + order_qt_choose_max_group """ + select a,b,c,d,sum(d) c1 from t1 group by grouping sets((a,b,c),(d),(d,a),(a,b,c,d)); + """ + order_qt_multi_grouping_func """ + select a,b,c,d,count(d) c1, grouping(d),grouping_id(c) from t1 group by grouping sets((a,b,c),(d),(d,a),(a,b,c,d)); + """ + order_qt_grouping_func "select a,b,c,d,sum(d),grouping_id(a) from t1 group by grouping sets((a,b,c),(a,b,c,d),(a),(a,b,c,c))" + // negative case + order_qt_avg "select a,b,c,d,avg(d) from t1 group by grouping sets((a,b,c),(a,b,c,d),(a),(a,b,c,c));" + order_qt_distinct "select a,b,c,d,sum(distinct d) from t1 group by grouping sets((a,b,c),(a,b,c,d),(a),(a,b,c,c));" + order_qt_less_equal_than_3 "select a,b,c,d,sum(distinct d) from t1 group by grouping sets((a,b,c),(a,b,c,d),());" + + // test guard + sql "set enable_decimal256=true;" + multi_sql """ + drop view if exists test_guard; + create view test_guard as select a,b,c,d,sum(d) as c1, grouping_id(a) as c2 from t1 group by grouping sets((a,b,c),(a,b,c,d),(a),(a,b,c,c)); + """ + sql "set enable_decimal256=false;" + order_qt_guard "select * from test_guard;" + + // rollup,cube + order_qt_rollup "select a,b,c,d,sum(d),grouping_id(a) from t1 group by rollup(a,b,c,d)" + order_qt_cube "select a,b,c,d,sum(d),grouping_id(a) from t1 group by cube(a,b,c,d)" + order_qt_cube_add "select a,b,c,d,sum(d)+100+grouping_id(a) from t1 group by cube(a,b,c,d);" + order_qt_cube_sum_parm_add "select a,b,c,d,sum(a+1),grouping_id(a) from t1 group by cube(a,b,c,d);" +} \ No newline at end of file