diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModel.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModel.java index 9ae9b262827c73..9f828847eaa77c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModel.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModel.java @@ -380,10 +380,11 @@ public Cost visitPhysicalHashAggregate( exprCost / 100 + inputStatistics.getRowCount() / beNumber, inputStatistics.getRowCount() / beNumber, 0); } else { + int factor = aggregate.getGroupByExpressions().isEmpty() ? 1 : beNumber; // global - return Cost.of(context.getSessionVariable(), exprCost / 100 - + inputStatistics.getRowCount(), - inputStatistics.getRowCount(), 0); + return Cost.of(context.getSessionVariable(), + exprCost / 100 + inputStatistics.getRowCount() / factor, + inputStatistics.getRowCount() / factor, 0); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index e07cc4851dea62..a631932de2a376 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -244,6 +244,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1182,6 +1183,7 @@ public PlanFragment visitPhysicalHashAggregate( // 2. collect agg expressions and generate agg function to slot reference map List aggFunctionOutput = Lists.newArrayList(); ArrayList execAggregateFunctions = Lists.newArrayListWithCapacity(outputExpressions.size()); + AtomicBoolean hasPartialInAggFunc = new AtomicBoolean(false); for (NamedExpression o : outputExpressions) { if (o.containsType(AggregateExpression.class)) { aggFunctionOutput.add(o.toSlot()); @@ -1191,10 +1193,16 @@ public PlanFragment visitPhysicalHashAggregate( execAggregateFunctions.add( (FunctionCallExpr) ExpressionTranslator.translate((AggregateExpression) c, context) ); + hasPartialInAggFunc.set( + ((AggregateExpression) c).getAggregateParam().aggMode.productAggregateBuffer); } }); } } + // An agg may have different functions, some product buffer, some product result. + // The criterion for passing it to the be stage is: as long as there is a product buffer function in agg, + // it must be isPartial + boolean isPartial = hasPartialInAggFunc.get(); // 3. generate output tuple List slotList = Lists.newArrayList(); @@ -1211,7 +1219,6 @@ public PlanFragment visitPhysicalHashAggregate( aggFunOutputIds.add(slots.get(i).getId().asInt()); } } - boolean isPartial = aggregate.getAggregateParam().aggMode.productAggregateBuffer; AggregateInfo aggInfo = AggregateInfo.create(execGroupingExpressions, execAggregateFunctions, aggFunOutputIds, isPartial, outputTupleDesc, aggregate.getAggPhase().toExec()); AggregationNode aggregationNode = new AggregationNode(context.nextPlanNodeId(), @@ -1221,7 +1228,7 @@ public PlanFragment visitPhysicalHashAggregate( aggregationNode.setNereidsId(aggregate.getId()); context.getNereidsIdToPlanNodeIdMap().put(aggregate.getId(), aggregationNode.getId()); - if (!aggregate.getAggMode().isFinalPhase) { + if (isPartial) { aggregationNode.unsetNeedsFinalize(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java index 80fffc9f2ffe0c..76a659870d09fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java @@ -216,7 +216,8 @@ public void execute() { // if break when running the loop above, the condition must be false. if (curChildIndex == groupExpression.arity()) { if (!calculateEnforce(requestChildrenProperties, outputChildrenProperties)) { - return; // if error exists, return + clear(); + continue; // if error exists, return } if (curTotalCost.getValue() < context.getCostUpperBound()) { context.setCostUpperBound(curTotalCost.getValue()); 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 447c58311a903d..f73b75d2dd6965 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 @@ -59,6 +59,8 @@ import org.apache.doris.nereids.rules.rewrite.CreatePartitionTopNFromWindow; import org.apache.doris.nereids.rules.rewrite.DecoupleEncodeDecode; import org.apache.doris.nereids.rules.rewrite.DeferMaterializeTopNResult; +import org.apache.doris.nereids.rules.rewrite.DistinctAggStrategySelector; +import org.apache.doris.nereids.rules.rewrite.DistinctAggregateRewriter; import org.apache.doris.nereids.rules.rewrite.DistinctWindowExpression; import org.apache.doris.nereids.rules.rewrite.EliminateAggCaseWhen; import org.apache.doris.nereids.rules.rewrite.EliminateAggregate; @@ -153,7 +155,6 @@ import org.apache.doris.nereids.rules.rewrite.SimplifyWindowExpression; import org.apache.doris.nereids.rules.rewrite.SkewJoin; import org.apache.doris.nereids.rules.rewrite.SplitLimit; -import org.apache.doris.nereids.rules.rewrite.SplitMultiDistinct; import org.apache.doris.nereids.rules.rewrite.SumLiteralRewrite; import org.apache.doris.nereids.rules.rewrite.TransposeSemiJoinAgg; import org.apache.doris.nereids.rules.rewrite.TransposeSemiJoinAggProject; @@ -762,6 +763,7 @@ public class Rewriter extends AbstractBatchJobExecutor { new PushDownFilterThroughProject(), new MergeProjectable() )), + topDown(DistinctAggregateRewriter.INSTANCE), custom(RuleType.ELIMINATE_UNNECESSARY_PROJECT, EliminateUnnecessaryProject::new), topDown(new PushDownVectorTopNIntoOlapScan()), topDown(new PushDownVirtualColumnsIntoOlapScan()), @@ -881,8 +883,9 @@ private static List getWholeTreeRewriteJobs( rewriteJobs.addAll(jobs(topic("or expansion", custom(RuleType.OR_EXPANSION, () -> OrExpansion.INSTANCE)))); } + rewriteJobs.addAll(jobs(topic("split multi distinct", - custom(RuleType.SPLIT_MULTI_DISTINCT, () -> SplitMultiDistinct.INSTANCE)))); + custom(RuleType.DISTINCT_AGG_STRATEGY_SELECTOR, () -> DistinctAggStrategySelector.INSTANCE)))); if (needSubPathPushDown) { rewriteJobs.addAll(jobs( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java index 25102d21dc7090..de1a2cba892ab6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java @@ -29,7 +29,6 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.properties.RequestPropertyDeriver; -import org.apache.doris.nereids.properties.RequirePropertiesSupplier; import org.apache.doris.nereids.rules.exploration.mv.AbstractMaterializedViewRule; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.GroupPlan; @@ -1013,7 +1012,6 @@ private List> extractInputProperties(GroupExpression gr .filter(e -> e.stream().allMatch(PhysicalProperties.ANY::equals)) .findAny(); if (any.isPresent() - && !(groupExpression.getPlan() instanceof RequirePropertiesSupplier) && !(groupExpression.getPlan() instanceof SetOperation)) { res.clear(); res.add(any.get()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java index 28dc0a9941cf66..e4d17ffe72e648 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java @@ -23,18 +23,16 @@ import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; -import org.apache.doris.nereids.trees.expressions.AggregateExpression; -import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.stats.StatsCalculator; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinction; import org.apache.doris.nereids.trees.plans.AggMode; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.SortPhase; import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalSort; +import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; @@ -47,18 +45,21 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.AggregateUtils; import org.apache.doris.nereids.util.JoinUtils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.Statistics; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; /** * ensure child add enough distribute. update children properties if we do regular. @@ -66,7 +67,6 @@ * to process must shuffle except project and filter */ public class ChildrenPropertiesRegulator extends PlanVisitor>, Void> { - private final GroupExpression parent; private final List children; private final List originChildrenProperties; @@ -110,74 +110,147 @@ public List> visitPhysicalHashAggregate( if (agg.getGroupByExpressions().isEmpty() && agg.getOutputExpressions().isEmpty()) { return ImmutableList.of(); } + // If the origin attribute satisfies the group by key but does not meet the requirements, ban the plan. + // e.g. select count(distinct a) from t group by b; + // requiredChildProperty: a + // but the child is already distributed by b + // ban this plan + PhysicalProperties originChildProperty = originChildrenProperties.get(0); + PhysicalProperties requiredChildProperty = requiredProperties.get(0); + PhysicalProperties hashSpec = PhysicalProperties.createHash(agg.getGroupByExpressions(), ShuffleType.REQUIRE); + GroupExpression child = children.get(0); + if (child.getPlan() instanceof PhysicalDistribute) { + PhysicalProperties properties = new PhysicalProperties( + DistributionSpecAny.INSTANCE, originChildProperty.getOrderSpec()); + Optional> pair = child.getOwnerGroup().getLowestCostPlan(properties); + // add null check + if (!pair.isPresent()) { + return ImmutableList.of(); + } + GroupExpression distributeChild = pair.get().second; + PhysicalProperties distributeChildProperties = distributeChild.getOutputProperties(properties); + if (distributeChildProperties.satisfy(hashSpec) + && !distributeChildProperties.satisfy(requiredChildProperty)) { + return ImmutableList.of(); + } + } + if (!agg.getAggregateParam().canBeBanned) { return visit(agg, context); } - // forbid one phase agg on distribute - if (agg.getAggMode() == AggMode.INPUT_TO_RESULT && children.get(0).getPlan() instanceof PhysicalDistribute) { - // this means one stage gather agg, usually bad pattern + // return aggBanByStatistics(agg, context); + if (shouldBanOnePhaseAgg(agg, requiredChildProperty)) { return ImmutableList.of(); } + // process must shuffle + return visit(agg, context); + } - // forbid TWO_PHASE_AGGREGATE_WITH_DISTINCT after shuffle - // TODO: this is forbid good plan after cte reuse by mistake - if (agg.getAggMode() == AggMode.INPUT_TO_BUFFER - && requiredProperties.get(0).getDistributionSpec() instanceof DistributionSpecHash - && children.get(0).getPlan() instanceof PhysicalDistribute) { - return ImmutableList.of(); + /** + * 1. Generally, one-stage AGG is disabled unless the child of distribute is a CTE consumer. + * 2. If it is a CTE consumer, to avoid being banned, ensure that distribute is not a gather. + * Alternatively, if the distribute is a shuffle, ensure that the shuffle expr is not skewed. + * */ + private boolean shouldBanOnePhaseAgg(PhysicalHashAggregate aggregate, + PhysicalProperties requiredChildProperty) { + if (banAggUnionAll(aggregate)) { + return true; } + ConnectContext ctx = ConnectContext.get(); + if (ctx != null && ctx.getSessionVariable().aggPhase == 1) { + return false; + } + if (!onePhaseAggWithDistribute(aggregate)) { + return false; + } + if (childIsCTEConsumer()) { + // shape is agg-distribute-CTEConsumer + // distribute is gather + if (requireGather(requiredChildProperty)) { + return true; + } + // group by key is skew + return skewOnShuffleExpr(aggregate); - // agg(group by x)-union all(A, B) - // no matter x.ndv is high or not, it is not worthwhile to shuffle A and B by x - // and hence we forbid one phase agg - if (agg.getAggMode() == AggMode.INPUT_TO_RESULT - && children.get(0).getPlan() instanceof PhysicalUnion - && !((PhysicalUnion) children.get(0).getPlan()).isDistinct()) { - return ImmutableList.of(); + } else { + return true; } - // forbid multi distinct opt that bad than multi-stage version when multi-stage can be executed in one fragment - if (agg.getAggMode() == AggMode.INPUT_TO_BUFFER || agg.getAggMode() == AggMode.INPUT_TO_RESULT) { - List multiDistinctions = agg.getOutputExpressions().stream() - .filter(Alias.class::isInstance) - .map(a -> ((Alias) a).child()) - .filter(AggregateExpression.class::isInstance) - .map(a -> ((AggregateExpression) a).getFunction()) - .filter(MultiDistinction.class::isInstance) - .map(MultiDistinction.class::cast) - .collect(Collectors.toList()); - if (multiDistinctions.size() == 1) { - Expression distinctChild = multiDistinctions.get(0).child(0); - DistributionSpec childDistribution = originChildrenProperties.get(0).getDistributionSpec(); - if (distinctChild instanceof SlotReference && childDistribution instanceof DistributionSpecHash) { - SlotReference slotReference = (SlotReference) distinctChild; - DistributionSpecHash distributionSpecHash = (DistributionSpecHash) childDistribution; - List groupByColumns = agg.getGroupByExpressions().stream() - .map(SlotReference.class::cast) - .map(SlotReference::getExprId) - .collect(Collectors.toList()); - DistributionSpecHash groupByRequire = new DistributionSpecHash( - groupByColumns, ShuffleType.REQUIRE); - List distinctChildColumns = Lists.newArrayList(slotReference.getExprId()); - distinctChildColumns.add(slotReference.getExprId()); - DistributionSpecHash distinctChildRequire = new DistributionSpecHash( - distinctChildColumns, ShuffleType.REQUIRE); - if ((!groupByColumns.isEmpty() && distributionSpecHash.satisfy(groupByRequire)) - || (groupByColumns.isEmpty() && distributionSpecHash.satisfy(distinctChildRequire))) { - if (!agg.mustUseMultiDistinctAgg()) { - return ImmutableList.of(); - } - } - } - // if distinct without group by key, we prefer three or four stage distinct agg - // because the second phase of multi-distinct only have one instance, and it is slow generally. - if (agg.getOutputExpressions().size() == 1 && agg.getGroupByExpressions().isEmpty() - && !agg.mustUseMultiDistinctAgg()) { - return ImmutableList.of(); - } + } + + private boolean skewOnShuffleExpr(PhysicalHashAggregate agg) { + // if statistic is unknown -> not skew + Statistics aggStatistics = agg.getGroupExpression().get().getOwnerGroup().getStatistics(); + Statistics inputStatistics = agg.getGroupExpression().get().childStatistics(0); + if (aggStatistics == null || inputStatistics == null) { + return false; + } + if (AggregateUtils.hasUnknownStatistics(agg.getGroupByExpressions(), inputStatistics)) { + return false; + } + // There are two cases of skew: + double gbyNdv = aggStatistics.getRowCount(); + // 1. ndv is very low + if (gbyNdv <= AggregateUtils.LOW_NDV_THRESHOLD) { + return true; + } + // 2. There is a hot value, and the ndv of other keys is very low + return isSkew(agg.getGroupByExpressions(), inputStatistics); + } + + // if one group by key has hot value, and others ndv is low -> skew + private boolean isSkew(List groupBy, Statistics inputStatistics) { + for (int i = 0; i < groupBy.size(); ++i) { + Expression expr = groupBy.get(i); + ColumnStatistic colStat = inputStatistics.findColumnStatistics(expr); + if (colStat == null) { + continue; + } + if (colStat.getHotValues() == null) { + continue; + } + List otherExpr = excludeElement(groupBy, i); + double otherNdv = StatsCalculator.estimateGroupByRowCount(otherExpr, inputStatistics); + if (otherNdv <= AggregateUtils.LOW_NDV_THRESHOLD) { + return true; } } - // process must shuffle - return visit(agg, context); + return false; + } + + private static List excludeElement(List list, int index) { + List newList = new ArrayList<>(); + for (int i = 0; i < list.size(); i++) { + if (index != i) { + newList.add(list.get(i)); + } + } + return newList; + } + + private boolean onePhaseAggWithDistribute(PhysicalHashAggregate aggregate) { + return aggregate.getAggMode() == AggMode.INPUT_TO_RESULT + && children.get(0).getPlan() instanceof PhysicalDistribute; + } + + private boolean childIsCTEConsumer() { + List groupExpressions = children.get(0).children().get(0).getPhysicalExpressions(); + if (groupExpressions != null && !groupExpressions.isEmpty()) { + return groupExpressions.get(0).getPlan() instanceof PhysicalCTEConsumer; + } + return false; + } + + private boolean requireGather(PhysicalProperties requiredChildProperty) { + return requiredChildProperty.getDistributionSpec() instanceof DistributionSpecGather; + } + + /* agg(group by x)-union all(A, B) + * no matter x.ndv is high or not, it is not worthwhile to shuffle A and B by x + * and hence we forbid one phase agg */ + private boolean banAggUnionAll(PhysicalHashAggregate aggregate) { + return aggregate.getAggMode() == AggMode.INPUT_TO_RESULT + && children.get(0).getPlan() instanceof PhysicalUnion + && !((PhysicalUnion) children.get(0).getPlan()).isDistinct(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java index 4fb1ad88a997ea..05ed27603078e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java @@ -25,10 +25,13 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow.WindowFrameGroup; +import org.apache.doris.nereids.stats.StatsCalculator; 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.OrderExpression; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.DistributeType; import org.apache.doris.nereids.trees.plans.Plan; @@ -40,6 +43,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalDictionarySink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalHiveTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalIcebergTableSink; @@ -54,17 +58,26 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.AggregateUtils; import org.apache.doris.nereids.util.JoinUtils; +import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.Statistics; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +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; import java.util.stream.Collectors; /** @@ -105,14 +118,6 @@ public List> getRequestChildrenPropertyList(GroupExpres @Override public Void visit(Plan plan, PlanContext context) { - if (plan instanceof RequirePropertiesSupplier) { - RequireProperties requireProperties = ((RequirePropertiesSupplier) plan).getRequireProperties(); - List requestPhysicalProperties = - requireProperties.computeRequirePhysicalProperties(plan, requestPropertyFromParent); - addRequestPropertyToChildren(requestPhysicalProperties); - return null; - } - List requiredPropertyList = Lists.newArrayListWithCapacity(context.arity()); for (int i = context.arity(); i > 0; --i) { @@ -413,6 +418,76 @@ public Void visitPhysicalWindow(PhysicalWindow window, PlanConte return null; } + @Override + public Void visitPhysicalHashAggregate(PhysicalHashAggregate agg, PlanContext context) { + DistributionSpec parentDist = requestPropertyFromParent.getDistributionSpec(); + if (agg.getAggPhase().isLocal()) { + addRequestPropertyToChildren(PhysicalProperties.ANY); + return null; + } else if (agg.getAggPhase().isGlobal()) { + if (agg.getPartitionExpressions().isPresent() && !agg.getPartitionExpressions().get().isEmpty()) { + addRequestPropertyToChildren( + PhysicalProperties.createHash(agg.getPartitionExpressions().get(), ShuffleType.REQUIRE)); + return null; + } + if (agg.getGroupByExpressions().isEmpty()) { + addRequestPropertyToChildren(PhysicalProperties.GATHER); + return null; + } + List groupByExprIds = agg.getGroupByExpressions().stream() + .filter(SlotReference.class::isInstance) + .map(SlotReference.class::cast) + .map(SlotReference::getExprId) + .collect(Collectors.toList()); + // If the request received by agg is (a), the request sent by agg is (a,b), and (a) is a subset of (a,b), + // then agg sends (a) to the child + if (parentDist instanceof DistributionSpecHash) { + DistributionSpecHash distributionRequestFromParent = (DistributionSpecHash) parentDist; + List parentHashExprIds = distributionRequestFromParent.getOrderedShuffledColumns(); + Set intersectId = Sets.intersection(new HashSet<>(parentHashExprIds), + new HashSet<>(groupByExprIds)); + if (!intersectId.isEmpty() && intersectId.size() < groupByExprIds.size()) { + if (shouldUseParent(parentHashExprIds, agg)) { + addRequestPropertyToChildren(PhysicalProperties.createHash( + Utils.fastToImmutableList(intersectId), ShuffleType.REQUIRE)); + } + addRequestPropertyToChildren(PhysicalProperties.createHash(groupByExprIds, ShuffleType.REQUIRE)); + return null; + } + } + addRequestPropertyToChildren(PhysicalProperties.createHash(groupByExprIds, ShuffleType.REQUIRE)); + return null; + } + return null; + } + + private boolean shouldUseParent(List parentHashExprIds, PhysicalHashAggregate agg) { + Optional groupExpression = agg.getGroupExpression(); + if (!groupExpression.isPresent()) { + return true; + } + Statistics aggChildStats = groupExpression.get().childStatistics(0); + if (aggChildStats == null) { + return true; + } + List aggChildOutput = agg.child().getOutput(); + Map exprIdSlotMap = new HashMap<>(); + for (Slot slot : aggChildOutput) { + exprIdSlotMap.put(slot.getExprId(), slot); + } + List parentHashExprs = new ArrayList<>(parentHashExprIds.size()); + for (ExprId exprId : parentHashExprIds) { + if (exprIdSlotMap.containsKey(exprId)) { + parentHashExprs.add(exprIdSlotMap.get(exprId)); + } + } + if (AggregateUtils.hasUnknownStatistics(parentHashExprs, aggChildStats)) { + return true; + } + double combinedNdv = StatsCalculator.estimateGroupByRowCount(parentHashExprs, aggChildStats); + return combinedNdv > AggregateUtils.LOW_NDV_THRESHOLD; + } + private List createHashRequestAccordingToParent( SetOperation setOperation, DistributionSpecHash distributionRequestFromParent, PlanContext context) { List requiredPropertyList = diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequirePropertiesSupplier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequirePropertiesSupplier.java deleted file mode 100644 index 7f3f6084ff85dc..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequirePropertiesSupplier.java +++ /dev/null @@ -1,65 +0,0 @@ -// 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.properties; - -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.properties.RequireProperties.RequirePropertiesTree; -import org.apache.doris.nereids.trees.plans.Plan; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; - -import java.util.List; - -/** RequirePropertiesSupplier */ -public interface RequirePropertiesSupplier

{ - List children(); - - RequireProperties getRequireProperties(); - - Plan withRequireAndChildren(RequireProperties requireProperties, List children); - - default P withRequire(RequireProperties requireProperties) { - return (P) withRequireAndChildren(requireProperties, children()); - } - - /** withRequireTree */ - default P withRequireTree(RequirePropertiesTree tree) { - List childrenRequires = tree.children; - List children = children(); - if (!childrenRequires.isEmpty() && children.size() != childrenRequires.size()) { - throw new AnalysisException("The number of RequireProperties mismatch the plan tree"); - } - - List newChildren = children; - if (!childrenRequires.isEmpty()) { - ImmutableList.Builder newChildrenBuilder = - ImmutableList.builderWithExpectedSize(childrenRequires.size()); - for (int i = 0; i < children.size(); i++) { - Plan child = children.get(i); - Preconditions.checkState(child instanceof RequirePropertiesSupplier, - "child should be RequirePropertiesTree: " + child); - Plan newChild = ((RequirePropertiesSupplier) child).withRequireTree(childrenRequires.get(i)); - newChildrenBuilder.add(newChild); - } - newChildren = newChildrenBuilder.build(); - } - - return (P) withRequireAndChildren(tree.requireProperties, newChildren); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index 9fc6b4d573d34f..6e47834d6c92ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -87,6 +87,9 @@ import org.apache.doris.nereids.rules.implementation.LogicalTopNToPhysicalTopN; import org.apache.doris.nereids.rules.implementation.LogicalUnionToPhysicalUnion; import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow; +import org.apache.doris.nereids.rules.implementation.SplitAggMultiPhase; +import org.apache.doris.nereids.rules.implementation.SplitAggMultiPhaseWithoutGbyKey; +import org.apache.doris.nereids.rules.implementation.SplitAggWithoutDistinct; import org.apache.doris.nereids.rules.rewrite.CreatePartitionTopNFromWindow; import org.apache.doris.nereids.rules.rewrite.EliminateFilter; import org.apache.doris.nereids.rules.rewrite.EliminateOuterJoin; @@ -202,6 +205,9 @@ public class RuleSet { .add(new LogicalEmptyRelationToPhysicalEmptyRelation()) .add(new LogicalTVFRelationToPhysicalTVFRelation()) .add(new AggregateStrategies()) + .add(SplitAggWithoutDistinct.INSTANCE) + .add(SplitAggMultiPhase.INSTANCE) + .add(SplitAggMultiPhaseWithoutGbyKey.INSTANCE) .add(new LogicalUnionToPhysicalUnion()) .add(new LogicalExceptToPhysicalExcept()) .add(new LogicalIntersectToPhysicalIntersect()) 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 0f2c77b6a1cb95..cd7bbd89712543 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 @@ -337,7 +337,7 @@ public enum RuleType { MERGE_TOP_N(RuleTypeClass.REWRITE), BUILD_AGG_FOR_UNION(RuleTypeClass.REWRITE), COUNT_DISTINCT_REWRITE(RuleTypeClass.REWRITE), - SPLIT_MULTI_DISTINCT(RuleTypeClass.REWRITE), + DISTINCT_AGG_STRATEGY_SELECTOR(RuleTypeClass.REWRITE), INNER_TO_CROSS_JOIN(RuleTypeClass.REWRITE), CROSS_TO_INNER_JOIN(RuleTypeClass.REWRITE), PRUNE_EMPTY_PARTITION(RuleTypeClass.REWRITE), @@ -399,6 +399,10 @@ public enum RuleType { SHOR_CIRCUIT_POINT_QUERY(RuleTypeClass.REWRITE), // skew rewrtie SALT_JOIN(RuleTypeClass.REWRITE), + + DISTINCT_AGGREGATE_SPLIT(RuleTypeClass.REWRITE), + PROCESS_SCALAR_AGG_MUST_USE_MULTI_DISTINCT(RuleTypeClass.REWRITE), + // exploration rules REORDER_INTERSECT(RuleTypeClass.EXPLORATION), TEST_EXPLORATION(RuleTypeClass.EXPLORATION), @@ -505,23 +509,17 @@ public enum RuleType { STORAGE_LAYER_AGGREGATE_MINMAX_ON_UNIQUE_WITHOUT_PROJECT(RuleTypeClass.IMPLEMENTATION), COUNT_ON_INDEX(RuleTypeClass.IMPLEMENTATION), COUNT_ON_INDEX_WITHOUT_PROJECT(RuleTypeClass.IMPLEMENTATION), - ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT(RuleTypeClass.IMPLEMENTATION), - TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT(RuleTypeClass.IMPLEMENTATION), - TWO_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI(RuleTypeClass.IMPLEMENTATION), - THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI(RuleTypeClass.IMPLEMENTATION), TWO_PHASE_AGGREGATE_WITH_DISTINCT(RuleTypeClass.IMPLEMENTATION), - ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI(RuleTypeClass.IMPLEMENTATION), - TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI(RuleTypeClass.IMPLEMENTATION), - TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT(RuleTypeClass.IMPLEMENTATION), - THREE_PHASE_AGGREGATE_WITH_DISTINCT(RuleTypeClass.IMPLEMENTATION), - FOUR_PHASE_AGGREGATE_WITH_DISTINCT(RuleTypeClass.IMPLEMENTATION), - FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE(RuleTypeClass.IMPLEMENTATION), LOGICAL_UNION_TO_PHYSICAL_UNION(RuleTypeClass.IMPLEMENTATION), LOGICAL_EXCEPT_TO_PHYSICAL_EXCEPT(RuleTypeClass.IMPLEMENTATION), LOGICAL_INTERSECT_TO_PHYSICAL_INTERSECT(RuleTypeClass.IMPLEMENTATION), LOGICAL_GENERATE_TO_PHYSICAL_GENERATE(RuleTypeClass.IMPLEMENTATION), LOGICAL_WINDOW_TO_PHYSICAL_WINDOW_RULE(RuleTypeClass.IMPLEMENTATION), AGG_SKEW_REWRITE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_AGGREGATE_TO_PHYSICAL_HASH_AGGREGATE(RuleTypeClass.IMPLEMENTATION), + SPLIT_AGG_WITHOUT_DISTINCT(RuleTypeClass.IMPLEMENTATION), + SPLIT_AGG_MULTI_PHASE(RuleTypeClass.IMPLEMENTATION), + SPLIT_AGG_MULTI_PHASE_WITHOUT_GBY_KEY(RuleTypeClass.IMPLEMENTATION), IMPLEMENTATION_SENTINEL(RuleTypeClass.IMPLEMENTATION), // sentinel, use to count rules diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PreMaterializedViewRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PreMaterializedViewRewriter.java index 8611224ac388aa..db8290b95f8662 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PreMaterializedViewRewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PreMaterializedViewRewriter.java @@ -64,9 +64,11 @@ public class PreMaterializedViewRewriter { NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.ELIMINATE_CONST_JOIN_CONDITION.ordinal()); NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.MERGE_PERCENTILE_TO_ARRAY.ordinal()); NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.SUM_LITERAL_REWRITE.ordinal()); - NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.SPLIT_MULTI_DISTINCT.ordinal()); + NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.DISTINCT_AGG_STRATEGY_SELECTOR.ordinal()); NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.CONSTANT_PROPAGATION.ordinal()); NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.PUSH_DOWN_VIRTUAL_COLUMNS_INTO_OLAP_SCAN.ordinal()); + NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.DISTINCT_AGGREGATE_SPLIT.ordinal()); + NEED_PRE_REWRITE_RULE_TYPES.set(RuleType.PROCESS_SCALAR_AGG_MUST_USE_MULTI_DISTINCT.ordinal()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java index 603dcba98a814b..8e52c8aa28c996 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java @@ -24,51 +24,26 @@ import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.annotation.DependsRules; -import org.apache.doris.nereids.pattern.PatternDescriptor; -import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; -import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.properties.RequireProperties; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; -import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; -import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.IsNull; -import org.apache.doris.nereids.trees.expressions.Mod; -import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Or; -import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; -import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; -import org.apache.doris.nereids.trees.expressions.functions.agg.AggregatePhase; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; -import org.apache.doris.nereids.trees.expressions.functions.agg.GroupConcat; 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.functions.agg.SupportMultiDistinct; -import org.apache.doris.nereids.trees.expressions.functions.scalar.If; -import org.apache.doris.nereids.trees.expressions.functions.scalar.XxHash32; -import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; -import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; -import org.apache.doris.nereids.trees.plans.AggMode; -import org.apache.doris.nereids.trees.plans.AggPhase; -import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.algebra.Aggregate; import org.apache.doris.nereids.trees.plans.algebra.Project; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; @@ -78,34 +53,16 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate.PushDownAggOp; -import org.apache.doris.nereids.types.DataType; -import org.apache.doris.nereids.types.SmallIntType; -import org.apache.doris.nereids.types.StringType; -import org.apache.doris.nereids.types.TinyIntType; import org.apache.doris.nereids.util.ExpressionUtils; -import org.apache.doris.nereids.util.TypeCoercionUtils; -import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -117,12 +74,8 @@ FoldConstantRuleOnFE.class }) public class AggregateStrategies implements ImplementationRuleFactory { - private static final String SALT_EXPR = "saltExpr"; - @Override public List buildRules() { - PatternDescriptor> basePattern = logicalAggregate(); - return ImmutableList.of( RuleType.COUNT_ON_INDEX_WITHOUT_PROJECT.build( logicalAggregate( @@ -302,161 +255,6 @@ public List buildRules() { LogicalFileScan fileScan = project.child(); return storageLayerAggregate(agg, project, fileScan, ctx.cascadesContext); }) - ), - RuleType.ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT.build( - basePattern - .when(agg -> agg.getDistinctArguments().isEmpty()) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.ONE)) - .thenApplyMulti(ctx -> onePhaseAggregateWithoutDistinct(ctx.root, ctx.connectContext)) - ), - RuleType.TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT.build( - basePattern - .when(agg -> agg.getDistinctArguments().isEmpty()) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) - .thenApplyMulti(ctx -> twoPhaseAggregateWithoutDistinct(ctx.root, ctx.connectContext)) - ), - // RuleType.TWO_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI.build( - // basePattern - // .when(this::containsCountDistinctMultiExpr) - // .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) - // .thenApplyMulti(ctx -> twoPhaseAggregateWithCountDistinctMulti(ctx.root, ctx.cascadesContext)) - // ), - RuleType.THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI.build( - basePattern - .when(this::containsCountDistinctMultiExpr) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.THREE)) - .thenApplyMulti(ctx -> threePhaseAggregateWithCountDistinctMulti(ctx.root, ctx.cascadesContext)) - ), - RuleType.ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI.build( - basePattern - .when(agg -> agg.getDistinctArguments().size() == 1 && couldConvertToMulti(agg)) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.ONE)) - .thenApplyMulti(ctx -> onePhaseAggregateWithMultiDistinct(ctx.root, ctx.connectContext)) - ), - RuleType.TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI.build( - basePattern - .when(agg -> agg.getDistinctArguments().size() == 1 && couldConvertToMulti(agg)) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) - .whenNot(Aggregate::canSkewRewrite) - .thenApplyMulti(ctx -> twoPhaseAggregateWithMultiDistinct(ctx.root, ctx.connectContext)) - ), - RuleType.TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT.build( - basePattern - .when(agg -> agg.getDistinctArguments().size() > 1 - && !containsCountDistinctMultiExpr(agg) - && couldConvertToMulti(agg)) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) - .thenApplyMulti(ctx -> twoPhaseAggregateWithMultiDistinct(ctx.root, ctx.connectContext)) - ), - // RuleType.TWO_PHASE_AGGREGATE_WITH_DISTINCT.build( - // basePattern - // .when(agg -> agg.getDistinctArguments().size() == 1) - // .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) - // .thenApplyMulti(ctx -> twoPhaseAggregateWithDistinct(ctx.root, ctx.connectContext)) - // ), - RuleType.THREE_PHASE_AGGREGATE_WITH_DISTINCT.build( - basePattern - .when(agg -> agg.getDistinctArguments().size() == 1) - .whenNot(agg -> agg.mustUseMultiDistinctAgg()) - .whenNot(Aggregate::canSkewRewrite) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.THREE)) - .thenApplyMulti(ctx -> threePhaseAggregateWithDistinct(ctx.root, ctx.connectContext)) - ), - /* - * sql: - * select count(distinct name), sum(age) from student; - *

- * 4 phase plan - * DISTINCT_GLOBAL(BUFFER_TO_RESULT, groupBy(), - * output[count(partial_count(name)), sum(partial_sum(partial_sum(age)))], - * GATHER) - * +--DISTINCT_LOCAL(INPUT_TO_BUFFER, groupBy(), - * output(partial_count(name), partial_sum(partial_sum(age))), - * hash distribute by name) - * +--GLOBAL(BUFFER_TO_BUFFER, groupBy(name), - * output(name, partial_sum(age)), - * hash_distribute by name) - * +--LOCAL(INPUT_TO_BUFFER, groupBy(name), output(name, partial_sum(age))) - * +--scan(name, age) - */ - RuleType.FOUR_PHASE_AGGREGATE_WITH_DISTINCT.build( - basePattern - .when(agg -> agg.getDistinctArguments().size() == 1) - .when(agg -> agg.getGroupByExpressions().isEmpty()) - .whenNot(agg -> agg.mustUseMultiDistinctAgg()) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.FOUR)) - .thenApplyMulti(ctx -> { - Function, RequireProperties> secondPhaseRequireDistinctHash = - groupByAndDistinct -> RequireProperties.of( - PhysicalProperties.createHash( - ctx.root.getDistinctArguments(), ShuffleType.REQUIRE - ) - ); - Function, RequireProperties> fourPhaseRequireGather = - agg -> RequireProperties.of(PhysicalProperties.GATHER); - return fourPhaseAggregateWithDistinct( - ctx.root, ctx.connectContext, - secondPhaseRequireDistinctHash, fourPhaseRequireGather - ); - }) - ), - /* - * sql: - * select age, count(distinct name) from student group by age; - *

- * 4 phase plan - * DISTINCT_GLOBAL(BUFFER_TO_RESULT, groupBy(age), - * output[age, sum(partial_count(name))], - * hash distribute by name) - * +--DISTINCT_LOCAL(INPUT_TO_BUFFER, groupBy(age), - * output(age, partial_count(name)), - * hash distribute by age, name) - * +--GLOBAL(BUFFER_TO_BUFFER, groupBy(age, name), - * output(age, name), - * hash_distribute by age, name) - * +--LOCAL(INPUT_TO_BUFFER, groupBy(age, name), output(age, name)) - * +--scan(age, name) - */ - RuleType.FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE.build( - basePattern - .when(agg -> agg.everyDistinctArgumentNumIsOne() && !agg.getGroupByExpressions().isEmpty()) - .when(agg -> - ImmutableSet.builder() - .addAll(agg.getGroupByExpressions()) - .addAll(agg.getDistinctArguments()) - .build().size() > agg.getGroupByExpressions().size() - ) - .when(agg -> { - if (agg.getDistinctArguments().size() == 1) { - return true; - } - return couldConvertToMulti(agg); - }) - .when(agg -> agg.supportAggregatePhase(AggregatePhase.FOUR)) - .whenNot(Aggregate::mustUseMultiDistinctAgg) - .whenNot(Aggregate::canSkewRewrite) - .thenApplyMulti(ctx -> { - Function, RequireProperties> secondPhaseRequireGroupByAndDistinctHash = - groupByAndDistinct -> RequireProperties.of( - PhysicalProperties.createHash(groupByAndDistinct, ShuffleType.REQUIRE) - ); - - Function, RequireProperties> fourPhaseRequireGroupByHash = - agg -> RequireProperties.of( - PhysicalProperties.createHash( - agg.getGroupByExpressions(), ShuffleType.REQUIRE - ) - ); - return fourPhaseAggregateWithDistinct( - ctx.root, ctx.connectContext, - secondPhaseRequireGroupByAndDistinctHash, fourPhaseRequireGroupByHash - ); - }) - ), - RuleType.AGG_SKEW_REWRITE.build( - basePattern - .when(Aggregate::canSkewRewrite) - .thenApply(ctx -> aggSkewRewrite(ctx.root, ctx.cascadesContext)) ) ); } @@ -916,1346 +714,8 @@ private boolean enablePushDownStringMinMax() { return connectContext != null && connectContext.getSessionVariable().isEnablePushDownStringMinMax(); } - /** - * sql: select count(*) from tbl group by id - *

- * before: - *

- * LogicalAggregate(groupBy=[id], output=[count(*)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[id], output=[count(*)]) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[id], output=[count(*)]) - * | - * LogicalOlapScan(table=tbl, **already distribute by id**) - * - */ - private List> onePhaseAggregateWithoutDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext) { - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - boolean canBeBanned = true; - for (AggregateFunction aggregateFunction : logicalAgg.getAggregateFunctions()) { - if (aggregateFunction.forceSkipRegulator(AggregatePhase.ONE)) { - canBeBanned = false; - break; - } - } - AggregateParam inputToResultParam = new AggregateParam( - AggregateParam.LOCAL_RESULT.aggPhase, AggregateParam.LOCAL_RESULT.aggMode, canBeBanned - ); - List newOutput = ExpressionUtils.rewriteDownShortCircuit( - logicalAgg.getOutputExpressions(), outputChild -> { - if (outputChild instanceof AggregateFunction) { - return new AggregateExpression((AggregateFunction) outputChild, inputToResultParam); - } - return outputChild; - }); - PhysicalHashAggregate gatherLocalAgg = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), newOutput, Optional.empty(), - inputToResultParam, false, - logicalAgg.getLogicalProperties(), - requireGather, logicalAgg.child()); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - // TODO: usually bad, disable it until we could do better cost computation. - if (!canBeBanned) { - return ImmutableList.of(gatherLocalAgg); - } else { - return ImmutableList.of(); - } - } else { - RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate hashLocalAgg = gatherLocalAgg - .withRequire(requireHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // TODO: usually bad, disable it until we could do better cost computation. - //.add(gatherLocalAgg) - .add(hashLocalAgg) - .build(); - } - } - - /** - * sql: select count(distinct id, name) from tbl group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[count(distinct id, name)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[count(if(id is null, null, name))]) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id]) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[count(if(id is null, null, name))]) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id]) - * | - * PhysicalDistribute(distributionSpec=HASH(name)) - * | - * LogicalOlapScan(table=tbl, **already distribute by name**) - * - */ - private List> twoPhaseAggregateWithCountDistinctMulti( - LogicalAggregate logicalAgg, CascadesContext cascadesContext) { - AggregateParam inputToBufferParam = AggregateParam.LOCAL_BUFFER; - Collection countDistinctArguments = logicalAgg.getDistinctArguments(); - - List localAggGroupBy = ImmutableList.copyOf(ImmutableSet.builder() - .addAll(logicalAgg.getGroupByExpressions()) - .addAll(countDistinctArguments) - .build()); - - Set aggregateFunctions = logicalAgg.getAggregateFunctions(); - - Map nonDistinctAggFunctionToAliasPhase1 = aggregateFunctions.stream() - .filter(aggregateFunction -> !aggregateFunction.isDistinct()) - .collect(ImmutableMap.toImmutableMap(expr -> expr, expr -> { - AggregateExpression localAggExpr = new AggregateExpression(expr, inputToBufferParam); - return new Alias(localAggExpr); - })); - - List partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg); - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - List localOutput = ImmutableList.builder() - .addAll((List) (List) localAggGroupBy.stream() - .filter(g -> !(g instanceof Literal)) - .collect(ImmutableList.toImmutableList())) - .addAll(nonDistinctAggFunctionToAliasPhase1.values()) - .build(); - PhysicalHashAggregate gatherLocalAgg = new PhysicalHashAggregate<>( - localAggGroupBy, localOutput, Optional.of(partitionExpressions), - new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER), - maybeUsingStreamAgg(cascadesContext.getConnectContext(), logicalAgg), - logicalAgg.getLogicalProperties(), requireGather, logicalAgg.child() - ); - - List distinctGroupBy = logicalAgg.getGroupByExpressions(); - - LogicalAggregate countIfAgg = countDistinctMultiExprToCountIf( - logicalAgg, cascadesContext).first; - - AggregateParam distinctInputToResultParam - = new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_RESULT); - AggregateParam globalBufferToResultParam - = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); - List distinctOutput = ExpressionUtils.rewriteDownShortCircuit( - countIfAgg.getOutputExpressions(), outputChild -> { - if (outputChild instanceof AggregateFunction) { - AggregateFunction aggregateFunction = (AggregateFunction) outputChild; - Alias alias = nonDistinctAggFunctionToAliasPhase1.get(aggregateFunction); - if (alias == null) { - return new AggregateExpression(aggregateFunction, distinctInputToResultParam); - } else { - return new AggregateExpression(aggregateFunction, - globalBufferToResultParam, alias.toSlot()); - } - } else { - return outputChild; - } - }); - - PhysicalHashAggregate gatherLocalGatherDistinctAgg = new PhysicalHashAggregate<>( - distinctGroupBy, distinctOutput, Optional.of(partitionExpressions), - distinctInputToResultParam, false, - logicalAgg.getLogicalProperties(), requireGather, gatherLocalAgg - ); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - return ImmutableList.of(gatherLocalGatherDistinctAgg); - } else { - RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate hashLocalHashGlobalAgg = gatherLocalGatherDistinctAgg - .withRequireTree(requireHash.withChildren(requireHash)) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // TODO: usually bad, disable it until we could do better cost computation. - //.add(gatherLocalGatherDistinctAgg) - .add(hashLocalHashGlobalAgg) - .build(); - } - } - - /** - * sql: select count(distinct id, name) from tbl group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[count(distinct id, name)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[count(if(id is null, null, name))]) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=BUFFER_TO_BUFFER) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[count(if(id is null, null, name))]) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=BUFFER_TO_BUFFER) - * | - * PhysicalDistribute(distributionSpec=HASH(name)) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - * - */ - private List> threePhaseAggregateWithCountDistinctMulti( - LogicalAggregate logicalAgg, CascadesContext cascadesContext) { - AggregateParam inputToBufferParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER); - - Collection countDistinctArguments = logicalAgg.getDistinctArguments(); - - List localAggGroupBy = ImmutableList.copyOf(ImmutableSet.builder() - .addAll(logicalAgg.getGroupByExpressions()) - .addAll(countDistinctArguments) - .build()); - - Set aggregateFunctions = logicalAgg.getAggregateFunctions(); - - Map nonDistinctAggFunctionToAliasPhase1 = aggregateFunctions.stream() - .filter(aggregateFunction -> !aggregateFunction.isDistinct()) - .collect(ImmutableMap.toImmutableMap(expr -> expr, expr -> { - AggregateExpression localAggExpr = new AggregateExpression(expr, inputToBufferParam); - return new Alias(localAggExpr); - })); - - List partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg); - RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY); - List localOutput = ImmutableList.builder() - .addAll((List) (List) localAggGroupBy.stream() - .filter(g -> !(g instanceof Literal)) - .collect(ImmutableList.toImmutableList())) - .addAll(nonDistinctAggFunctionToAliasPhase1.values()) - .build(); - PhysicalHashAggregate anyLocalAgg = new PhysicalHashAggregate<>( - localAggGroupBy, localOutput, Optional.of(partitionExpressions), - new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER), - maybeUsingStreamAgg(cascadesContext.getConnectContext(), logicalAgg), - logicalAgg.getLogicalProperties(), requireAny, logicalAgg.child() - ); - - List globalAggGroupBy = localAggGroupBy; - - boolean hasCountDistinctMulti = logicalAgg.getAggregateFunctions().stream() - .filter(AggregateFunction::isDistinct) - .filter(Count.class::isInstance) - .anyMatch(c -> c.arity() > 1); - AggregateParam bufferToBufferParam = new AggregateParam( - AggPhase.GLOBAL, AggMode.BUFFER_TO_BUFFER, !hasCountDistinctMulti); - - Map nonDistinctAggFunctionToAliasPhase2 = - nonDistinctAggFunctionToAliasPhase1.entrySet() - .stream() - .collect(ImmutableMap.toImmutableMap(Entry::getKey, kv -> { - AggregateFunction originFunction = kv.getKey(); - Alias localOutputAlias = kv.getValue(); - AggregateExpression globalAggExpr = new AggregateExpression( - originFunction, bufferToBufferParam, localOutputAlias.toSlot()); - return new Alias(globalAggExpr); - })); - - Set slotInCountDistinct = ExpressionUtils.collect( - ImmutableList.copyOf(countDistinctArguments), SlotReference.class::isInstance); - List globalAggOutput = ImmutableList.copyOf(ImmutableSet.builder() - .addAll((List) (List) logicalAgg.getGroupByExpressions()) - .addAll(slotInCountDistinct) - .addAll(nonDistinctAggFunctionToAliasPhase2.values()) - .build()); - - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - PhysicalHashAggregate anyLocalGatherGlobalAgg = new PhysicalHashAggregate<>( - globalAggGroupBy, globalAggOutput, Optional.of(partitionExpressions), - bufferToBufferParam, false, logicalAgg.getLogicalProperties(), - requireGather, anyLocalAgg); - - LogicalAggregate countIfAgg = countDistinctMultiExprToCountIf( - logicalAgg, cascadesContext).first; - - AggregateParam distinctInputToResultParam - = new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_RESULT, !hasCountDistinctMulti); - AggregateParam globalBufferToResultParam - = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); - List distinctOutput = ExpressionUtils.rewriteDownShortCircuit( - countIfAgg.getOutputExpressions(), outputChild -> { - if (outputChild instanceof AggregateFunction) { - AggregateFunction aggregateFunction = (AggregateFunction) outputChild; - Alias alias = nonDistinctAggFunctionToAliasPhase2.get(aggregateFunction); - if (alias == null) { - return new AggregateExpression(aggregateFunction, distinctInputToResultParam); - } else { - return new AggregateExpression(aggregateFunction, - globalBufferToResultParam, alias.toSlot()); - } - } else { - return outputChild; - } - }); - - PhysicalHashAggregate anyLocalGatherGlobalGatherAgg = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), distinctOutput, Optional.empty(), - distinctInputToResultParam, false, - logicalAgg.getLogicalProperties(), requireGather, anyLocalGatherGlobalAgg - ); - - // RequireProperties requireDistinctHash = RequireProperties.of( - // PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - // PhysicalHashAggregate anyLocalHashGlobalGatherDistinctAgg - // = anyLocalGatherGlobalGatherAgg.withChildren(ImmutableList.of( - // anyLocalGatherGlobalAgg - // .withRequire(requireDistinctHash) - // .withPartitionExpressions(ImmutableList.copyOf(logicalAgg.getDistinctArguments())) - // )); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - return ImmutableList.>builder() - .add(anyLocalGatherGlobalGatherAgg) - //.add(anyLocalHashGlobalGatherDistinctAgg) - .build(); - } else { - RequireProperties requireGroupByHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate> anyLocalHashGlobalHashDistinctAgg - = anyLocalGatherGlobalGatherAgg.withRequirePropertiesAndChild(requireGroupByHash, - anyLocalGatherGlobalAgg - .withRequire(requireGroupByHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()) - ) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // .add(anyLocalGatherGlobalGatherAgg) - // .add(anyLocalHashGlobalGatherDistinctAgg) - .add(anyLocalHashGlobalHashDistinctAgg) - .build(); - } - } - - /** - * sql: select name, count(value) from tbl group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[name, count(value)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, count(value)], mode=BUFFER_TO_RESULT) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * PhysicalHashAggregate(groupBy=[name], output=[name, count(value)], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, count(value)], mode=BUFFER_TO_RESULT) - * | - * PhysicalDistribute(distributionSpec=HASH(name)) - * | - * PhysicalHashAggregate(groupBy=[name], output=[name, count(value)], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - * - */ - private List> twoPhaseAggregateWithoutDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext) { - AggregateParam inputToBufferParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER); - Map inputToBufferAliases = logicalAgg.getAggregateFunctions() - .stream() - .collect(ImmutableMap.toImmutableMap(function -> function, function -> { - AggregateExpression inputToBuffer = new AggregateExpression(function, inputToBufferParam); - return new Alias(inputToBuffer); - })); - - List localAggGroupBy = logicalAgg.getGroupByExpressions(); - List partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg); - List localAggOutput = ImmutableList.builder() - // we already normalized the group by expressions to List by the NormalizeAggregate rule - .addAll((List) localAggGroupBy) - .addAll(inputToBufferAliases.values()) - .build(); - - RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY); - PhysicalHashAggregate anyLocalAgg = new PhysicalHashAggregate<>( - localAggGroupBy, localAggOutput, Optional.of(partitionExpressions), - inputToBufferParam, maybeUsingStreamAgg(connectContext, logicalAgg), - logicalAgg.getLogicalProperties(), requireAny, - logicalAgg.child()); - - AggregateParam bufferToResultParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); - List globalAggOutput = ExpressionUtils.rewriteDownShortCircuit( - logicalAgg.getOutputExpressions(), outputChild -> { - if (!(outputChild instanceof AggregateFunction)) { - return outputChild; - } - Alias inputToBufferAlias = inputToBufferAliases.get(outputChild); - if (inputToBufferAlias == null) { - return outputChild; - } - AggregateFunction function = (AggregateFunction) outputChild; - return new AggregateExpression(function, bufferToResultParam, inputToBufferAlias.toSlot()); - }); - - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - PhysicalHashAggregate anyLocalGatherGlobalAgg = new PhysicalHashAggregate<>( - localAggGroupBy, globalAggOutput, Optional.of(partitionExpressions), - bufferToResultParam, false, anyLocalAgg.getLogicalProperties(), - requireGather, anyLocalAgg); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - return ImmutableList.of(anyLocalGatherGlobalAgg); - } else { - RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - - PhysicalHashAggregate anyLocalHashGlobalAgg = anyLocalGatherGlobalAgg - .withRequire(requireHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // TODO: usually bad, disable it until we could do better cost computation. - // .add(anyLocalGatherGlobalAgg) - .add(anyLocalHashGlobalAgg) - .build(); - } - } - - /** - * sql: select count(distinct id) from tbl group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[name, count(distinct id)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, count(distinct(id))], mode=BUFFER_TO_RESULT) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=INPUT_TO_BUFFER) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, count(distinct(id))], mode=BUFFER_TO_RESULT) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl, **if distribute by name**) - * - */ - private List> twoPhaseAggregateWithDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext) { - Set aggregateFunctions = logicalAgg.getAggregateFunctions(); - - Set distinctArguments = aggregateFunctions.stream() - .filter(AggregateFunction::isDistinct) - .flatMap(aggregateExpression -> aggregateExpression.getArguments().stream()) - .filter(NamedExpression.class::isInstance) - .map(NamedExpression.class::cast) - .collect(ImmutableSet.toImmutableSet()); - - Set localAggGroupBy = ImmutableSet.builder() - .addAll((List) (List) logicalAgg.getGroupByExpressions()) - .addAll(distinctArguments) - .build(); - - AggregateParam inputToBufferParam = AggregateParam.LOCAL_BUFFER; - - Map nonDistinctAggFunctionToAliasPhase1 = aggregateFunctions.stream() - .filter(aggregateFunction -> !aggregateFunction.isDistinct()) - .collect(ImmutableMap.toImmutableMap(expr -> expr, expr -> { - AggregateExpression localAggExpr = new AggregateExpression(expr, inputToBufferParam); - return new Alias(localAggExpr); - })); - - List localAggOutput = ImmutableList.builder() - .addAll(localAggGroupBy) - .addAll(nonDistinctAggFunctionToAliasPhase1.values()) - .build(); - - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - - List partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg); - PhysicalHashAggregate gatherLocalAgg = new PhysicalHashAggregate<>(ImmutableList.copyOf(localAggGroupBy), - localAggOutput, Optional.of(partitionExpressions), inputToBufferParam, - /* - * should not use streaming, there has some bug in be will compute wrong result, - * see aggregate_strategies.groovy - */ - false, Optional.empty(), logicalAgg.getLogicalProperties(), - requireGather, logicalAgg.child()); - - AggregateParam bufferToResultParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); - List globalOutput = ExpressionUtils.rewriteDownShortCircuit( - logicalAgg.getOutputExpressions(), outputChild -> { - if (outputChild instanceof AggregateFunction) { - AggregateFunction aggregateFunction = (AggregateFunction) outputChild; - if (aggregateFunction.isDistinct()) { - List aggChild = aggregateFunction.children(); - Preconditions.checkArgument(new HashSet<>(aggChild).size() == 1 - || aggregateFunction.getDistinctArguments().size() == 1, - "cannot process more than one child in aggregate distinct function: " - + aggregateFunction); - AggregateFunction nonDistinct = aggregateFunction - .withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); - return new AggregateExpression(nonDistinct, AggregateParam.LOCAL_RESULT); - } else { - Alias alias = nonDistinctAggFunctionToAliasPhase1.get(outputChild); - return new AggregateExpression(aggregateFunction, bufferToResultParam, alias.toSlot()); - } - } else { - return outputChild; - } - }); - - PhysicalHashAggregate gatherLocalGatherGlobalAgg - = new PhysicalHashAggregate<>(logicalAgg.getGroupByExpressions(), globalOutput, - Optional.empty(), bufferToResultParam, false, - logicalAgg.getLogicalProperties(), requireGather, gatherLocalAgg); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - RequireProperties requireDistinctHash = RequireProperties.of(PhysicalProperties.createHash( - distinctArguments, ShuffleType.REQUIRE)); - PhysicalHashAggregate hashLocalGatherGlobalAgg = gatherLocalGatherGlobalAgg - .withChildren(ImmutableList.of(gatherLocalAgg - .withRequire(requireDistinctHash) - .withPartitionExpressions(ImmutableList.copyOf(logicalAgg.getDistinctArguments())) - )); - return ImmutableList.>builder() - //.add(gatherLocalGatherGlobalAgg) - .add(hashLocalGatherGlobalAgg) - .build(); - } else { - RequireProperties requireGroupByHash = RequireProperties.of(PhysicalProperties.createHash( - logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate> hashLocalHashGlobalAgg = gatherLocalGatherGlobalAgg - .withRequirePropertiesAndChild(requireGroupByHash, gatherLocalAgg - .withRequire(requireGroupByHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()) - ) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // .add(gatherLocalGatherGlobalAgg) - .add(hashLocalHashGlobalAgg) - .build(); - } - } - - /** - * sql: select count(distinct id) from tbl group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[name, count(distinct id)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, count(distinct(id))], mode=BUFFER_TO_RESULT) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=BUFFER_TO_BUFFER) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, count(distinct(id))], mode=BUFFER_TO_RESULT) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=BUFFER_TO_BUFFER) - * | - * PhysicalDistribute(distributionSpec=HASH(name)) - * | - * PhysicalHashAggregate(groupBy=[name, id], output=[name, id], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - * - */ - // TODO: support one phase aggregate(group by columns + distinct columns) + two phase distinct aggregate - private List> threePhaseAggregateWithDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext) { - boolean couldBanned = couldConvertToMulti(logicalAgg); - - Set aggregateFunctions = logicalAgg.getAggregateFunctions(); - - Set distinctArguments = aggregateFunctions.stream() - .filter(AggregateFunction::isDistinct) - .flatMap(aggregateExpression -> aggregateExpression.getArguments().stream()) - .filter(NamedExpression.class::isInstance) - .map(NamedExpression.class::cast) - .collect(ImmutableSet.toImmutableSet()); - - Set localAggGroupBySet = ImmutableSet.builder() - .addAll((List) (List) logicalAgg.getGroupByExpressions()) - .addAll(distinctArguments) - .build(); - - AggregateParam inputToBufferParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER, couldBanned); - - Map nonDistinctAggFunctionToAliasPhase1 = aggregateFunctions.stream() - .filter(aggregateFunction -> !aggregateFunction.isDistinct()) - .collect(ImmutableMap.toImmutableMap(expr -> expr, expr -> { - AggregateExpression localAggExpr = new AggregateExpression(expr, inputToBufferParam); - return new Alias(localAggExpr); - })); - - List localAggOutput = ImmutableList.builder() - .addAll(localAggGroupBySet) - .addAll(nonDistinctAggFunctionToAliasPhase1.values()) - .build(); - - List localAggGroupBy = ImmutableList.copyOf(localAggGroupBySet); - boolean isGroupByEmptySelectEmpty = localAggGroupBy.isEmpty() && localAggOutput.isEmpty(); - - // be not recommend generate an aggregate node with empty group by and empty output, - // so add a null int slot to group by slot and output - if (isGroupByEmptySelectEmpty) { - localAggGroupBy = ImmutableList.of(new NullLiteral(TinyIntType.INSTANCE)); - localAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE))); - } - - boolean maybeUsingStreamAgg = maybeUsingStreamAgg(connectContext, localAggGroupBy); - List partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg); - RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY); - PhysicalHashAggregate anyLocalAgg = new PhysicalHashAggregate<>(localAggGroupBy, - localAggOutput, Optional.of(partitionExpressions), inputToBufferParam, - maybeUsingStreamAgg, Optional.empty(), logicalAgg.getLogicalProperties(), - requireAny, logicalAgg.child()); - - AggregateParam bufferToBufferParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_BUFFER, couldBanned); - Map nonDistinctAggFunctionToAliasPhase2 = - nonDistinctAggFunctionToAliasPhase1.entrySet() - .stream() - .collect(ImmutableMap.toImmutableMap(Entry::getKey, kv -> { - AggregateFunction originFunction = kv.getKey(); - Alias localOutput = kv.getValue(); - AggregateExpression globalAggExpr = new AggregateExpression( - originFunction, bufferToBufferParam, localOutput.toSlot()); - return new Alias(globalAggExpr); - })); - - List globalAggOutput = ImmutableList.builder() - .addAll(localAggGroupBySet) - .addAll(nonDistinctAggFunctionToAliasPhase2.values()) - .build(); - - // be not recommend generate an aggregate node with empty group by and empty output, - // so add a null int slot to group by slot and output - if (isGroupByEmptySelectEmpty) { - globalAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE))); - } - - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - PhysicalHashAggregate anyLocalGatherGlobalAgg = new PhysicalHashAggregate<>( - localAggGroupBy, globalAggOutput, Optional.of(partitionExpressions), - bufferToBufferParam, false, logicalAgg.getLogicalProperties(), - requireGather, anyLocalAgg); - - AggregateParam bufferToResultParam = new AggregateParam( - AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_RESULT, couldBanned); - List distinctOutput = ExpressionUtils.rewriteDownShortCircuit( - logicalAgg.getOutputExpressions(), expr -> { - if (expr instanceof AggregateFunction) { - AggregateFunction aggregateFunction = (AggregateFunction) expr; - if (aggregateFunction.isDistinct()) { - List aggChild = aggregateFunction.children(); - Preconditions.checkArgument(new HashSet<>(aggChild).size() == 1 - || aggregateFunction.getDistinctArguments().size() == 1, - "cannot process more than one child in aggregate distinct function: " - + aggregateFunction); - AggregateFunction nonDistinct = aggregateFunction - .withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); - return new AggregateExpression(nonDistinct, bufferToResultParam, aggregateFunction); - } else { - Alias alias = nonDistinctAggFunctionToAliasPhase2.get(expr); - return new AggregateExpression(aggregateFunction, - new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.BUFFER_TO_RESULT), - alias.toSlot()); - } - } - return expr; - }); - - PhysicalHashAggregate anyLocalGatherGlobalGatherDistinctAgg = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), distinctOutput, Optional.empty(), - bufferToResultParam, false, logicalAgg.getLogicalProperties(), - requireGather, anyLocalGatherGlobalAgg); - - RequireProperties requireDistinctHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getDistinctArguments(), ShuffleType.REQUIRE)); - PhysicalHashAggregate anyLocalHashGlobalGatherDistinctAgg - = anyLocalGatherGlobalGatherDistinctAgg - .withChildren(ImmutableList.of(anyLocalGatherGlobalAgg - .withRequire(requireDistinctHash) - .withPartitionExpressions(ImmutableList.copyOf(logicalAgg.getDistinctArguments())) - )); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - return ImmutableList.>builder() - // TODO: this plan pattern is not good usually, we remove it temporary. - // .add(anyLocalGatherGlobalGatherDistinctAgg) - .add(anyLocalHashGlobalGatherDistinctAgg) - .build(); - } else { - RequireProperties requireGroupByHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate anyLocalHashGlobalHashDistinctAgg - = anyLocalGatherGlobalGatherDistinctAgg - .withRequirePropertiesAndChild(requireGroupByHash, anyLocalGatherGlobalAgg - .withRequire(requireGroupByHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()) - ) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // TODO: this plan pattern is not good usually, we remove it temporary. - //.add(anyLocalGatherGlobalGatherDistinctAgg) - //.add(anyLocalHashGlobalGatherDistinctAgg) - .add(anyLocalHashGlobalHashDistinctAgg) - .build(); - } - } - - /** - * sql: select count(distinct id) from (...) group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[count(distinct id)]) - * | - * any plan - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[multi_distinct_count(id)]) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * any plan - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[multi_distinct_count(id)]) - * | - * any plan(**already distribute by name**) - * - */ - private List> onePhaseAggregateWithMultiDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext) { - AggregateParam inputToResultParam = AggregateParam.LOCAL_RESULT; - List newOutput = ExpressionUtils.rewriteDownShortCircuit( - logicalAgg.getOutputExpressions(), outputChild -> { - if (outputChild instanceof AggregateFunction) { - AggregateFunction function = tryConvertToMultiDistinct((AggregateFunction) outputChild); - return new AggregateExpression(function, inputToResultParam); - } - return outputChild; - }); - - RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); - PhysicalHashAggregate gatherLocalAgg = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), newOutput, inputToResultParam, - maybeUsingStreamAgg(connectContext, logicalAgg), - logicalAgg.getLogicalProperties(), requireGather, logicalAgg.child()); - if (logicalAgg.getGroupByExpressions().isEmpty()) { - // TODO: usually bad, disable it until we could do better cost computation. - // return ImmutableList.of(gatherLocalAgg); - return ImmutableList.of(); - } else { - RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate hashLocalAgg = gatherLocalAgg - .withRequire(requireHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // TODO: usually bad, disable it until we could do better cost computation. - // .add(gatherLocalAgg) - .add(hashLocalAgg) - .build(); - } - } - - /** - * sql: select count(distinct id) from tbl group by name - *

- * before: - *

- * LogicalAggregate(groupBy=[name], output=[name, count(distinct id)]) - * | - * LogicalOlapScan(table=tbl) - *

- * after: - *

- * single node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, multi_count_distinct(value)], mode=BUFFER_TO_RESULT) - * | - * PhysicalDistribute(distributionSpec=GATHER) - * | - * PhysicalHashAggregate(groupBy=[name], output=[name, multi_count_distinct(value)], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - *

- * distribute node aggregate: - *

- * PhysicalHashAggregate(groupBy=[name], output=[name, multi_count_distinct(value)], mode=BUFFER_TO_RESULT) - * | - * PhysicalDistribute(distributionSpec=HASH(name)) - * | - * PhysicalHashAggregate(groupBy=[name], output=[name, multi_count_distinct(value)], mode=INPUT_TO_BUFFER) - * | - * LogicalOlapScan(table=tbl) - * - */ - private List> twoPhaseAggregateWithMultiDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext) { - Set aggregateFunctions = logicalAgg.getAggregateFunctions(); - AggregateParam inputToBufferParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER); - Map aggFunctionToAliasPhase1 = aggregateFunctions.stream() - .collect(ImmutableMap.toImmutableMap(function -> function, function -> { - AggregateFunction multiDistinct = tryConvertToMultiDistinct(function); - AggregateExpression localAggExpr = new AggregateExpression(multiDistinct, inputToBufferParam); - return new Alias(localAggExpr); - })); - - List localAggOutput = ImmutableList.builder() - // already normalize group by expression to List - .addAll((List) (List) logicalAgg.getGroupByExpressions()) - .addAll(aggFunctionToAliasPhase1.values()) - .build(); - - RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY); - PhysicalHashAggregate anyLocalAgg = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), localAggOutput, - inputToBufferParam, maybeUsingStreamAgg(connectContext, logicalAgg), - logicalAgg.getLogicalProperties(), requireAny, logicalAgg.child()); - - AggregateParam bufferToResultParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); - List globalOutput = ExpressionUtils.rewriteDownShortCircuit( - logicalAgg.getOutputExpressions(), outputChild -> { - if (outputChild instanceof AggregateFunction) { - Alias alias = aggFunctionToAliasPhase1.get(outputChild); - AggregateExpression localAggExpr = (AggregateExpression) alias.child(); - return new AggregateExpression(localAggExpr.getFunction(), - bufferToResultParam, alias.toSlot()); - } else { - return outputChild; - } - }); - - PhysicalHashAggregate anyLocalGatherGlobalAgg = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), globalOutput, Optional.empty(), - bufferToResultParam, false, logicalAgg.getLogicalProperties(), - RequireProperties.of(PhysicalProperties.GATHER), anyLocalAgg); - - if (logicalAgg.getGroupByExpressions().isEmpty()) { - // Collection distinctArguments = logicalAgg.getDistinctArguments(); - // RequireProperties requireDistinctHash = RequireProperties.of(PhysicalProperties.createHash( - // distinctArguments, ShuffleType.REQUIRE)); - // PhysicalHashAggregate hashLocalGatherGlobalAgg = anyLocalGatherGlobalAgg - // .withChildren(ImmutableList.of(anyLocalAgg - // .withRequire(requireDistinctHash) - // .withPartitionExpressions(ImmutableList.copyOf(logicalAgg.getDistinctArguments())) - // )); - return ImmutableList.>builder() - .add(anyLocalGatherGlobalAgg) - .build(); - } else { - RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); - PhysicalHashAggregate anyLocalHashGlobalAgg = anyLocalGatherGlobalAgg - .withRequire(requireHash) - .withPartitionExpressions(logicalAgg.getGroupByExpressions()); - return ImmutableList.>builder() - // TODO: usually bad, disable it until we could do better cost computation. - // .add(anyLocalGatherGlobalAgg) - .add(anyLocalHashGlobalAgg) - .build(); - } - } - - private boolean maybeUsingStreamAgg( - ConnectContext connectContext, LogicalAggregate logicalAggregate) { - return !connectContext.getSessionVariable().disableStreamPreaggregations - && !logicalAggregate.getGroupByExpressions().isEmpty(); - } - - private boolean maybeUsingStreamAgg( - ConnectContext connectContext, List groupByExpressions) { - return !connectContext.getSessionVariable().disableStreamPreaggregations - && !groupByExpressions.isEmpty(); - } - - private List getHashAggregatePartitionExpressions( - LogicalAggregate logicalAggregate) { - return logicalAggregate.getGroupByExpressions().isEmpty() - ? ImmutableList.copyOf(logicalAggregate.getDistinctArguments()) - : logicalAggregate.getGroupByExpressions(); - } - - private AggregateFunction tryConvertToMultiDistinct(AggregateFunction function) { - if (function instanceof SupportMultiDistinct && function.isDistinct()) { - return ((SupportMultiDistinct) function).convertToMultiDistinct(); - } - return function; - } - - /** - * countDistinctMultiExprToCountIf. - *

- * NOTE: this function will break the normalized output, e.g. from `count(distinct slot1, slot2)` to - * `count(if(slot1 is null, null, slot2))`. So if you invoke this method, and separate the - * phase of aggregate, please normalize to slot and create a bottom project like NormalizeAggregate. - */ - private Pair, List> countDistinctMultiExprToCountIf( - LogicalAggregate aggregate, CascadesContext cascadesContext) { - ImmutableList.Builder countIfList = ImmutableList.builder(); - List newOutput = ExpressionUtils.rewriteDownShortCircuit( - aggregate.getOutputExpressions(), outputChild -> { - if (outputChild instanceof Count) { - Count count = (Count) outputChild; - if (count.isDistinct() && count.arity() > 1) { - Set arguments = ImmutableSet.copyOf(count.getArguments()); - Expression countExpr = count.getArgument(arguments.size() - 1); - for (int i = arguments.size() - 2; i >= 0; --i) { - Expression argument = count.getArgument(i); - If ifNull = new If(new IsNull(argument), NullLiteral.INSTANCE, countExpr); - countExpr = assignNullType(ifNull, cascadesContext); - } - Count countIf = new Count(countExpr); - countIfList.add(countIf); - return countIf; - } - } - return outputChild; - }); - return Pair.of(aggregate.withAggOutput(newOutput), countIfList.build()); - } - - private boolean containsCountDistinctMultiExpr(LogicalAggregate aggregate) { - return ExpressionUtils.deapAnyMatch(aggregate.getOutputExpressions(), expr -> - expr instanceof Count && ((Count) expr).isDistinct() && expr.arity() > 1); - } - - // don't invoke the ExpressionNormalization, because the expression maybe simplified and get rid of some slots - private If assignNullType(If ifExpr, CascadesContext cascadesContext) { - If ifWithCoercion = (If) TypeCoercionUtils.processBoundFunction(ifExpr); - Expression trueValue = ifWithCoercion.getArgument(1); - if (trueValue instanceof Cast && trueValue.child(0) instanceof NullLiteral) { - List newArgs = Lists.newArrayList(ifWithCoercion.getArguments()); - // backend don't support null type, so we should set the type - newArgs.set(1, new NullLiteral(((Cast) trueValue).getDataType())); - return ifWithCoercion.withChildren(newArgs); - } - return ifWithCoercion; - } - private boolean enablePushDownNoGroupAgg() { ConnectContext connectContext = ConnectContext.get(); return connectContext == null || connectContext.getSessionVariable().enablePushDownNoGroupAgg(); } - - private List> fourPhaseAggregateWithDistinct( - LogicalAggregate logicalAgg, ConnectContext connectContext, - Function, RequireProperties> secondPhaseRequireSupplier, - Function, RequireProperties> fourPhaseRequireSupplier) { - boolean couldBanned = couldConvertToMulti(logicalAgg); - - Set aggregateFunctions = logicalAgg.getAggregateFunctions(); - - Set distinctArguments = aggregateFunctions.stream() - .filter(AggregateFunction::isDistinct) - .flatMap(aggregateExpression -> aggregateExpression.getArguments().stream()) - .filter(NamedExpression.class::isInstance) - .map(NamedExpression.class::cast) - .collect(ImmutableSet.toImmutableSet()); - - Set localAggGroupBySet = ImmutableSet.builder() - .addAll((List) (List) logicalAgg.getGroupByExpressions()) - .addAll(distinctArguments) - .build(); - - AggregateParam inputToBufferParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER, couldBanned); - - Map nonDistinctAggFunctionToAliasPhase1 = aggregateFunctions.stream() - .filter(aggregateFunction -> !aggregateFunction.isDistinct()) - .collect(ImmutableMap.toImmutableMap(expr -> expr, expr -> { - AggregateExpression localAggExpr = new AggregateExpression(expr, inputToBufferParam); - return new Alias(localAggExpr); - }, (oldValue, newValue) -> newValue)); - - List localAggOutput = ImmutableList.builder() - .addAll(localAggGroupBySet) - .addAll(nonDistinctAggFunctionToAliasPhase1.values()) - .build(); - - List localAggGroupBy = ImmutableList.copyOf(localAggGroupBySet); - boolean maybeUsingStreamAgg = maybeUsingStreamAgg(connectContext, localAggGroupBy); - List partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg); - RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY); - - boolean isGroupByEmptySelectEmpty = localAggGroupBy.isEmpty() && localAggOutput.isEmpty(); - - // be not recommend generate an aggregate node with empty group by and empty output, - // so add a null int slot to group by slot and output - if (isGroupByEmptySelectEmpty) { - localAggGroupBy = ImmutableList.of(new NullLiteral(TinyIntType.INSTANCE)); - localAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE))); - } - - PhysicalHashAggregate anyLocalAgg = new PhysicalHashAggregate<>(localAggGroupBy, - localAggOutput, Optional.of(partitionExpressions), inputToBufferParam, - maybeUsingStreamAgg, Optional.empty(), logicalAgg.getLogicalProperties(), - requireAny, logicalAgg.child()); - - AggregateParam bufferToBufferParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_BUFFER, couldBanned); - Map nonDistinctAggFunctionToAliasPhase2 = - nonDistinctAggFunctionToAliasPhase1.entrySet() - .stream() - .collect(ImmutableMap.toImmutableMap(Entry::getKey, kv -> { - AggregateFunction originFunction = kv.getKey(); - Alias localOutput = kv.getValue(); - AggregateExpression globalAggExpr = new AggregateExpression( - originFunction, bufferToBufferParam, localOutput.toSlot()); - return new Alias(globalAggExpr); - })); - - List globalAggOutput = ImmutableList.builder() - .addAll(localAggGroupBySet) - .addAll(nonDistinctAggFunctionToAliasPhase2.values()) - .build(); - - // be not recommend generate an aggregate node with empty group by and empty output, - // so add a null int slot to group by slot and output - if (isGroupByEmptySelectEmpty) { - globalAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE))); - } - - RequireProperties secondPhaseRequire = secondPhaseRequireSupplier.apply(localAggGroupBy); - - //phase 2 - PhysicalHashAggregate anyLocalHashGlobalAgg = new PhysicalHashAggregate<>( - localAggGroupBy, globalAggOutput, Optional.of(ImmutableList.copyOf(logicalAgg.getDistinctArguments())), - bufferToBufferParam, false, logicalAgg.getLogicalProperties(), - secondPhaseRequire, anyLocalAgg); - - boolean shouldDistinctAfterPhase2 = distinctArguments.size() > 1; - - // phase 3 - AggregateParam distinctLocalParam = new AggregateParam( - AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_BUFFER, couldBanned); - Map nonDistinctAggFunctionToAliasPhase3 = new HashMap<>(); - List localDistinctOutput = Lists.newArrayList(); - for (int i = 0; i < logicalAgg.getOutputExpressions().size(); i++) { - NamedExpression outputExpr = logicalAgg.getOutputExpressions().get(i); - List needUpdateSlot = Lists.newArrayList(); - NamedExpression outputExprPhase3 = (NamedExpression) outputExpr - .rewriteDownShortCircuit(expr -> { - if (expr instanceof AggregateFunction) { - AggregateFunction aggregateFunction = (AggregateFunction) expr; - if (aggregateFunction.isDistinct()) { - List aggChild = aggregateFunction.children(); - Preconditions.checkArgument(new HashSet<>(aggChild).size() == 1 - || aggregateFunction.getDistinctArguments().size() == 1, - "cannot process more than one child in aggregate distinct function: " - + aggregateFunction); - - AggregateFunction newDistinct; - if (shouldDistinctAfterPhase2) { - // we use aggregate function to process distinct, - // so need to change to multi distinct function - newDistinct = tryConvertToMultiDistinct( - aggregateFunction.withDistinctAndChildren( - true, ImmutableList.copyOf(aggChild)) - ); - } else { - // we use group by to process distinct, - // so no distinct param in the aggregate function - newDistinct = aggregateFunction.withDistinctAndChildren( - false, ImmutableList.copyOf(aggChild)); - } - - AggregateExpression newDistinctAggExpr = new AggregateExpression( - newDistinct, distinctLocalParam, newDistinct); - return newDistinctAggExpr; - } else { - needUpdateSlot.add(aggregateFunction); - Alias alias = nonDistinctAggFunctionToAliasPhase2.get(expr); - return new AggregateExpression(aggregateFunction, - new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.BUFFER_TO_BUFFER), - alias.toSlot()); - } - } - return expr; - }); - for (AggregateFunction originFunction : needUpdateSlot) { - nonDistinctAggFunctionToAliasPhase3.put(originFunction, (Alias) outputExprPhase3); - } - localDistinctOutput.add(outputExprPhase3); - - } - PhysicalHashAggregate distinctLocal = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), localDistinctOutput, Optional.empty(), - distinctLocalParam, false, logicalAgg.getLogicalProperties(), - secondPhaseRequire, anyLocalHashGlobalAgg); - - //phase 4 - AggregateParam distinctGlobalParam = new AggregateParam( - AggPhase.DISTINCT_GLOBAL, AggMode.BUFFER_TO_RESULT, couldBanned); - List globalDistinctOutput = Lists.newArrayList(); - for (int i = 0; i < logicalAgg.getOutputExpressions().size(); i++) { - NamedExpression outputExpr = logicalAgg.getOutputExpressions().get(i); - NamedExpression outputExprPhase4 = (NamedExpression) outputExpr.rewriteDownShortCircuit(expr -> { - if (expr instanceof AggregateFunction) { - AggregateFunction aggregateFunction = (AggregateFunction) expr; - if (aggregateFunction.isDistinct()) { - List aggChild = aggregateFunction.children(); - Preconditions.checkArgument(new HashSet<>(aggChild).size() == 1 - || aggregateFunction.getDistinctArguments().size() == 1, - "cannot process more than one child in aggregate distinct function: " - + aggregateFunction); - AggregateFunction newDistinct; - if (shouldDistinctAfterPhase2) { - newDistinct = tryConvertToMultiDistinct( - aggregateFunction.withDistinctAndChildren( - true, ImmutableList.copyOf(aggChild)) - ); - } else { - newDistinct = aggregateFunction - .withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); - } - int idx = logicalAgg.getOutputExpressions().indexOf(outputExpr); - Alias localDistinctAlias = (Alias) (localDistinctOutput.get(idx)); - return new AggregateExpression(newDistinct, - distinctGlobalParam, localDistinctAlias.toSlot()); - } else { - Alias alias = nonDistinctAggFunctionToAliasPhase3.get(expr); - return new AggregateExpression(aggregateFunction, - new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.BUFFER_TO_RESULT), - alias.toSlot()); - } - } - return expr; - }); - globalDistinctOutput.add(outputExprPhase4); - } - - RequireProperties fourPhaseRequire = fourPhaseRequireSupplier.apply(logicalAgg); - PhysicalHashAggregate distinctGlobal = new PhysicalHashAggregate<>( - logicalAgg.getGroupByExpressions(), globalDistinctOutput, Optional.empty(), - distinctGlobalParam, false, logicalAgg.getLogicalProperties(), - fourPhaseRequire, distinctLocal); - - return ImmutableList.>builder() - .add(distinctGlobal) - .build(); - } - - private boolean couldConvertToMulti(LogicalAggregate aggregate) { - Set aggregateFunctions = aggregate.getAggregateFunctions(); - for (AggregateFunction func : aggregateFunctions) { - if (!func.isDistinct()) { - continue; - } - if (!(func instanceof Count || func instanceof Sum || func instanceof GroupConcat - || func instanceof Sum0)) { - return false; - } - if (func.arity() <= 1) { - continue; - } - for (int i = 1; i < func.arity(); i++) { - // think about group_concat(distinct col_1, ',') - if (!(func.child(i) instanceof OrderExpression) && !func.child(i).getInputSlots().isEmpty()) { - return false; - } - } - } - return true; - } - - /** - * LogicalAggregate(groupByExpr=[a], outputExpr=[a,count(distinct b)]) - * -> - * +--PhysicalHashAggregate(groupByExpr=[a], outputExpr=[a, sum0(partial_sum0(m))] - * +--PhysicalDistribute(shuffleColumn=[a]) - * +--PhysicalHashAggregate(groupByExpr=[a], outputExpr=[a, partial_sum0(m)] - * +--PhysicalHashAggregate(groupByExpr=[a, saltExpr], outputExpr=[a, multi_distinct_count(b) as m]) - * +--PhysicalDistribute(shuffleColumn=[a, saltExpr]) - * +--PhysicalProject(projects=[a, b, xxhash_32(b)%512 as saltExpr]) - * +--PhysicalHashAggregate(groupByExpr=[a, b], outputExpr=[a, b]) - * */ - private PhysicalHashAggregate aggSkewRewrite(LogicalAggregate logicalAgg, - CascadesContext cascadesContext) { - // 1.local agg - ImmutableList.Builder localAggGroupByBuilder = ImmutableList.builderWithExpectedSize( - logicalAgg.getGroupByExpressions().size() + 1); - localAggGroupByBuilder.addAll(logicalAgg.getGroupByExpressions()); - AggregateFunction aggFunc = logicalAgg.getAggregateFunctions().iterator().next(); - localAggGroupByBuilder.add(aggFunc.child(0)); - List localAggGroupBy = localAggGroupByBuilder.build(); - List localAggOutput = Utils.fastToImmutableList((List) localAggGroupBy); - RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY); - boolean maybeUsingStreamAgg = maybeUsingStreamAgg(cascadesContext.getConnectContext(), - localAggGroupBy); - boolean couldBanned = false; - AggregateParam localParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER, couldBanned); - PhysicalHashAggregate localAgg = new PhysicalHashAggregate<>(localAggGroupBy, localAggOutput, - Optional.empty(), localParam, maybeUsingStreamAgg, Optional.empty(), null, - requireAny, logicalAgg.child()); - // add shuffle expr in project - ImmutableList.Builder projections = ImmutableList.builderWithExpectedSize( - localAgg.getOutputs().size() + 1); - projections.addAll(localAgg.getOutputs()); - Alias modAlias = getShuffleExpr(aggFunc, cascadesContext); - projections.add(modAlias); - PhysicalProject physicalProject = new PhysicalProject<>(projections.build(), null, localAgg); - - // 2.second phase agg: multi_distinct_count(b) group by a,h - ImmutableList.Builder secondPhaseAggGroupByBuilder = ImmutableList.builderWithExpectedSize( - logicalAgg.getGroupByExpressions().size() + 1); - secondPhaseAggGroupByBuilder.addAll(logicalAgg.getGroupByExpressions()); - secondPhaseAggGroupByBuilder.add(modAlias.toSlot()); - List secondPhaseAggGroupBy = secondPhaseAggGroupByBuilder.build(); - ImmutableList.Builder secondPhaseAggOutput = ImmutableList.builderWithExpectedSize( - secondPhaseAggGroupBy.size() + 1); - secondPhaseAggOutput.addAll((List) secondPhaseAggGroupBy); - Alias aliasTarget = new Alias(new TinyIntLiteral((byte) 0)); - for (NamedExpression ne : logicalAgg.getOutputExpressions()) { - if (ne instanceof Alias) { - if (((Alias) ne).child().equals(aggFunc)) { - aliasTarget = (Alias) ne; - } - } - } - AggregateParam secondParam = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT, couldBanned); - AggregateFunction multiDistinct = ((SupportMultiDistinct) aggFunc).convertToMultiDistinct(); - Alias multiDistinctAlias = new Alias(new AggregateExpression(multiDistinct, secondParam)); - secondPhaseAggOutput.add(multiDistinctAlias); - List shuffleIds = new ArrayList<>(); - for (Expression expr : secondPhaseAggGroupBy) { - if (expr instanceof Slot) { - shuffleIds.add(((Slot) expr).getExprId()); - } - } - RequireProperties secondRequireProperties = RequireProperties.of( - PhysicalProperties.createHash(shuffleIds, ShuffleType.REQUIRE)); - PhysicalHashAggregate secondPhaseAgg = new PhysicalHashAggregate<>( - secondPhaseAggGroupBy, secondPhaseAggOutput.build(), - Optional.empty(), secondParam, false, Optional.empty(), null, - secondRequireProperties, physicalProject); - - // 3. third phase agg - List thirdPhaseAggGroupBy = Utils.fastToImmutableList(logicalAgg.getGroupByExpressions()); - ImmutableList.Builder thirdPhaseAggOutput = ImmutableList.builderWithExpectedSize( - thirdPhaseAggGroupBy.size() + 1); - thirdPhaseAggOutput.addAll((List) thirdPhaseAggGroupBy); - AggregateParam thirdParam = new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_BUFFER, couldBanned); - AggregateFunction function = getAggregateFunction(aggFunc); - AggregateFunction thirdAggFunc = function.withDistinctAndChildren(false, - ImmutableList.of(multiDistinctAlias.toSlot())); - Alias thirdCountAlias = new Alias(new AggregateExpression(thirdAggFunc, thirdParam)); - thirdPhaseAggOutput.add(thirdCountAlias); - PhysicalHashAggregate thirdPhaseAgg = new PhysicalHashAggregate<>( - thirdPhaseAggGroupBy, thirdPhaseAggOutput.build(), - Optional.empty(), thirdParam, false, Optional.empty(), null, - secondRequireProperties, secondPhaseAgg); - - // 4. fourth phase agg - ImmutableList.Builder fourthPhaseAggOutput = ImmutableList.builderWithExpectedSize( - thirdPhaseAggGroupBy.size() + 1); - fourthPhaseAggOutput.addAll((List) thirdPhaseAggGroupBy); - AggregateParam fourthParam = new AggregateParam(AggPhase.DISTINCT_GLOBAL, AggMode.BUFFER_TO_RESULT, - couldBanned); - Alias sumAliasFour = new Alias(aliasTarget.getExprId(), - new AggregateExpression(thirdAggFunc, fourthParam, thirdCountAlias.toSlot()), - aliasTarget.getName()); - fourthPhaseAggOutput.add(sumAliasFour); - List shuffleIdsFour = new ArrayList<>(); - for (Expression expr : logicalAgg.getExpressions()) { - if (expr instanceof Slot) { - shuffleIdsFour.add(((Slot) expr).getExprId()); - } - } - RequireProperties fourthRequireProperties = RequireProperties.of( - PhysicalProperties.createHash(shuffleIdsFour, ShuffleType.REQUIRE)); - return new PhysicalHashAggregate<>(thirdPhaseAggGroupBy, - fourthPhaseAggOutput.build(), Optional.empty(), fourthParam, - false, Optional.empty(), logicalAgg.getLogicalProperties(), - fourthRequireProperties, thirdPhaseAgg); - } - - private AggregateFunction getAggregateFunction(AggregateFunction aggFunc) { - if (aggFunc instanceof Count) { - return new Sum0(aggFunc.child(0)); - } else { - return aggFunc; - } - } - - private Alias getShuffleExpr(AggregateFunction aggFunc, CascadesContext cascadesContext) { - int bucketNum = cascadesContext.getConnectContext().getSessionVariable().skewRewriteAggBucketNum; - // divide bucketNum by 2 is because XxHash32 return negative and positive number - int bucket = bucketNum / 2; - DataType type = bucket <= 128 ? TinyIntType.INSTANCE : SmallIntType.INSTANCE; - Mod mod = new Mod(new XxHash32(TypeCoercionUtils.castIfNotSameType( - aggFunc.child(0), StringType.INSTANCE)), new IntegerLiteral((short) bucket)); - Cast cast = new Cast(mod, type); - return new Alias(cast, SALT_EXPR + cascadesContext.getStatementContext().generateColumnName()); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggBaseRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggBaseRule.java new file mode 100644 index 00000000000000..a43fb753c2af64 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggBaseRule.java @@ -0,0 +1,202 @@ +// 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.implementation; + +import org.apache.doris.nereids.trees.expressions.AggregateExpression; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.plans.AggMode; +import org.apache.doris.nereids.trees.plans.AggPhase; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; +import org.apache.doris.nereids.types.TinyIntType; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.collect.ImmutableList; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/**SplitAggRule*/ +public abstract class SplitAggBaseRule { + /** + * This functions is used to split bottom deduplicate Aggregate(phase1): + * e.g.select count(distinct a) group by b + * agg(group by b, count(a); distinct global) ---phase2 + * +--agg(group by a,b; global) ---phase1 + * +--hashShuffle(b) + * */ + protected PhysicalHashAggregate splitDeduplicateOnePhase(LogicalAggregate aggregate, + Set localAggGroupBySet, AggregateParam inputToBufferParam, AggregateParam paramForAggFunc, + Map localAggFunctionToAlias, Plan child, List partitionExpressions) { + aggregate.getAggregateFunctions().stream() + .filter(aggFunc -> !aggFunc.isDistinct()) + .collect(Collectors.toMap( + expr -> expr, + expr -> { + AggregateExpression localAggExpr = new AggregateExpression(expr, paramForAggFunc); + return new Alias(localAggExpr); + }, + (existing, replacement) -> existing, + () -> localAggFunctionToAlias + )); + List localAggOutput = ImmutableList.builder() + .addAll(localAggGroupBySet) + .addAll(localAggFunctionToAlias.values()) + .build(); + List localAggGroupBy = Utils.fastToImmutableList(localAggGroupBySet); + boolean isGroupByEmptySelectEmpty = localAggGroupBy.isEmpty() && localAggOutput.isEmpty(); + // be not recommend generate an aggregate node with empty group by and empty output, + // so add a null int slot to group by slot and output + if (isGroupByEmptySelectEmpty) { + localAggGroupBy = ImmutableList.of(new NullLiteral(TinyIntType.INSTANCE)); + localAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE))); + } + return new PhysicalHashAggregate<>(localAggGroupBy, localAggOutput, Optional.ofNullable(partitionExpressions), + inputToBufferParam, AggregateUtils.maybeUsingStreamAgg(localAggGroupBy, inputToBufferParam), + null, child); + } + + /** + * This functions is used to split bottom deduplicate Aggregate(phase1 and phase2): + * e.g.select count(distinct a) group by b + * agg(group by b, count(a); distinct global) ---phase3 + * +--agg(group by a,b; global) ---phase2 + * +--hashShuffle(b) + * +--agg(group by a,b; local) ---phase1 + * */ + protected PhysicalHashAggregate splitDeduplicateTwoPhase(LogicalAggregate aggregate, + Map middleAggFunctionToAlias, List partitionExpressions, + Set localAggGroupBySet) { + // first phase + AggregateParam inputToBufferParam = AggregateParam.LOCAL_BUFFER; + Map localAggFunctionToAlias = new LinkedHashMap<>(); + PhysicalHashAggregate localAgg = splitDeduplicateOnePhase(aggregate, localAggGroupBySet, + inputToBufferParam, inputToBufferParam, localAggFunctionToAlias, aggregate.child(), ImmutableList.of()); + + // second phase + AggregateParam bufferToResultParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); + AggregateParam bufferToBufferParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_BUFFER); + localAggFunctionToAlias.entrySet() + .stream() + .collect(Collectors.toMap(Entry::getKey, + kv -> { + AggregateExpression middleAggExpr = new AggregateExpression(kv.getKey(), + bufferToBufferParam, kv.getValue().toSlot()); + return new Alias(middleAggExpr); + }, + (existing, replacement) -> existing, + () -> middleAggFunctionToAlias)); + List middleAggOutput = ImmutableList.builder() + .addAll(localAggGroupBySet) + .addAll(middleAggFunctionToAlias.values()) + .build(); + if (middleAggOutput.isEmpty()) { + middleAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE))); + } + return new PhysicalHashAggregate<>(localAgg.getGroupByExpressions(), middleAggOutput, + Optional.ofNullable(partitionExpressions), bufferToResultParam, + AggregateUtils.maybeUsingStreamAgg(localAgg.getGroupByExpressions(), bufferToResultParam), + null, localAgg); + } + + /** + * This functions is used to split distinct phase Aggregate(phase2 and phase3): + * e.g. select count(distinct a) group by b + * agg(group by b, count(a); distinct global) --phase3 + * +--hashShuffle(b) + * +--agg(group by b, count(a); distinct local) --phase2 + * +--agg(group by a,b; global) --phase1 + * +--hashShuffle(a) + * */ + protected PhysicalHashAggregate splitDistinctTwoPhase(LogicalAggregate aggregate, + Map middleAggFunctionToAlias, Plan child) { + AggregateParam thirdParam = new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_BUFFER); + Map aggFuncToAliasThird = new LinkedHashMap<>(); + middleAggFunctionToAlias.entrySet().stream().collect( + Collectors.toMap(Entry::getKey, + entry -> new Alias(new AggregateExpression(entry.getKey(), + new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.BUFFER_TO_BUFFER), + entry.getValue().toSlot())), + (k1, k2) -> k1, + () -> aggFuncToAliasThird + ) + ); + for (AggregateFunction func : aggregate.getAggregateFunctions()) { + if (!func.isDistinct()) { + continue; + } + if (func instanceof Count && func.arity() > 1) { + Expression countIf = AggregateUtils.countDistinctMultiExprToCountIf((Count) func); + aggFuncToAliasThird.put(func, new Alias(new AggregateExpression((Count) countIf, thirdParam))); + } else { + aggFuncToAliasThird.put(func, new Alias(new AggregateExpression( + func.withDistinctAndChildren(false, func.children()), thirdParam))); + } + } + List thirdAggOutput = ImmutableList.builder() + .addAll((List) aggregate.getGroupByExpressions()) + .addAll(aggFuncToAliasThird.values()) + .build(); + Plan thirdAgg = new PhysicalHashAggregate<>(aggregate.getGroupByExpressions(), thirdAggOutput, thirdParam, + AggregateUtils.maybeUsingStreamAgg(aggregate.getGroupByExpressions(), thirdParam), null, child); + + // fourth phase + AggregateParam fourthParam = new AggregateParam(AggPhase.DISTINCT_GLOBAL, AggMode.BUFFER_TO_RESULT); + List globalOutput = ExpressionUtils.rewriteDownShortCircuit( + aggregate.getOutputExpressions(), expr -> { + if (expr instanceof AggregateFunction) { + AggregateFunction aggFunc = (AggregateFunction) expr; + if (aggFunc.isDistinct()) { + Alias alias = aggFuncToAliasThird.get(aggFunc); + if (aggFunc instanceof Count && aggFunc.arity() > 1) { + return new AggregateExpression(((AggregateExpression) alias.child()).getFunction(), + fourthParam, alias.toSlot()); + } else { + return new AggregateExpression( + aggFunc.withDistinctAndChildren(false, aggFunc.children()), + fourthParam, alias.toSlot()); + } + } else { + return new AggregateExpression(aggFunc, + new AggregateParam(AggPhase.DISTINCT_GLOBAL, AggMode.BUFFER_TO_RESULT), + aggFuncToAliasThird.get(aggFunc).toSlot()); + } + } + return expr; + } + ); + return new PhysicalHashAggregate<>(aggregate.getGroupByExpressions(), globalOutput, + Optional.ofNullable(aggregate.getGroupByExpressions()), fourthParam, + AggregateUtils.maybeUsingStreamAgg(aggregate.getGroupByExpressions(), fourthParam), + aggregate.getLogicalProperties(), thirdAgg); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggMultiPhase.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggMultiPhase.java new file mode 100644 index 00000000000000..11d1baa0f75e1d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggMultiPhase.java @@ -0,0 +1,374 @@ +// 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.implementation; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.pattern.MatchingContext; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.AggregateExpression; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Cast; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Mod; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregatePhase; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum0; +import org.apache.doris.nereids.trees.expressions.functions.agg.SupportMultiDistinct; +import org.apache.doris.nereids.trees.expressions.functions.scalar.XxHash32; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +import org.apache.doris.nereids.trees.plans.AggMode; +import org.apache.doris.nereids.trees.plans.AggPhase; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.TinyIntType; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.nereids.util.TypeCoercionUtils; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.Statistics; + +import com.google.common.collect.ImmutableList; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/**SplitAggMultiPhase + * only process agg with distinct function, split Agg into multi phase + * */ +public class SplitAggMultiPhase extends SplitAggBaseRule implements ImplementationRuleFactory { + public static final SplitAggMultiPhase INSTANCE = new SplitAggMultiPhase(); + private static final String SALT_EXPR = "saltExpr"; + + @Override + public List buildRules() { + return ImmutableList.of( + logicalAggregate() + .when(agg -> !agg.getGroupByExpressions().isEmpty()) + .when(agg -> agg.getDistinctArguments().size() == 1 || agg.distinctFuncNum() == 1) + .thenApplyMulti(this::rewrite) + .toRule(RuleType.SPLIT_AGG_MULTI_PHASE) + ); + } + + private List rewrite(MatchingContext> ctx) { + LogicalAggregate aggregate = ctx.root; + if (aggregate.canSkewRewrite()) { + return ImmutableList.of(aggSkewRewrite(aggregate, ctx.cascadesContext)); + } else { + if (twoPlusOneBetterThanTwoPlusTwo(aggregate)) { + return ImmutableList.builder() + .addAll(splitToTwoPlusOnePhase(aggregate)) + .addAll(splitToOnePlusTwoPhase(aggregate)) + .build(); + } else { + return ImmutableList.builder() + .addAll(splitToTwoPlusTwoPhase(aggregate)) + .build(); + } + } + } + + /** + * select count(distinct a) group by b (deduplicated agg hashShuffle by group by key b) + * splitToTwoPlusOnePhase: + * agg(group by b, count(a); distinct global) + * +--agg(group by a,b; global) + * +--hashShuffle(b) + * +--agg(group by a,b; local) + * agg(group by b, count(a); distinct global) + * +--agg(group by a,b; global) + * +--hashShuffle(b) + */ + private List splitToTwoPlusOnePhase(LogicalAggregate aggregate) { + ImmutableList.Builder builder = ImmutableList.builder(); + if (aggregate.supportAggregatePhase(AggregatePhase.THREE)) { + Set localAggGroupBySet = AggregateUtils.getAllKeySet(aggregate); + Map middleAggFuncToAlias = new LinkedHashMap<>(); + Plan middleAgg = splitDeduplicateTwoPhase(aggregate, middleAggFuncToAlias, + aggregate.getGroupByExpressions(), localAggGroupBySet); + builder.add(splitDistinctOnePhase(aggregate, middleAggFuncToAlias, middleAgg)); + } + if (aggregate.supportAggregatePhase(AggregatePhase.TWO)) { + Map localAggFuncToAlias = new HashMap<>(); + Plan localAgg = splitToOnePhase(aggregate, Utils.fastToImmutableList(aggregate.getGroupByExpressions()), + localAggFuncToAlias); + builder.add(splitDistinctOnePhase(aggregate, localAggFuncToAlias, localAgg)); + } + return builder.build(); + } + + private Plan splitToOnePhase(LogicalAggregate aggregate, + List partitionExpressions, Map localAggFunctionToAlias) { + Set localAggGroupBySet = AggregateUtils.getAllKeySet(aggregate); + // first phase + AggregateParam inputToResultParamFirst = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT); + AggregateParam paramForAggFunc = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_BUFFER); + return splitDeduplicateOnePhase(aggregate, localAggGroupBySet, inputToResultParamFirst, + paramForAggFunc, localAggFunctionToAlias, aggregate.child(), + partitionExpressions); + } + + /** + * select count(distinct a) group by b (deduplicated agg hashShuffle by group by key b) + * splitToTwoPlusTwoPhase: + * agg(group by b, count(a); distinct global) + * +--hashShuffle(b) + * +--agg(group by b, count(a); distinct local) + * +--agg(group by a,b; global) + * +--hashShuffle(a,b) + * +--agg(group by a,b; local) + * agg(group by b, count(a); distinct global) + * +--hashShuffle(b) + * +--agg(group by b, count(a); distinct local) + * +--agg(group by a,b; global) + * +--hashShuffle(a,b) + */ + private List splitToTwoPlusTwoPhase(LogicalAggregate aggregate) { + ImmutableList.Builder builder = ImmutableList.builder(); + Set localAggGroupBySet = AggregateUtils.getAllKeySet(aggregate); + if (aggregate.supportAggregatePhase(AggregatePhase.FOUR)) { + Map middleAggFunctionToAlias = new LinkedHashMap<>(); + Plan twoPhaseAgg = splitDeduplicateTwoPhase(aggregate, middleAggFunctionToAlias, + Utils.fastToImmutableList(localAggGroupBySet), localAggGroupBySet); + builder.add(splitDistinctTwoPhase(aggregate, middleAggFunctionToAlias, twoPhaseAgg)); + } + Map localAggFunctionToAlias = new HashMap<>(); + Plan onePhaseAgg = splitToOnePhase(aggregate, Utils.fastToImmutableList(localAggGroupBySet), + localAggFunctionToAlias); + if (aggregate.supportAggregatePhase(AggregatePhase.THREE)) { + builder.add(splitDistinctTwoPhase(aggregate, localAggFunctionToAlias, onePhaseAgg)); + } + if (aggregate.supportAggregatePhase(AggregatePhase.TWO)) { + builder.add(splitDistinctOnePhase(aggregate, localAggFunctionToAlias, onePhaseAgg)); + } + return builder.build(); + } + + /** + * select count(distinct a) group by b (deduplicated agg hashShuffle by group by key b) + * splitToOnePlusTwoPhase: (deduplicated agg hashShuffle by distinct key a) + * agg(group by b, count(a); distinct global) + * +--hashShuffle(b) + * +--agg(group by b, count(a); distinct local) + * +--agg(group by a,b; global) + * +--hashShuffle(a) + */ + private List splitToOnePlusTwoPhase(LogicalAggregate aggregate) { + if (!aggregate.supportAggregatePhase(AggregatePhase.THREE)) { + return ImmutableList.of(); + } + Set localAggGroupBySet = AggregateUtils.getAllKeySet(aggregate); + // first phase + AggregateParam paramForAgg = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT); + AggregateParam paramForAggFunc = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_BUFFER); + + Map localAggFunctionToAlias = new LinkedHashMap<>(); + Plan localAgg = splitDeduplicateOnePhase(aggregate, localAggGroupBySet, paramForAgg, paramForAggFunc, + localAggFunctionToAlias, aggregate.child(), + Utils.fastToImmutableList(aggregate.getDistinctArguments())); + return ImmutableList.of(splitDistinctTwoPhase(aggregate, localAggFunctionToAlias, localAgg)); + } + + private PhysicalHashAggregate splitDistinctOnePhase(LogicalAggregate aggregate, + Map childAggFuncMap, Plan child) { + AggregateParam inputToResultParamSecond = new AggregateParam(AggPhase.DISTINCT_GLOBAL, AggMode.INPUT_TO_RESULT); + List globalOutput = ExpressionUtils.rewriteDownShortCircuit( + aggregate.getOutputExpressions(), expr -> { + if (expr instanceof AggregateFunction) { + AggregateFunction aggFunc = (AggregateFunction) expr; + if (aggFunc.isDistinct()) { + // TODO: add test about aggFun arity + if (aggFunc instanceof Count && aggFunc.arity() > 1) { + Expression countIf = AggregateUtils.countDistinctMultiExprToCountIf((Count) aggFunc); + return new AggregateExpression((Count) countIf, inputToResultParamSecond); + } else { + return new AggregateExpression( + aggFunc.withDistinctAndChildren(false, aggFunc.children()), + inputToResultParamSecond); + } + } else { + return new AggregateExpression(aggFunc, + new AggregateParam(AggPhase.DISTINCT_GLOBAL, AggMode.BUFFER_TO_RESULT), + childAggFuncMap.get(aggFunc).toSlot()); + } + } + return expr; + }); + return new PhysicalHashAggregate<>(aggregate.getGroupByExpressions(), globalOutput, + Optional.ofNullable(aggregate.getGroupByExpressions()), inputToResultParamSecond, + AggregateUtils.maybeUsingStreamAgg(aggregate.getGroupByExpressions(), inputToResultParamSecond), + aggregate.getLogicalProperties(), child); + } + + /** + * LogicalAggregate(groupByExpr=[a], outputExpr=[a,count(distinct b)]) + * -> + * +--PhysicalHashAggregate(groupByExpr=[a], outputExpr=[a, sum0(partial_sum0(m))] + * +--PhysicalDistribute(shuffleColumn=[a]) + * +--PhysicalHashAggregate(groupByExpr=[a], outputExpr=[a, partial_sum0(m)] + * +--PhysicalHashAggregate(groupByExpr=[a, saltExpr], outputExpr=[a, multi_distinct_count(b) as m]) + * +--PhysicalDistribute(shuffleColumn=[a, saltExpr]) + * +--PhysicalProject(projects=[a, b, xxhash_32(b)%512 as saltExpr]) + * +--PhysicalHashAggregate(groupByExpr=[a, b], outputExpr=[a, b]) + * */ + private static PhysicalHashAggregate aggSkewRewrite(LogicalAggregate logicalAgg, + CascadesContext cascadesContext) { + // 1.local agg + ImmutableList.Builder localAggGroupByBuilder = ImmutableList.builderWithExpectedSize( + logicalAgg.getGroupByExpressions().size() + 1); + localAggGroupByBuilder.addAll(logicalAgg.getGroupByExpressions()); + AggregateFunction aggFunc = logicalAgg.getAggregateFunctions().iterator().next(); + localAggGroupByBuilder.add(aggFunc.child(0)); + List localAggGroupBy = localAggGroupByBuilder.build(); + List localAggOutput = Utils.fastToImmutableList((List) localAggGroupBy); + AggregateParam localParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER); + boolean maybeUsingStreamAgg = AggregateUtils.maybeUsingStreamAgg(localAggGroupBy, localParam); + PhysicalHashAggregate localAgg = new PhysicalHashAggregate<>(localAggGroupBy, localAggOutput, + Optional.empty(), localParam, maybeUsingStreamAgg, + null, logicalAgg.child()); + // add shuffle expr in project + ImmutableList.Builder projections = ImmutableList.builderWithExpectedSize( + localAgg.getOutputs().size() + 1); + projections.addAll(localAgg.getOutputs()); + Alias modAlias = getShuffleExpr(aggFunc, cascadesContext); + projections.add(modAlias); + PhysicalProject physicalProject = new PhysicalProject<>(projections.build(), null, localAgg); + + // 2.second phase agg: multi_distinct_count(b) group by a,h + ImmutableList.Builder secondPhaseAggGroupByBuilder = ImmutableList.builderWithExpectedSize( + logicalAgg.getGroupByExpressions().size() + 1); + secondPhaseAggGroupByBuilder.addAll(logicalAgg.getGroupByExpressions()); + secondPhaseAggGroupByBuilder.add(modAlias.toSlot()); + List secondPhaseAggGroupBy = secondPhaseAggGroupByBuilder.build(); + ImmutableList.Builder secondPhaseAggOutput = ImmutableList.builderWithExpectedSize( + secondPhaseAggGroupBy.size() + 1); + secondPhaseAggOutput.addAll((List) secondPhaseAggGroupBy); + Alias aliasTarget = new Alias(new TinyIntLiteral((byte) 0)); + for (NamedExpression ne : logicalAgg.getOutputExpressions()) { + if (ne instanceof Alias) { + if (((Alias) ne).child().equals(aggFunc)) { + aliasTarget = (Alias) ne; + } + } + } + AggregateParam secondParam = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT, false); + AggregateFunction multiDistinct = ((SupportMultiDistinct) aggFunc).convertToMultiDistinct(); + Alias multiDistinctAlias = new Alias(new AggregateExpression(multiDistinct, secondParam)); + secondPhaseAggOutput.add(multiDistinctAlias); + PhysicalHashAggregate secondPhaseAgg = new PhysicalHashAggregate<>( + secondPhaseAggGroupBy, secondPhaseAggOutput.build(), + Optional.of(secondPhaseAggGroupBy), secondParam, false, + null, physicalProject); + + // 3. third phase agg + List thirdPhaseAggGroupBy = Utils.fastToImmutableList(logicalAgg.getGroupByExpressions()); + ImmutableList.Builder thirdPhaseAggOutput = ImmutableList.builderWithExpectedSize( + thirdPhaseAggGroupBy.size() + 1); + thirdPhaseAggOutput.addAll((List) thirdPhaseAggGroupBy); + AggregateParam thirdParam = new AggregateParam(AggPhase.DISTINCT_LOCAL, AggMode.INPUT_TO_BUFFER); + AggregateFunction thirdAggFunc = getAggregateFunction(aggFunc, multiDistinctAlias.toSlot()); + Alias thirdCountAlias = new Alias(new AggregateExpression(thirdAggFunc, thirdParam)); + thirdPhaseAggOutput.add(thirdCountAlias); + PhysicalHashAggregate thirdPhaseAgg = new PhysicalHashAggregate<>( + thirdPhaseAggGroupBy, thirdPhaseAggOutput.build(), + Optional.empty(), thirdParam, false, + null, secondPhaseAgg); + + // 4. fourth phase agg + ImmutableList.Builder fourthPhaseAggOutput = ImmutableList.builderWithExpectedSize( + thirdPhaseAggGroupBy.size() + 1); + fourthPhaseAggOutput.addAll((List) thirdPhaseAggGroupBy); + AggregateParam fourthParam = new AggregateParam(AggPhase.DISTINCT_GLOBAL, AggMode.BUFFER_TO_RESULT); + Alias sumAliasFour = new Alias(aliasTarget.getExprId(), + new AggregateExpression(thirdAggFunc, fourthParam, thirdCountAlias.toSlot()), + aliasTarget.getName()); + fourthPhaseAggOutput.add(sumAliasFour); + return new PhysicalHashAggregate<>(thirdPhaseAggGroupBy, + fourthPhaseAggOutput.build(), Optional.of(logicalAgg.getGroupByExpressions()), fourthParam, + false, logicalAgg.getLogicalProperties(), thirdPhaseAgg); + } + + private static AggregateFunction getAggregateFunction(AggregateFunction aggFunc, Slot child) { + if (aggFunc instanceof Count) { + return new Sum0(false, child); + } else { + return aggFunc.withDistinctAndChildren(false, ImmutableList.of(child)); + } + } + + private static Alias getShuffleExpr(AggregateFunction aggFunc, CascadesContext cascadesContext) { + int bucketNum = cascadesContext.getConnectContext().getSessionVariable().skewRewriteAggBucketNum; + // divide bucketNum by 2 is because XxHash32 return negative and positive number + int bucket = bucketNum / 2; + DataType type = bucket <= 128 ? TinyIntType.INSTANCE : SmallIntType.INSTANCE; + Mod mod = new Mod(new XxHash32(TypeCoercionUtils.castIfNotSameType( + aggFunc.child(0), StringType.INSTANCE)), new IntegerLiteral((short) bucket)); + Cast cast = new Cast(mod, type); + return new Alias(cast, SALT_EXPR + cascadesContext.getStatementContext().generateColumnName()); + } + + /**twoPlusOneBetterThanTwoPlusTwo*/ + public boolean twoPlusOneBetterThanTwoPlusTwo(LogicalAggregate aggregate) { + ConnectContext ctx = ConnectContext.get(); + if (ctx == null) { + return true; + } + switch (ctx.getSessionVariable().aggPhase) { + case 3: + return true; + case 4: + return false; + default: + break; + } + // select count(distinct a) from t group by a; should use twoPlusOne + Set groupBySet = AggregateUtils.getGroupBySetNamedExpr(aggregate); + Set distinctSet = AggregateUtils.getDistinctNamedExpr(aggregate); + if (groupBySet.containsAll(distinctSet)) { + return true; + } + Statistics aggStats = aggregate.getGroupExpression().get().getOwnerGroup().getStatistics(); + Statistics aggChildStats = aggregate.getGroupExpression().get().childStatistics(0); + if (aggStats == null || aggChildStats == null) { + return true; + } + if (AggregateUtils.hasUnknownStatistics(aggregate.getGroupByExpressions(), aggChildStats)) { + return true; + } + double ndv = aggStats.getRowCount(); + // When ndv is very low, three-stage AGG cannot be used and there will be data skew + return ndv > AggregateUtils.LOW_NDV_THRESHOLD; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggMultiPhaseWithoutGbyKey.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggMultiPhaseWithoutGbyKey.java new file mode 100644 index 00000000000000..cb3a7435a0f99d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggMultiPhaseWithoutGbyKey.java @@ -0,0 +1,232 @@ +// 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.implementation; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.AggregateExpression; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregatePhase; +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.GroupConcat; +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.MultiDistinctCount; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctGroupConcat; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum0; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinction; +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.plans.AggMode; +import org.apache.doris.nereids.trees.plans.AggPhase; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; + +/**SplitAggMultiPhaseWithoutGbyKey*/ +public class SplitAggMultiPhaseWithoutGbyKey extends SplitAggBaseRule implements ImplementationRuleFactory { + public static final SplitAggMultiPhaseWithoutGbyKey INSTANCE = new SplitAggMultiPhaseWithoutGbyKey(); + public static final List> finalMultiDistinctSupportFunc = + ImmutableList.of(Count.class, Sum.class, Sum0.class); + public static final List> finalMultiDistinctSupportOtherFunc = + ImmutableList.of(Count.class, Sum.class, Min.class, Max.class, Sum0.class, AnyValue.class); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalAggregate() + .when(agg -> agg.getGroupByExpressions().isEmpty()) + .when(agg -> agg.getDistinctArguments().size() == 1 || agg.distinctFuncNum() == 1) + .thenApplyMulti(ctx -> rewrite(ctx.root)) + .toRule(RuleType.SPLIT_AGG_MULTI_PHASE_WITHOUT_GBY_KEY) + ); + } + + List rewrite(LogicalAggregate aggregate) { + if (canUseFinalMultiDistinct(aggregate)) { + return ImmutableList.builder() + .addAll(twoPhaseAggregateWithFinalMultiDistinct(aggregate)) + .addAll(splitToFourPhase(aggregate)) + .build(); + } else { + return ImmutableList.builder() + .addAll(splitToThreePhase(aggregate)) + .addAll(splitToFourPhase(aggregate)) + .build(); + } + } + + /** + * select count(distinct a) from t + * splitToFourPhase: + * agg(count(a); agg_phase: distinct_global) + * +--gather + * +--agg(count(a); agg_phase: distinct_local) + * +--agg(group by a; agg_phase: global) + * +--hashShuffle(a) + * +--agg(group by a; agg_phase: local) + */ + private List splitToFourPhase(LogicalAggregate aggregate) { + if (!aggregate.supportAggregatePhase(AggregatePhase.FOUR)) { + return ImmutableList.of(); + } + Map localAggFuncToAlias = new LinkedHashMap<>(); + Plan secondAgg = splitDeduplicateTwoPhase(aggregate, localAggFuncToAlias, + Utils.fastToImmutableList(aggregate.getDistinctArguments()), AggregateUtils.getAllKeySet(aggregate)); + return ImmutableList.of(splitDistinctTwoPhase(aggregate, localAggFuncToAlias, secondAgg)); + } + + /** + * select count(distinct a) from t + * splitToThreePhase: + * agg(count(a); agg_phase: distinct_global) + * +--gather + * +--agg(count(a); agg_phase: distinct_local) + * +--agg(group by a; agg_phase: global) + * +--hashShuffle(a) + */ + private List splitToThreePhase(LogicalAggregate aggregate) { + if (!aggregate.supportAggregatePhase(AggregatePhase.THREE)) { + return ImmutableList.of(); + } + AggregateParam inputToResult = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT); + AggregateParam paramForAggFunc = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_BUFFER); + Map localAggFuncToAlias = new LinkedHashMap<>(); + Set keySet = AggregateUtils.getAllKeySet(aggregate); + Plan localAgg = splitDeduplicateOnePhase(aggregate, keySet, inputToResult, + paramForAggFunc, localAggFuncToAlias, aggregate.child(), Utils.fastToImmutableList(keySet)); + return ImmutableList.of(splitDistinctTwoPhase(aggregate, localAggFuncToAlias, localAgg)); + } + + /** + * select count(distinct a) from t + * twoPhaseAggregateWithFinalMultiDistinct: + * agg(sum0(c1), agg_phase: final) + * +--gather + * +--agg(multi_distinct_count(a) as c1, agg_phase: final) + * +--hashShuffle(a) + * */ + private List twoPhaseAggregateWithFinalMultiDistinct( + LogicalAggregate logicalAgg) { + Set aggregateFunctions = logicalAgg.getAggregateFunctions(); + + AggregateParam inputToResultParam = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT); + + Map originFuncToAliasPhase1 = new HashMap<>(); + for (AggregateFunction function : aggregateFunctions) { + AggregateFunction aggFunc = AggregateUtils.tryConvertToMultiDistinct(function); + AggregateExpression localAggExpr = new AggregateExpression(aggFunc, inputToResultParam); + originFuncToAliasPhase1.put(function, new Alias(localAggExpr)); + } + + List localAggOutput = ImmutableList.builder() + .addAll(originFuncToAliasPhase1.values()) + .build(); + Plan anyLocalAgg = new PhysicalHashAggregate<>(logicalAgg.getGroupByExpressions(), localAggOutput, + Optional.of(Utils.fastToImmutableList(logicalAgg.getDistinctArguments())), inputToResultParam, + AggregateUtils.maybeUsingStreamAgg(logicalAgg.getGroupByExpressions(), inputToResultParam), + null, logicalAgg.child()); + + AggregateParam param = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT, false); + + List globalOutput = ExpressionUtils.rewriteDownShortCircuit( + logicalAgg.getOutputExpressions(), outputChild -> { + if (outputChild instanceof AggregateFunction) { + Alias alias = originFuncToAliasPhase1.get(outputChild); + AggregateExpression localAggExpr = (AggregateExpression) alias.child(); + AggregateFunction aggFunc = localAggExpr.getFunction(); + Slot childSlot = alias.toSlot(); + if (aggFunc instanceof MultiDistinction) { + Map, Supplier> functionMap = + ImmutableMap.of( + MultiDistinctCount.class, () -> new Sum0(childSlot), + MultiDistinctSum.class, () -> new Sum(childSlot), + MultiDistinctSum0.class, () -> new Sum0(childSlot), + // TODO: now we don't support group_concat, + // we need add support for group_concat without order by, + // and add test for group_concat + MultiDistinctGroupConcat.class, () -> new GroupConcat(childSlot)); + return new AggregateExpression(functionMap.get(aggFunc.getClass()).get(), param); + } else { + Map, Supplier> functionMap = + ImmutableMap.of( + Count.class, () -> new Sum0(childSlot), + Sum.class, () -> new Sum(childSlot), + Sum0.class, () -> new Sum0(childSlot), + Min.class, () -> new Min(childSlot), + Max.class, () -> new Max(childSlot), + AnyValue.class, () -> new AnyValue(childSlot), + // TODO: now we don't support group_concat, + // we need add support for group_concat without order by, + // and add test for group_concat + GroupConcat.class, () -> new GroupConcat(childSlot)); + return new AggregateExpression(functionMap.get(aggFunc.getClass()).get(), param, childSlot); + } + } else { + return outputChild; + } + }); + return ImmutableList.of(new PhysicalHashAggregate<>(logicalAgg.getGroupByExpressions(), globalOutput, param, + AggregateUtils.maybeUsingStreamAgg(logicalAgg.getGroupByExpressions(), param), + logicalAgg.getLogicalProperties(), anyLocalAgg)); + } + + private boolean canUseFinalMultiDistinct(Aggregate agg) { + for (AggregateFunction aggFunc : agg.getAggregateFunctions()) { + if (aggFunc.isDistinct()) { + if (!finalMultiDistinctSupportFunc.contains(aggFunc.getClass())) { + return false; + } + if (aggFunc instanceof Count && aggFunc.arity() > 1) { + return false; + } + } else { + if (!finalMultiDistinctSupportOtherFunc.contains(aggFunc.getClass())) { + return false; + } + } + } + ConnectContext ctx = ConnectContext.get(); + if (ctx != null && (ctx.getSessionVariable().aggPhase == 3 || ctx.getSessionVariable().aggPhase == 4)) { + return false; + } + return true; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggWithoutDistinct.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggWithoutDistinct.java new file mode 100644 index 00000000000000..f64daaf7b0a214 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/SplitAggWithoutDistinct.java @@ -0,0 +1,171 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.implementation; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.AggregateExpression; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregatePhase; +import org.apache.doris.nereids.trees.plans.AggMode; +import org.apache.doris.nereids.trees.plans.AggPhase; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.Statistics; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.List; +import java.util.Map; + +/**SplitAgg + * only process agg without distinct function, split Agg into 2 phase: local agg and global agg + * */ +public class SplitAggWithoutDistinct extends OneImplementationRuleFactory { + public static final SplitAggWithoutDistinct INSTANCE = new SplitAggWithoutDistinct(); + + @Override + public Rule build() { + return logicalAggregate() + .whenNot(Aggregate::hasDistinctFunc) + .thenApplyMulti(ctx -> rewrite(ctx.root, ctx.connectContext)) + .toRule(RuleType.SPLIT_AGG_WITHOUT_DISTINCT); + } + + private List rewrite(LogicalAggregate aggregate, ConnectContext ctx) { + switch (ctx.getSessionVariable().aggPhase) { + case 1: + return ImmutableList.builder() + .addAll(implementOnePhase(aggregate)) + .build(); + case 2: + return ImmutableList.builder() + .addAll(splitTwoPhase(aggregate)) + .build(); + default: + return ImmutableList.builder() + .addAll(implementOnePhase(aggregate)) + .addAll(splitTwoPhase(aggregate)) + .build(); + } + } + + /** + * select sum(a) from t group by b; + * LogicalAggregate(group by b, outputExpr: sum(a), b) + * -> + * PhysicalHashAggregate(group by b, outputExpr: sum(a), b; AGG_PHASE:GLOBAL) + * */ + private List implementOnePhase(LogicalAggregate logicalAgg) { + if (!logicalAgg.supportAggregatePhase(AggregatePhase.ONE)) { + return ImmutableList.of(); + } + ImmutableList.Builder builder = ImmutableList.builder(); + boolean changed = false; + for (NamedExpression expr : logicalAgg.getOutputExpressions()) { + if (expr instanceof Alias && expr.child(0) instanceof AggregateFunction) { + Alias alias = (Alias) expr; + AggregateExpression aggExpr = new AggregateExpression((AggregateFunction) expr.child(0), + AggregateParam.GLOBAL_RESULT); + builder.add(alias.withChildren(ImmutableList.of(aggExpr))); + changed = true; + } else { + builder.add(expr); + } + } + AggregateParam param = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT, !skipRegulator(logicalAgg)); + List aggOutput = changed ? builder.build() : logicalAgg.getOutputExpressions(); + return ImmutableList.of(new PhysicalHashAggregate<>(logicalAgg.getGroupByExpressions(), aggOutput, param, + AggregateUtils.maybeUsingStreamAgg(logicalAgg.getGroupByExpressions(), param), + null, logicalAgg.child())); + } + + /** + * select sum(a) from t group by b; + * LogicalAggregate(group by b, outputExpr: sum(a), b) + * -> + * PhysicalHashAggregate(group by b, outputExpr: sum(a), b; AGG_PHASE:GLOBAL) + * +--PhysicalHashAggregate(group by b, outputExpr: partial_sum(a), b; AGG_PHASE:LOCAL) + * */ + private List splitTwoPhase(LogicalAggregate aggregate) { + if (!aggregate.supportAggregatePhase(AggregatePhase.TWO)) { + return ImmutableList.of(); + } + AggregateParam inputToBufferParam = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER); + Map aggFunctionToAlias = aggregate.getAggregateFunctions().stream() + .collect(ImmutableMap.toImmutableMap(function -> function, function -> { + AggregateExpression localAggFunc = new AggregateExpression(function, inputToBufferParam); + return new Alias(localAggFunc); + })); + List localAggOutput = ImmutableList.builder() + .addAll((List) aggregate.getGroupByExpressions()) + .addAll(aggFunctionToAlias.values()) + .build(); + + PhysicalHashAggregate localAgg = new PhysicalHashAggregate<>(aggregate.getGroupByExpressions(), + localAggOutput, inputToBufferParam, + AggregateUtils.maybeUsingStreamAgg(aggregate.getGroupByExpressions(), inputToBufferParam), + null, aggregate.child()); + + // global agg + AggregateParam bufferToResultParam = new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT); + List globalAggOutput = ExpressionUtils.rewriteDownShortCircuit( + aggregate.getOutputExpressions(), expr -> { + if (!(expr instanceof AggregateFunction)) { + return expr; + } + Alias alias = aggFunctionToAlias.get(expr); + if (alias == null) { + return expr; + } + AggregateFunction aggFunc = (AggregateFunction) expr; + return new AggregateExpression(aggFunc, bufferToResultParam, alias.toSlot()); + }); + return ImmutableList.of(new PhysicalHashAggregate<>(aggregate.getGroupByExpressions(), + globalAggOutput, bufferToResultParam, + AggregateUtils.maybeUsingStreamAgg(aggregate.getGroupByExpressions(), bufferToResultParam), + aggregate.getLogicalProperties(), localAgg)); + } + + private boolean shouldUseLocalAgg(LogicalAggregate aggregate) { + Statistics aggStats = aggregate.getGroupExpression().get().getOwnerGroup().getStatistics(); + Statistics aggChildStats = aggregate.getGroupExpression().get().childStatistics(0); + // if gbyNdv is high, should not use local agg + double rows = aggChildStats.getRowCount(); + double gbyNdv = aggStats.getRowCount(); + return gbyNdv * 10 < rows; + } + + private boolean skipRegulator(LogicalAggregate aggregate) { + for (AggregateFunction aggregateFunction : aggregate.getAggregateFunctions()) { + if (aggregateFunction.forceSkipRegulator(AggregatePhase.ONE)) { + return true; + } + } + return false; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DistinctAggStrategySelector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DistinctAggStrategySelector.java new file mode 100644 index 00000000000000..5b4e4092e54953 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DistinctAggStrategySelector.java @@ -0,0 +1,177 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.rules.rewrite.DistinctAggStrategySelector.DistinctSelectorContext; +import org.apache.doris.nereids.rules.rewrite.StatsDerive.DeriveContext; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.Statistics; + +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; + +/** + * Chooses the optimal execution strategy for queries with multiple DISTINCT aggregations. + * + * Handles queries like "SELECT COUNT(DISTINCT c1), COUNT(DISTINCT c2) FROM t" by selecting between: + * - CTE decomposition: Splits into multiple CTEs, each computing one DISTINCT aggregate + * - Multi-DISTINCT function: Processes all distinct function use multi distinct function + * + * Selection criteria includes: + * - Number of distinct aggregates + * - Estimated cardinality of distinct values + * - Available memory resources + * - Query complexity + */ +public class DistinctAggStrategySelector extends DefaultPlanRewriter + implements CustomRewriter { + public static DistinctAggStrategySelector INSTANCE = new DistinctAggStrategySelector(); + + /**DistinctSplitContext*/ + public static class DistinctSelectorContext { + List> cteProducerList; + StatementContext statementContext; + CascadesContext cascadesContext; + + public DistinctSelectorContext(StatementContext statementContext, CascadesContext cascadesContext) { + this.statementContext = statementContext; + this.cteProducerList = new ArrayList<>(); + this.cascadesContext = cascadesContext; + } + } + + @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 agg, DistinctSelectorContext ctx) { + Plan newChild = agg.child().accept(this, ctx); + agg = agg.withChildren(ImmutableList.of(newChild)); + // not process: + // count(distinct a,b); + // count(distinct a), sum(distinct a); + // count(distinct a) + // process: + // count(distinct a,b), count(distinct a,c) + // count(distinct a), sum(distinct b) + if (agg.distinctFuncNum() < 2 || agg.getDistinctArguments().size() < 2) { + return agg; + } + if (shouldUseMultiDistinct(agg)) { + return MultiDistinctFunctionStrategy.rewrite(agg); + } else { + return SplitMultiDistinctStrategy.rewrite(agg, ctx); + } + } + + private boolean shouldUseMultiDistinct(LogicalAggregate agg) { + if (AggregateUtils.containsCountDistinctMultiExpr(agg)) { + return false; + } + ConnectContext ctx = ConnectContext.get(); + if (ctx == null) { + return true; + } + switch (ctx.getSessionVariable().multiDistinctStrategy) { + case 1: + return true; + case 2: + return false; + default: + break; + } + if (agg.getStats() == null || agg.child().getStats() == null) { + StatsDerive derive = new StatsDerive(false); + agg.accept(derive, new DeriveContext()); + } + Statistics aggStats = agg.getStats(); + Statistics childStats = agg.child().getStats(); + double row = childStats.getRowCount(); + if (agg.getGroupByExpressions().isEmpty()) { + for (Expression distinctArgument : agg.getDistinctArguments()) { + ColumnStatistic columnStatistic = childStats.findColumnStatistics(distinctArgument); + if (columnStatistic == null || columnStatistic.isUnKnown) { + return false; + } + if (columnStatistic.ndv >= row * AggregateUtils.MID_CARDINALITY_THRESHOLD) { + // If there is a distinct key with high ndv, then do not use multi distinct + return false; + } + } + } else { + if (AggregateUtils.hasUnknownStatistics(agg.getGroupByExpressions(), childStats)) { + return false; + } + // The joint ndv of Group by key is high, so multi_distinct is not selected; + if (aggStats.getRowCount() >= row * AggregateUtils.LOW_CARDINALITY_THRESHOLD) { + return false; + } + // // TODO:Also need to consider the size of the group by key. + // //If the group by key size is larger than + // //a certain threshold, the network distribution of CTE will be very slow, + // so using multi_distinct will be better. + // double groupByKeyByte = 0; + // for (Expression groupByKey : agg.getGroupByExpressions()) { + // ColumnStatistic columnStatistic = childStats.findColumnStatistics(groupByKey); + // groupByKeyByte += columnStatistic.avgSizeByte; + // } + // // If ndv is large and group by key size is also large, should I use cte or multi distinct? + // if (groupByKeyByte < 20) { + // return false; + // } + } + return true; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DistinctAggregateRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DistinctAggregateRewriter.java new file mode 100644 index 00000000000000..2b7804973682a1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DistinctAggregateRewriter.java @@ -0,0 +1,216 @@ +// 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.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.StatsDerive.DeriveContext; +import org.apache.doris.nereids.stats.ExpressionEstimation; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.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.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.Statistics; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Process aggregate with a DISTINCT AGGFunction + * Decide whether to split DISTINCT aggregate into top-bottom aggregation or use multi_distinct; + * For example, transforms: + * SELECT COUNT(DISTINCT a), count(c) FROM t GROUP BY b + * 1.Rewrites queries containing DISTINCT aggregate functions by splitting them into two processing layers: + * Lower layer: Performs deduplication on both grouping columns and DISTINCT columns + * Upper layer: Applies simple aggregation on the deduplicated results + * rewrite into: + * SELECT COUNT(a), sum0(cnt) FROM ( + * SELECT a, b, count(c) cnt FROM t GROUP BY a, b + * ) GROUP BY b + * + * use multi_distinct: + * rewrite into: + *. SELECT MULTI_DISTINCT_COUNT(a), count(c) FROM t GROUP BY b + */ +public class DistinctAggregateRewriter implements RewriteRuleFactory { + public static final DistinctAggregateRewriter INSTANCE = new DistinctAggregateRewriter(); + // TODO: add other functions + private static final Set> supportSplitOtherFunctions = ImmutableSet.of( + Sum.class, Min.class, Max.class, Count.class, Sum0.class, AnyValue.class); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalAggregate() + .whenNot(agg -> agg.getGroupByExpressions().isEmpty()) + .whenNot(Aggregate::canSkewRewrite) + .thenApply(ctx -> rewrite(ctx.root, ctx.connectContext)) + .toRule(RuleType.DISTINCT_AGGREGATE_SPLIT), + logicalAggregate() + .when(agg -> agg.getGroupByExpressions().isEmpty() + && agg.mustUseMultiDistinctAgg()) + .then(this::convertToMultiDistinct) + .toRule(RuleType.PROCESS_SCALAR_AGG_MUST_USE_MULTI_DISTINCT) + ); + } + + private boolean shouldUseMultiDistinct(LogicalAggregate aggregate) { + // count(distinct a,b) cannot use multi_distinct + if (AggregateUtils.containsCountDistinctMultiExpr(aggregate)) { + return false; + } + if (aggregate.mustUseMultiDistinctAgg()) { + return true; + } + ConnectContext ctx = ConnectContext.get(); + if (ctx.getSessionVariable().aggPhase == 1 || ctx.getSessionVariable().aggPhase == 2) { + return true; + } + if (aggregate.getStats() == null || aggregate.child().getStats() == null) { + StatsDerive derive = new StatsDerive(false); + aggregate.accept(derive, new DeriveContext()); + } + Statistics aggStats = aggregate.getStats(); + Statistics aggChildStats = aggregate.child().getStats(); + Set dstArgs = aggregate.getDistinctArguments(); + // has unknown statistics, split to bottom and top agg + if (AggregateUtils.hasUnknownStatistics(aggregate.getGroupByExpressions(), aggChildStats) + || AggregateUtils.hasUnknownStatistics(dstArgs, aggChildStats)) { + return false; + } + + double gbyNdv = aggStats.getRowCount(); + Expression dstKey = dstArgs.iterator().next(); + ColumnStatistic dstKeyStats = aggChildStats.findColumnStatistics(dstKey); + if (dstKeyStats == null) { + dstKeyStats = ExpressionEstimation.estimate(dstKey, aggChildStats); + } + double dstNdv = dstKeyStats.ndv; + double inputRows = aggChildStats.getRowCount(); + // group by key ndv is low, distinct key ndv is high, multi_distinct is better + // otherwise split to bottom and top agg + return gbyNdv < inputRows * AggregateUtils.LOW_CARDINALITY_THRESHOLD + && dstNdv > inputRows * AggregateUtils.HIGH_CARDINALITY_THRESHOLD; + } + + private Plan rewrite(LogicalAggregate aggregate, ConnectContext ctx) { + if (aggregate.distinctFuncNum() == 0) { + return null; + } + if (ctx.getSessionVariable().aggPhase == 3 || ctx.getSessionVariable().aggPhase == 4) { + return null; + } + if (shouldUseMultiDistinct(aggregate)) { + return convertToMultiDistinct(aggregate); + } else { + return splitDistinctAgg(aggregate); + } + } + + private Plan convertToMultiDistinct(LogicalAggregate aggregate) { + return MultiDistinctFunctionStrategy.rewrite(aggregate); + } + + private Plan splitDistinctAgg(LogicalAggregate aggregate) { + Set aggFuncs = aggregate.getAggregateFunctions(); + Set distinctAggFuncs = new HashSet<>(); + Set otherFunctions = new HashSet<>(); + for (AggregateFunction aggFunc : aggFuncs) { + if (aggFunc.isDistinct()) { + distinctAggFuncs.add(aggFunc); + } else { + otherFunctions.add(aggFunc); + } + } + if (distinctAggFuncs.size() != 1) { + return null; + } + // If there are some functions that cannot be split in other function, AGG cannot be split + for (AggregateFunction aggFunc : otherFunctions) { + if (!supportSplitOtherFunctions.contains(aggFunc.getClass())) { + return null; + } + } + + // construct bottom agg + // group by key: group by key + distinct key + Set groupByKeys = AggregateUtils.getAllKeySet(aggregate); + ImmutableList.Builder bottomAggOtherFunctions = ImmutableList.builder(); + Map aggFuncToSlot = new HashMap<>(); + for (AggregateFunction aggFunc : otherFunctions) { + Alias bottomAggFuncAlias = new Alias(aggFunc); + bottomAggOtherFunctions.add(bottomAggFuncAlias); + aggFuncToSlot.put(aggFunc, bottomAggFuncAlias.toSlot()); + } + + List aggOutput = ImmutableList.builder() + .addAll(groupByKeys) + .addAll(bottomAggOtherFunctions.build()) + .build(); + + LogicalAggregate bottomAgg = new LogicalAggregate<>(Utils.fastToImmutableList(groupByKeys), + aggOutput, aggregate.child()); + + // construct top agg + List topAggOutput = ExpressionUtils.rewriteDownShortCircuit(aggregate.getOutputExpressions(), + expr -> { + if (expr instanceof AggregateFunction) { + AggregateFunction aggFunc = (AggregateFunction) expr; + if (aggFunc.isDistinct()) { + if (aggFunc instanceof Count && aggFunc.arity() > 1) { + return AggregateUtils.countDistinctMultiExprToCountIf((Count) aggFunc); + } else { + return aggFunc.withDistinctAndChildren(false, aggFunc.children()); + } + } else { + if (aggFuncToSlot.get(aggFunc) != null) { + if (aggFunc instanceof Count) { + return new Sum0(aggFuncToSlot.get(aggFunc)); + } else { + return aggFunc.withChildren(aggFuncToSlot.get(aggFunc)); + } + } + return aggFunc; + } + } + return expr; + } + ); + return new LogicalAggregate(aggregate.getGroupByExpressions(), topAggOutput, bottomAgg); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MultiDistinctFunctionStrategy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MultiDistinctFunctionStrategy.java new file mode 100644 index 00000000000000..bfef40da8cb848 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MultiDistinctFunctionStrategy.java @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.util.AggregateUtils; +import org.apache.doris.nereids.util.ExpressionUtils; + +import java.util.List; + +/** + * Processes all distinct function use multi distinct function + */ +public class MultiDistinctFunctionStrategy { + /** rewrite aggFunc(distinct) to multi_distinct_xxx */ + public static LogicalAggregate rewrite(LogicalAggregate aggregate) { + List newOutput = ExpressionUtils.rewriteDownShortCircuit( + aggregate.getOutputExpressions(), outputChild -> { + if (outputChild instanceof AggregateFunction) { + return AggregateUtils.tryConvertToMultiDistinct((AggregateFunction) outputChild); + } + return outputChild; + }); + return aggregate.withAggOutput(newOutput); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinct.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctStrategy.java similarity index 64% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinct.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctStrategy.java index 6adb3270b5b30c..950149657ef3a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinct.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctStrategy.java @@ -17,10 +17,8 @@ package org.apache.doris.nereids.rules.rewrite; -import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.jobs.JobContext; -import org.apache.doris.nereids.rules.rewrite.SplitMultiDistinct.DistinctSplitContext; +import org.apache.doris.nereids.rules.rewrite.DistinctAggStrategySelector.DistinctSelectorContext; +import org.apache.doris.nereids.rules.rewrite.StatsDerive.DeriveContext; import org.apache.doris.nereids.trees.copier.DeepCopierContext; import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; import org.apache.doris.nereids.trees.expressions.Alias; @@ -30,130 +28,84 @@ import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; -import org.apache.doris.nereids.trees.expressions.functions.agg.SupportMultiDistinct; import org.apache.doris.nereids.trees.plans.JoinType; 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.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.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -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.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; /** - * LogicalAggregate(output:count(distinct a) as c1, count(distinct b) as c2) - * +--Plan - * -> - * LogicalCTEAnchor - * +--LogicalCTEProducer - * +--Plan - * +--LogicalProject(c1, c2) - * +--LogicalJoin - * +--LogicalAggregate(output:count(distinct a)) - * +--LogicalCTEConsumer - * +--LogicalAggregate(output:count(distinct b)) - * +--LogicalCTEConsumer + * Split multi distinct strategy * */ -public class SplitMultiDistinct extends DefaultPlanRewriter implements CustomRewriter { - public static SplitMultiDistinct INSTANCE = new SplitMultiDistinct(); - - /**DistinctSplitContext*/ - public static class DistinctSplitContext { - List> cteProducerList; - StatementContext statementContext; - CascadesContext cascadesContext; - - public DistinctSplitContext(StatementContext statementContext, CascadesContext cascadesContext) { - this.statementContext = statementContext; - this.cteProducerList = new ArrayList<>(); - this.cascadesContext = cascadesContext; - } - } - - @Override - public Plan rewriteRoot(Plan plan, JobContext jobContext) { - DistinctSplitContext ctx = new DistinctSplitContext( - 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, DistinctSplitContext ctx) { - Plan child1 = anchor.child(0).accept(this, ctx); - DistinctSplitContext consumerContext = - new DistinctSplitContext(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 agg, DistinctSplitContext ctx) { - Plan newChild = agg.child().accept(this, ctx); - agg = agg.withChildren(ImmutableList.of(newChild)); - List distinctFuncWithAlias = new ArrayList<>(); +public class SplitMultiDistinctStrategy { + /** rewrite function*/ + public static Plan rewrite(LogicalAggregate agg, DistinctSelectorContext ctx) { + List> distinctFuncWithAlias = new ArrayList<>(); List otherAggFuncs = new ArrayList<>(); - if (!needTransform((LogicalAggregate) agg, distinctFuncWithAlias, otherAggFuncs)) { - return agg; - } - + collectDistinctAndNonDistinctFunctions(agg, distinctFuncWithAlias, otherAggFuncs); LogicalAggregate cloneAgg = (LogicalAggregate) LogicalPlanDeepCopier.INSTANCE .deepCopy(agg, new DeepCopierContext()); LogicalCTEProducer producer = new LogicalCTEProducer<>(ctx.statementContext.getNextCTEId(), cloneAgg.child()); ctx.cteProducerList.add(producer); + StatsDerive derive = new StatsDerive(false); + producer.accept(derive, new DeriveContext()); + Map originToProducerSlot = new HashMap<>(); for (int i = 0; i < agg.child().getOutput().size(); ++i) { Slot originSlot = agg.child().getOutput().get(i); Slot cloneSlot = cloneAgg.child().getOutput().get(i); originToProducerSlot.put(originSlot, cloneSlot); } - distinctFuncWithAlias = ExpressionUtils.replace((List) distinctFuncWithAlias, originToProducerSlot); + List> distinctFuncWithAliasReplaced = new ArrayList<>(); + for (List aliasList : distinctFuncWithAlias) { + distinctFuncWithAliasReplaced.add(ExpressionUtils.replace((List) aliasList, originToProducerSlot)); + } otherAggFuncs = ExpressionUtils.replace((List) otherAggFuncs, originToProducerSlot); // construct cte consumer and aggregate List> newAggs = new ArrayList<>(); // All otherAggFuncs are placed in the first one Map newToOriginDistinctFuncAlias = new HashMap<>(); List outputJoinGroupBys = new ArrayList<>(); - for (int i = 0; i < distinctFuncWithAlias.size(); ++i) { - Expression distinctAggFunc = distinctFuncWithAlias.get(i).child(0); + for (int i = 0; i < distinctFuncWithAliasReplaced.size(); ++i) { + List aliases = distinctFuncWithAliasReplaced.get(i); + List aggFuncs = new ArrayList<>(); + for (Alias alias : aliases) { + aggFuncs.add(alias.child()); + } Map producerToConsumerSlotMap = new HashMap<>(); List outputExpressions = new ArrayList<>(); List replacedGroupBy = new ArrayList<>(); LogicalCTEConsumer consumer = constructConsumerAndReplaceGroupBy(ctx, producer, cloneAgg, outputExpressions, producerToConsumerSlotMap, replacedGroupBy); - Expression newDistinctAggFunc = ExpressionUtils.replace(distinctAggFunc, producerToConsumerSlotMap); - Alias alias = new Alias(newDistinctAggFunc); - outputExpressions.add(alias); + List newDistinctAggFuncs = ExpressionUtils.replace(aggFuncs, producerToConsumerSlotMap); + List newAliases = new ArrayList<>(); + for (Expression expr : newDistinctAggFuncs) { + newAliases.add(new Alias(expr)); + } + outputExpressions.addAll(newAliases); if (i == 0) { // save replacedGroupBy outputJoinGroupBys.addAll(replacedGroupBy); } LogicalAggregate newAgg = new LogicalAggregate<>(replacedGroupBy, outputExpressions, consumer); newAggs.add(newAgg); - newToOriginDistinctFuncAlias.put(alias, distinctFuncWithAlias.get(i)); + for (int j = 0; j < aliases.size(); ++j) { + newToOriginDistinctFuncAlias.put(newAliases.get(j), aliases.get(j)); + } } buildOtherAggFuncAggregate(otherAggFuncs, producer, ctx, cloneAgg, newToOriginDistinctFuncAlias, newAggs); List groupBy = agg.getGroupByExpressions(); @@ -162,8 +114,8 @@ public Plan visitLogicalAggregate(LogicalAggregate agg, Distinct } private static void buildOtherAggFuncAggregate(List otherAggFuncs, LogicalCTEProducer producer, - DistinctSplitContext ctx, LogicalAggregate cloneAgg, Map newToOriginDistinctFuncAlias, - List> newAggs) { + DistinctSelectorContext ctx, LogicalAggregate cloneAgg, + Map newToOriginDistinctFuncAlias, List> newAggs) { if (otherAggFuncs.isEmpty()) { return; } @@ -184,7 +136,8 @@ private static void buildOtherAggFuncAggregate(List otherAggFuncs, Logica newAggs.add(newAgg); } - private static LogicalCTEConsumer constructConsumerAndReplaceGroupBy(DistinctSplitContext ctx, + private static LogicalCTEConsumer constructConsumerAndReplaceGroupBy( + DistinctSelectorContext ctx, LogicalCTEProducer producer, LogicalAggregate cloneAgg, List outputExpressions, Map producerToConsumerSlotMap, List replacedGroupBy) { LogicalCTEConsumer consumer = new LogicalCTEConsumer(ctx.statementContext.getNextRelationId(), @@ -211,38 +164,28 @@ private static boolean isDistinctMultiColumns(AggregateFunction func) { return false; } - private static boolean needTransform(LogicalAggregate agg, List aliases, List otherAggFuncs) { + private static void collectDistinctAndNonDistinctFunctions(LogicalAggregate agg, + List> aliases, List otherAggFuncs) { // TODO with source repeat aggregate need to be supported in future - if (agg.getSourceRepeat().isPresent()) { - return false; - } - Set distinctFunc = new HashSet<>(); - boolean distinctMultiColumns = false; + // 这个可能也没有关系,可以先注释掉,之后加一下关于grouping的测试 + // if (agg.getSourceRepeat().isPresent()) { + // return false; + // } + // boolean distinctMultiColumns = false; + Map, List> distinctArgToAliasList = new LinkedHashMap<>(); for (NamedExpression namedExpression : agg.getOutputExpressions()) { if (!(namedExpression instanceof Alias) || !(namedExpression.child(0) instanceof AggregateFunction)) { continue; } AggregateFunction aggFunc = (AggregateFunction) namedExpression.child(0); - if (aggFunc instanceof SupportMultiDistinct && aggFunc.isDistinct()) { - aliases.add((Alias) namedExpression); - distinctFunc.add(aggFunc); - distinctMultiColumns = distinctMultiColumns || isDistinctMultiColumns(aggFunc); + if (aggFunc.isDistinct()) { + distinctArgToAliasList.computeIfAbsent(ImmutableSet.copyOf(aggFunc.getDistinctArguments()), + k -> new ArrayList<>()).add((Alias) namedExpression); } else { otherAggFuncs.add((Alias) namedExpression); } } - if (distinctFunc.size() <= 1) { - return false; - } - // when this aggregate is not distinctMultiColumns, and group by expressions is not empty - // e.g. sql1: select count(distinct a), count(distinct b) from t1 group by c; - // sql2: select count(distinct a) from t1 group by c; - // the physical plan of sql1 and sql2 is similar, both are 2-phase aggregate, - // so there is no need to do this rewrite - if (!distinctMultiColumns && !agg.getGroupByExpressions().isEmpty()) { - return false; - } - return true; + aliases.addAll(distinctArgToAliasList.values()); } private static LogicalProject constructProject(List groupBy, Map joinOutput, @@ -251,7 +194,6 @@ private static LogicalProject constructProject(List groupBy, M for (Map.Entry entry : joinOutput.entrySet()) { projects.add(new Alias(entry.getValue().getExprId(), entry.getKey().toSlot(), entry.getValue().getName())); } - // outputJoinGroupBys.size() == agg.getGroupByExpressions().size() for (int i = 0; i < groupBy.size(); ++i) { Slot slot = (Slot) groupBy.get(i); projects.add(new Alias(slot.getExprId(), outputJoinGroupBys.get(i), slot.getName())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 2c09095332fdf9..8bd5c033349582 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -1316,7 +1316,7 @@ public Statistics computeLimit(Limit limit, Statistics inputStats) { /** * computeAggregate */ - private double estimateGroupByRowCount(List groupByExpressions, Statistics childStats) { + public static double estimateGroupByRowCount(List groupByExpressions, Statistics childStats) { double rowCount = 1; // if there is group-bys, output row count is childStats.getRowCount() * DEFAULT_AGGREGATE_RATIO, // o.w. output row count is 1 diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateParam.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateParam.java index 89e9c1ea7d7990..fa2e6ed582b6b1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateParam.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateParam.java @@ -27,6 +27,7 @@ public class AggregateParam { public static AggregateParam LOCAL_RESULT = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_RESULT); public static AggregateParam LOCAL_BUFFER = new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER); + public static AggregateParam GLOBAL_RESULT = new AggregateParam(AggPhase.GLOBAL, AggMode.INPUT_TO_RESULT); public final AggPhase aggPhase; public final AggMode aggMode; @@ -38,24 +39,13 @@ public AggregateParam(AggPhase aggPhase, AggMode aggMode) { this(aggPhase, aggMode, true); } + /** AggregateParam */ public AggregateParam(AggPhase aggPhase, AggMode aggMode, boolean canBeBanned) { this.aggMode = Objects.requireNonNull(aggMode, "aggMode cannot be null"); this.aggPhase = Objects.requireNonNull(aggPhase, "aggPhase cannot be null"); this.canBeBanned = canBeBanned; } - public AggregateParam withAggPhase(AggPhase aggPhase) { - return new AggregateParam(aggPhase, aggMode, canBeBanned); - } - - public AggregateParam withAggPhase(AggMode aggMode) { - return new AggregateParam(aggPhase, aggMode, canBeBanned); - } - - public AggregateParam withAppPhaseAndAppMode(AggPhase aggPhase, AggMode aggMode) { - return new AggregateParam(aggPhase, aggMode, canBeBanned); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java index 110ff349e0f7fd..15ae8b9c1c1a4a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java @@ -162,6 +162,7 @@ public Optional getGroupExpression() { return groupExpression; } + @Override public Statistics getStats() { return statistics; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AggMode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AggMode.java index 6fd1a477a5ed6a..dbbf77336e3232 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AggMode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AggMode.java @@ -19,19 +19,16 @@ /** AggregateMode */ public enum AggMode { - INPUT_TO_BUFFER(true, false, false), - INPUT_TO_RESULT(false, false, true), - BUFFER_TO_BUFFER(true, true, false), - BUFFER_TO_RESULT(false, true, true); + INPUT_TO_BUFFER(true, false), + INPUT_TO_RESULT(false, false), + BUFFER_TO_BUFFER(true, true), + BUFFER_TO_RESULT(false, true); public final boolean productAggregateBuffer; public final boolean consumeAggregateBuffer; - public final boolean isFinalPhase; - - AggMode(boolean productAggregateBuffer, boolean consumeAggregateBuffer, boolean isFinalPhase) { + AggMode(boolean productAggregateBuffer, boolean consumeAggregateBuffer) { this.productAggregateBuffer = productAggregateBuffer; this.consumeAggregateBuffer = consumeAggregateBuffer; - this.isFinalPhase = isFinalPhase; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java index bb842f6688d3fe..2d564e94bcc1b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.util.MutableState; import org.apache.doris.nereids.util.PlanUtils; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.Statistics; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -310,4 +311,8 @@ default DataTrait computeDataTrait() { void computeEqualSet(DataTrait.Builder builder); void computeFd(DataTrait.Builder builder); + + default Statistics getStats() { + throw new IllegalStateException("Not support getStats for " + getClass().getName()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Aggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Aggregate.java index 0c00515ac4517e..2d3058e434a1a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Aggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Aggregate.java @@ -112,6 +112,33 @@ default boolean isDistinct() { && getGroupByExpressions().stream().allMatch(e -> e instanceof Slot); } + /** + * isAggregateDistinct + * @return true if there is at least one distinct aggregate function + */ + default boolean hasDistinctFunc() { + for (AggregateFunction aggFunc : getAggregateFunctions()) { + if (aggFunc.isDistinct()) { + return true; + } + } + return false; + } + + /** + * distinctFuncNum + * @return number of distinct aggregate functions + */ + default int distinctFuncNum() { + int num = 0; + for (AggregateFunction aggFunc : getAggregateFunctions()) { + if (aggFunc.isDistinct()) { + ++num; + } + } + return num; + } + /** * Skew rewrite is applicable only when all the following conditions are met: * 1. The rule is not disabled in the current session (checked via `disableRules`). diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java index d3d42d2a3fefad..a46c46e975ca85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java @@ -22,8 +22,6 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.properties.RequireProperties; -import org.apache.doris.nereids.properties.RequirePropertiesSupplier; import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; @@ -61,7 +59,7 @@ * Physical hash aggregation plan. */ public class PhysicalHashAggregate extends PhysicalUnary - implements Aggregate, RequirePropertiesSupplier> { + implements Aggregate { private final List groupByExpressions; @@ -73,21 +71,18 @@ public class PhysicalHashAggregate extends PhysicalUnar private final boolean maybeUsingStream; - private final RequireProperties requireProperties; - public PhysicalHashAggregate(List groupByExpressions, List outputExpressions, AggregateParam aggregateParam, boolean maybeUsingStream, LogicalProperties logicalProperties, - RequireProperties requireProperties, CHILD_TYPE child) { + CHILD_TYPE child) { this(groupByExpressions, outputExpressions, Optional.empty(), aggregateParam, - maybeUsingStream, Optional.empty(), logicalProperties, requireProperties, child); + maybeUsingStream, Optional.empty(), logicalProperties, child); } public PhysicalHashAggregate(List groupByExpressions, List outputExpressions, Optional> partitionExpressions, AggregateParam aggregateParam, - boolean maybeUsingStream, LogicalProperties logicalProperties, RequireProperties requireProperties, - CHILD_TYPE child) { + boolean maybeUsingStream, LogicalProperties logicalProperties, CHILD_TYPE child) { this(groupByExpressions, outputExpressions, partitionExpressions, aggregateParam, - maybeUsingStream, Optional.empty(), logicalProperties, requireProperties, child); + maybeUsingStream, Optional.empty(), logicalProperties, child); } /** @@ -97,12 +92,10 @@ public PhysicalHashAggregate(List groupByExpressions, List groupByExpressions, List outputExpressions, Optional> partitionExpressions, AggregateParam aggregateParam, boolean maybeUsingStream, - Optional groupExpression, LogicalProperties logicalProperties, - RequireProperties requireProperties, CHILD_TYPE child) { + Optional groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) { super(PlanType.PHYSICAL_HASH_AGGREGATE, groupExpression, logicalProperties, child); this.groupByExpressions = ImmutableList.copyOf( Objects.requireNonNull(groupByExpressions, "groupByExpressions cannot be null")); @@ -112,8 +105,8 @@ public PhysicalHashAggregate(List groupByExpressions, List groupByExpressions, List groupByExpressions, List outputExpressions, Optional> partitionExpressions, AggregateParam aggregateParam, boolean maybeUsingStream, Optional groupExpression, LogicalProperties logicalProperties, - RequireProperties requireProperties, PhysicalProperties physicalProperties, - Statistics statistics, CHILD_TYPE child) { + PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { super(PlanType.PHYSICAL_HASH_AGGREGATE, groupExpression, logicalProperties, physicalProperties, statistics, child); this.groupByExpressions = ImmutableList.copyOf( @@ -141,7 +132,6 @@ public PhysicalHashAggregate(List groupByExpressions, List withRequireAndChildren( - RequireProperties requireProperties, List children) { - Preconditions.checkArgument(children.size() == 1); - return withRequirePropertiesAndChild(requireProperties, children.get(0)); - } - @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalHashAggregate(this, context); @@ -263,21 +241,13 @@ public boolean equals(Object o) { && Objects.equals(outputExpressions, that.outputExpressions) && Objects.equals(partitionExpressions, that.partitionExpressions) && Objects.equals(aggregateParam, that.aggregateParam) - && maybeUsingStream == that.maybeUsingStream - && Objects.equals(requireProperties, that.requireProperties); + && maybeUsingStream == that.maybeUsingStream; } @Override public int hashCode() { return Objects.hash(groupByExpressions, outputExpressions, partitionExpressions, - aggregateParam, maybeUsingStream, requireProperties); - } - - public PhysicalHashAggregate withGroupByExpressions(List newGroupByExpressions) { - return new PhysicalHashAggregate<>(newGroupByExpressions, outputExpressions, partitionExpressions, - aggregateParam, maybeUsingStream, groupExpression, getLogicalProperties(), - requireProperties, physicalProperties, statistics, - child()); + aggregateParam, maybeUsingStream); } @Override @@ -285,20 +255,13 @@ public PhysicalHashAggregate withChildren(List children) { Preconditions.checkArgument(children.size() == 1); return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, partitionExpressions, aggregateParam, maybeUsingStream, groupExpression, getLogicalProperties(), - requireProperties, physicalProperties, statistics, - children.get(0)); - } - - public PhysicalHashAggregate withPartitionExpressions(List partitionExpressions) { - return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, - Optional.ofNullable(partitionExpressions), aggregateParam, maybeUsingStream, - Optional.empty(), getLogicalProperties(), requireProperties, child()); + physicalProperties, statistics, children.get(0)); } @Override public PhysicalHashAggregate withGroupExpression(Optional groupExpression) { return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, partitionExpressions, - aggregateParam, maybeUsingStream, groupExpression, getLogicalProperties(), requireProperties, child()); + aggregateParam, maybeUsingStream, groupExpression, getLogicalProperties(), child()); } @Override @@ -306,8 +269,7 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, partitionExpressions, - aggregateParam, maybeUsingStream, groupExpression, logicalProperties.get(), - requireProperties, children.get(0)); + aggregateParam, maybeUsingStream, groupExpression, logicalProperties.get(), children.get(0)); } @Override @@ -315,22 +277,14 @@ public PhysicalHashAggregate withPhysicalPropertiesAndStats(Physical Statistics statistics) { return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, partitionExpressions, aggregateParam, maybeUsingStream, groupExpression, getLogicalProperties(), - requireProperties, physicalProperties, statistics, - child()); + physicalProperties, statistics, child()); } @Override public PhysicalHashAggregate withAggOutput(List newOutput) { return new PhysicalHashAggregate<>(groupByExpressions, newOutput, partitionExpressions, aggregateParam, maybeUsingStream, Optional.empty(), getLogicalProperties(), - requireProperties, physicalProperties, statistics, child()); - } - - public PhysicalHashAggregate withRequirePropertiesAndChild( - RequireProperties requireProperties, C newChild) { - return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, partitionExpressions, - aggregateParam, maybeUsingStream, Optional.empty(), getLogicalProperties(), - requireProperties, physicalProperties, statistics, newChild); + physicalProperties, statistics, child()); } @Override @@ -363,8 +317,7 @@ public List computeOutput() { public PhysicalHashAggregate resetLogicalProperties() { return new PhysicalHashAggregate<>(groupByExpressions, outputExpressions, partitionExpressions, aggregateParam, maybeUsingStream, groupExpression, null, - requireProperties, physicalProperties, statistics, - child()); + physicalProperties, statistics, child()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java index 889a5d8eb606e2..2ac48d8296b2a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java @@ -22,7 +22,6 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.properties.RequireProperties; -import org.apache.doris.nereids.properties.RequirePropertiesSupplier; import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow.WindowFrameGroup; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -49,8 +48,7 @@ /** * physical node for window function */ -public class PhysicalWindow extends PhysicalUnary implements Window, - RequirePropertiesSupplier> { +public class PhysicalWindow extends PhysicalUnary implements Window { private final WindowFrameGroup windowFrameGroup; private final RequireProperties requireProperties; @@ -139,11 +137,6 @@ public int hashCode() { return Objects.hash(windowFrameGroup, requireProperties); } - @Override - public RequireProperties getRequireProperties() { - return requireProperties; - } - @Override public Plan withChildren(List children) { Preconditions.checkState(children.size() == 1); @@ -172,12 +165,6 @@ public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalPr getLogicalProperties(), physicalProperties, statistics, child()); } - @Override - public PhysicalWindow withRequireAndChildren(RequireProperties requireProperties, List children) { - Preconditions.checkArgument(children.size() == 1); - return withRequirePropertiesAndChild(requireProperties, children.get(0)); - } - public PhysicalWindow withRequirePropertiesAndChild(RequireProperties requireProperties, C newChild) { return new PhysicalWindow<>(windowFrameGroup, requireProperties, windowExpressions, isSkew, Optional.empty(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/AggregateUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/AggregateUtils.java new file mode 100644 index 00000000000000..d641ec3d2578ec --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/AggregateUtils.java @@ -0,0 +1,137 @@ +// 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.util; + +import org.apache.doris.nereids.stats.ExpressionEstimation; +import org.apache.doris.nereids.trees.expressions.Cast; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.IsNull; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.SupportMultiDistinct; +import org.apache.doris.nereids.trees.expressions.functions.scalar.If; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.Statistics; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; + +import java.util.Collection; +import java.util.List; +import java.util.Set; + +/** + * Utils for aggregate + */ +public class AggregateUtils { + public static final double LOW_CARDINALITY_THRESHOLD = 0.001; + public static final double MID_CARDINALITY_THRESHOLD = 0.01; + public static final double HIGH_CARDINALITY_THRESHOLD = 0.1; + public static final int LOW_NDV_THRESHOLD = 1024; + + public static AggregateFunction tryConvertToMultiDistinct(AggregateFunction function) { + if (function instanceof SupportMultiDistinct && function.isDistinct()) { + return ((SupportMultiDistinct) function).convertToMultiDistinct(); + } + return function; + } + + /**countDistinctMultiExprToCountIf*/ + public static Expression countDistinctMultiExprToCountIf(Count count) { + Set arguments = ImmutableSet.copyOf(count.getArguments()); + Expression countExpr = count.getArgument(arguments.size() - 1); + for (int i = arguments.size() - 2; i >= 0; --i) { + Expression argument = count.getArgument(i); + If ifNull = new If(new IsNull(argument), NullLiteral.INSTANCE, countExpr); + countExpr = assignNullType(ifNull); + } + return new Count(countExpr); + } + + private static If assignNullType(If ifExpr) { + If ifWithCoercion = (If) TypeCoercionUtils.processBoundFunction(ifExpr); + Expression trueValue = ifWithCoercion.getArgument(1); + if (trueValue instanceof Cast && trueValue.child(0) instanceof NullLiteral) { + List newArgs = Lists.newArrayList(ifWithCoercion.getArguments()); + // backend don't support null type, so we should set the type + newArgs.set(1, new NullLiteral(((Cast) trueValue).getDataType())); + return ifWithCoercion.withChildren(newArgs); + } + return ifWithCoercion; + } + + public static boolean maybeUsingStreamAgg(List groupExpressions, AggregateParam param) { + ConnectContext ctx = ConnectContext.get(); + return ctx != null && !ctx.getSessionVariable().disableStreamPreaggregations + && !groupExpressions.isEmpty() + && param.aggPhase.isLocal(); + } + + /**hasUnknownStatistics*/ + public static boolean hasUnknownStatistics(Collection expressions, Statistics inputStatistics) { + for (Expression gbyExpr : expressions) { + ColumnStatistic colStats = inputStatistics.findColumnStatistics(gbyExpr); + if (colStats == null) { + colStats = ExpressionEstimation.estimate(gbyExpr, inputStatistics); + } + if (colStats == null || colStats.isUnKnown()) { + return true; + } + } + return false; + } + + public static boolean containsCountDistinctMultiExpr(LogicalAggregate aggregate) { + return ExpressionUtils.deapAnyMatch(aggregate.getOutputExpressions(), expr -> + expr instanceof Count && ((Count) expr).isDistinct() && expr.arity() > 1); + } + + /**getAllKeySet*/ + public static Set getAllKeySet(LogicalAggregate aggregate) { + Set distinctArguments = getDistinctNamedExpr(aggregate); + Set groupBySet = getGroupBySetNamedExpr(aggregate); + return ImmutableSet.builder() + .addAll(groupBySet) + .addAll(distinctArguments) + .build(); + } + + /**getGroupBySetNamedExpr*/ + public static Set getGroupBySetNamedExpr(LogicalAggregate aggregate) { + return aggregate.getGroupByExpressions().stream() + .filter(NamedExpression.class::isInstance) + .map(NamedExpression.class::cast) + .collect(ImmutableSet.toImmutableSet()); + } + + /**getDistinctNamedExpr*/ + public static Set getDistinctNamedExpr(LogicalAggregate aggregate) { + return aggregate.getAggregateFunctions().stream() + .filter(AggregateFunction::isDistinct) + .flatMap(aggFunc -> aggFunc.getArguments().stream()) + .filter(NamedExpression.class::isInstance) + .map(NamedExpression.class::cast) + .collect(ImmutableSet.toImmutableSet()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index a4b01bbdb9184d..7ec363b4c87ce0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -806,6 +806,8 @@ public static double getHotValueThreshold() { public static final String DEFAULT_VARIANT_MAX_SPARSE_COLUMN_STATISTICS_SIZE = "default_variant_max_sparse_column_statistics_size"; + public static final String MULTI_DISTINCT_STRATEGY = "multi_distinct_strategy"; + public static final String AGG_PHASE = "agg_phase"; /** * If set false, user couldn't submit analyze SQL and FE won't allocate any related resources. @@ -2458,6 +2460,28 @@ public void setSkewRewriteAggBucketNum(int num) { description = {"cast使用严格模式", "Use strict mode for cast"}) public boolean enableStrictCast = false; + @VariableMgr.VarAttr(name = MULTI_DISTINCT_STRATEGY, description = {"用于控制在包含多个 DISTINCT 函数的 SQL 查询中所采用的" + + "执行策略。默认值为 0,表示由系统自动选择最优策略;设为 1 表示强制使用 MultiDistinct 方式处理;" + + "设为 2 表示强制采用 CTE 拆分方式执行。", + "Used to control the execution strategy used in SQL queries containing multiple DISTINCT " + + "functions. The default value is 0, which means that the system automatically selects " + + "the optimal strategy; setting it to 1 means forcing the use of MultiDistinct processing;" + + " setting it to 2 means forcing the use of CTE splitting execution"}, + checker = "checkMultiDistinctStrategy") + public int multiDistinctStrategy = 0; + + @VariableMgr.VarAttr(name = AGG_PHASE, description = {"用于控制聚合查询的执行阶段划分策略。默认值为 0," + + "表示由系统自动选择最优执行阶段;设为 1 至 4 之间的值则表示强制指定使用对应 1 至 4 阶段进行聚合计算。", + "Controls the execution phase strategy for aggregate queries. The default value is 0," + + "which means the system automatically selects the optimal execution phase. Setting this value" + + "between 1 and 4 forces the use of phases 1 to 4 for aggregate calculations."}, + checker = "checkAggPhase") + public int aggPhase = 0; + + public void setAggPhase(int phase) { + aggPhase = phase; + } + public Set getIgnoredRuntimeFilterIds() { Set ids = Sets.newLinkedHashSet(); if (ignoreRuntimeFilterIds.isEmpty()) { @@ -5000,6 +5024,28 @@ public void checkSkewRewriteJoinSaltExplodeFactor(String factorStr) { } } + public void checkAggPhase(String aggPhaseStr) { + try { + long aggPhase = Long.parseLong(aggPhaseStr); + if (aggPhase < 0 || aggPhase > 4) { + throw new InvalidParameterException(AGG_PHASE + " should be between 0 and 4"); + } + } catch (NumberFormatException e) { + throw new InvalidParameterException(AGG_PHASE + " must be a valid number between 0 and 4"); + } + } + + public void checkMultiDistinctStrategy(String multiDistinctStrategyStr) { + try { + long aggPhase = Long.parseLong(multiDistinctStrategyStr); + if (aggPhase < 0 || aggPhase > 2) { + throw new InvalidParameterException(MULTI_DISTINCT_STRATEGY + " should be between 0 and 2"); + } + } catch (NumberFormatException e) { + throw new InvalidParameterException(MULTI_DISTINCT_STRATEGY + " must be a valid number between 0 and 2"); + } + } + public boolean isEnableInsertGroupCommit() { return Config.wait_internal_group_commit_finish || GroupCommitBlockSink.parseGroupCommit(groupCommit) == TGroupCommitMode.ASYNC_MODE diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java index b981028988be51..44b97f5125bb19 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java @@ -698,7 +698,6 @@ void testLocalPhaseAggregate() { new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_BUFFER), true, logicalProperties, - RequireProperties.of(PhysicalProperties.GATHER), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); @@ -723,7 +722,6 @@ void testGlobalPhaseAggregate() { new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.createHash(ImmutableList.of(partition), ShuffleType.REQUIRE)), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); @@ -753,7 +751,6 @@ void testAggregateWithoutGroupBy() { new AggregateParam(AggPhase.LOCAL, AggMode.BUFFER_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.GATHER), groupPlan ); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java index b9dbaaaec7d5dc..4524fc10e5e591 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java @@ -200,7 +200,6 @@ void testLocalAggregate() { new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.ANY), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); @@ -223,7 +222,6 @@ void testGlobalAggregate() { new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.createHash(ImmutableList.of(partition), ShuffleType.REQUIRE)), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); @@ -248,7 +246,6 @@ void testGlobalAggregateWithoutPartition() { new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.GATHER), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java index 8660d63f858b3a..996a4bc675b2dd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java @@ -21,7 +21,9 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; -import org.apache.doris.nereids.rules.implementation.AggregateStrategies; +import org.apache.doris.nereids.rules.implementation.SplitAggMultiPhase; +import org.apache.doris.nereids.rules.implementation.SplitAggMultiPhaseWithoutGbyKey; +import org.apache.doris.nereids.rules.implementation.SplitAggWithoutDistinct; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; @@ -239,9 +241,10 @@ public void distinctAggregateWithoutGroupByApply2PhaseRule() { Plan root = new LogicalAggregate<>(groupExpressionList, outputExpressionList, false, Optional.empty(), rStudent); - PlanChecker.from(MemoTestUtils.createConnectContext(), root) + ConnectContext connectContext = MemoTestUtils.createConnectContext(); + connectContext.getSessionVariable().setAggPhase(2); + PlanChecker.from(connectContext, root) .applyBottomUp(new NormalizeAggregate()) - .applyImplementation(twoPhaseAggregateWithDistinct()) .matches( physicalHashAggregate( physicalHashAggregate() @@ -281,8 +284,9 @@ public void distinctWithNormalAggregateFunctionApply2PhaseRule() { // sum Sum sumId = new Sum(false, id.toSlot()); - PlanChecker.from(MemoTestUtils.createConnectContext(), root) - .applyImplementation(twoPhaseAggregateWithDistinct()) + ConnectContext connectContext = MemoTestUtils.createConnectContext(); + connectContext.getSessionVariable().setAggPhase(2); + PlanChecker.from(connectContext, root) .matches( physicalHashAggregate( physicalHashAggregate() @@ -397,7 +401,7 @@ public void distinctApply4PhaseRuleNullableChange() { // select count(distinct id), sum(id) from t; PlanChecker.from(MemoTestUtils.createConnectContext(), root) - .applyImplementation(fourPhaseAggregateWithDistinct()) + .applyImplementation(fourPhaseAggregateWithDistinctWithoutGbyKey()) .matches( physicalHashAggregate( physicalHashAggregate( @@ -420,26 +424,27 @@ public void distinctApply4PhaseRuleNullableChange() { } private Rule twoPhaseAggregateWithoutDistinct() { - return new AggregateStrategies().buildRules() + return SplitAggWithoutDistinct.INSTANCE.buildRules() .stream() - .filter(rule -> rule.getRuleType() == RuleType.TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT) + .filter(rule -> rule.getRuleType() == RuleType.SPLIT_AGG_WITHOUT_DISTINCT) .findFirst() .get(); } - private Rule twoPhaseAggregateWithDistinct() { - return new AggregateStrategies().buildRules() + @Developing + private Rule fourPhaseAggregateWithDistinct() { + return SplitAggMultiPhase.INSTANCE.buildRules() .stream() - .filter(rule -> rule.getRuleType() == RuleType.TWO_PHASE_AGGREGATE_WITH_DISTINCT) + .filter(rule -> rule.getRuleType() == RuleType.SPLIT_AGG_MULTI_PHASE) .findFirst() .get(); } @Developing - private Rule fourPhaseAggregateWithDistinct() { - return new AggregateStrategies().buildRules() + private Rule fourPhaseAggregateWithDistinctWithoutGbyKey() { + return SplitAggMultiPhaseWithoutGbyKey.INSTANCE.buildRules() .stream() - .filter(rule -> rule.getRuleType() == RuleType.FOUR_PHASE_AGGREGATE_WITH_DISTINCT) + .filter(rule -> rule.getRuleType() == RuleType.SPLIT_AGG_MULTI_PHASE_WITHOUT_GBY_KEY) .findFirst() .get(); } @@ -535,9 +540,9 @@ public void skewCountMultiColumnDistinctNotRewrite() { } private Rule skewRewriteRule() { - return new AggregateStrategies().buildRules() + return SplitAggMultiPhase.INSTANCE.buildRules() .stream() - .filter(rule -> rule.getRuleType() == RuleType.AGG_SKEW_REWRITE) + .filter(rule -> rule.getRuleType() == RuleType.SPLIT_AGG_MULTI_PHASE) .findFirst() .get(); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/DistinctAggregateRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/DistinctAggregateRewriterTest.java new file mode 100644 index 00000000000000..1664cc18dfcf98 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/DistinctAggregateRewriterTest.java @@ -0,0 +1,164 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctGroupConcat; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum0; +import org.apache.doris.nereids.trees.expressions.functions.scalar.If; +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +public class DistinctAggregateRewriterTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + createTable("create table test.distinct_agg_split_t(a int null, b int not null," + + "c varchar(10) null, d date, dt datetime)\n" + + "distributed by hash(a) properties('replication_num' = '1');"); + connectContext.setDatabase("test"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + void testSplitSingleDistinctAgg() { + PlanChecker.from(connectContext) + .analyze("select b, count(distinct a) from test.distinct_agg_split_t group by b") + .rewrite() + .printlnTree() + .matches( + logicalAggregate( + logicalAggregate() + .when(agg -> agg.getGroupByExpressions().size() == 2 + && agg.getAggregateFunctions().isEmpty()) + ).when(agg -> agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b") + && agg.getAggregateFunctions().iterator().next() instanceof Count + ) + ); + } + + @Test + void testSplitSingleDistinctAggOtherFunctionCount() { + PlanChecker.from(connectContext) + .analyze("select b, count(distinct a), count(a) from test.distinct_agg_split_t group by b") + .rewrite() + .printlnTree() + .matches( + logicalAggregate( + logicalAggregate() + .when(agg -> agg.getGroupByExpressions().size() == 2 + && agg.getAggregateFunctions().iterator().next() instanceof Count) + ).when(agg -> agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b") + && agg.getAggregateFunctions().stream().anyMatch(f -> f instanceof Sum0) + ) + ); + } + + @Test + void testSplitSingleDistinctWithOtherAgg() { + PlanChecker.from(connectContext) + .analyze("select b, count(distinct a), sum(c) from test.distinct_agg_split_t group by b") + .rewrite() + .printlnTree() + .matches( + logicalAggregate( + logicalAggregate() + .when(agg -> agg.getGroupByExpressions().size() == 2) + ).when(agg -> agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b") + && agg.getAggregateFunctions().stream().noneMatch(AggregateFunction::isDistinct) + )); + } + + @Test + void testNotSplitWhenNoGroupBy() { + PlanChecker.from(connectContext) + .analyze("select count(distinct a) from test.distinct_agg_split_t") + .rewrite() + .printlnTree() + .nonMatch(logicalAggregate(logicalAggregate())); + } + + @Test + void testSplitWhenNoGroupByHasGroupConcatDistinctOrderBy() { + PlanChecker.from(connectContext) + .analyze("select group_concat(distinct a, '' order by b) from test.distinct_agg_split_t") + .rewrite() + .printlnTree() + .matches(logicalAggregate() + .when(agg -> + agg.getAggregateFunctions().iterator().next() instanceof MultiDistinctGroupConcat)); + } + + @Test + void testSplitWhenNoGroupByHasGroupConcatDistinct() { + PlanChecker.from(connectContext) + .analyze("select group_concat(distinct a, '') from test.distinct_agg_split_t") + .rewrite() + .printlnTree() + .nonMatch(logicalAggregate() + .when(agg -> + agg.getAggregateFunctions().iterator().next() instanceof MultiDistinctGroupConcat)); + } + + @Test + void testMultiExprDistinct() { + PlanChecker.from(connectContext) + .analyze("select b, sum(a), count(distinct a,c) from test.distinct_agg_split_t group by b") + .rewrite() + .printlnTree() + .matches( + logicalAggregate( + logicalAggregate() + .when(agg -> agg.getGroupByExpressions().size() == 3 + && agg.getAggregateFunctions().size() == 1) + ).when(agg -> agg.getGroupByExpressions().size() == 1 + && agg.getAggregateFunctions().stream().anyMatch( + f -> f instanceof Count && f.child(0) instanceof If + && !f.isDistinct())) + ); + } + + @Test + void testNotSplitWhenNoDistinct() { + PlanChecker.from(connectContext) + .analyze("select b, sum(a), count(c) from test.distinct_agg_split_t group by b") + .rewrite() + .printlnTree() + .nonMatch(logicalAggregate(logicalAggregate())); + } + + @Test + void testSplitWithComplexExpression() { + PlanChecker.from(connectContext) + .analyze("select b, count(distinct a + 1) from test.distinct_agg_split_t group by b") + .rewrite() + .printlnTree() + .matches( + logicalAggregate( + logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2) + ).when(agg -> agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b"))); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctTest.java index 642bf5d669a605..870124d2a498a4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/SplitMultiDistinctTest.java @@ -151,14 +151,14 @@ void countMultiColumnsWithoutGby() { physicalNestedLoopJoin( physicalProject( physicalHashAggregate( - physicalHashAggregate( - physicalDistribute( + physicalDistribute( + physicalHashAggregate( physicalHashAggregate(any()))))), physicalDistribute( physicalProject( physicalHashAggregate( - physicalHashAggregate( - physicalDistribute( + physicalDistribute( + physicalHashAggregate( physicalHashAggregate(any())))))) ) @@ -184,13 +184,11 @@ void countMultiColumnsWithGby() { physicalProject( physicalHashAggregate( physicalHashAggregate( - physicalDistribute( - physicalHashAggregate(any()))))), + physicalDistribute(any())))), physicalProject( physicalHashAggregate( physicalHashAggregate( - physicalDistribute( - physicalHashAggregate(any()))))) + physicalDistribute(any())))) ).when(join -> join.getJoinType() == JoinType.INNER_JOIN && join.getHashJoinConjuncts().get(0) instanceof NullSafeEqual ) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java index 5e3152219c9a48..59c6eac3593dbb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java @@ -23,8 +23,6 @@ import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.OrderKey; -import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.properties.RequireProperties; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -251,14 +249,14 @@ void testPhysicalAggregate(@Mocked LogicalProperties logicalProperties) { new SlotReference(new ExprId(0), "a", BigIntType.INSTANCE, true, Lists.newArrayList())); PhysicalHashAggregate actual = new PhysicalHashAggregate<>(Lists.newArrayList(), outputExpressionList, new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.GATHER), child); + child); List outputExpressionList1 = ImmutableList.of( new SlotReference(new ExprId(0), "a", BigIntType.INSTANCE, true, Lists.newArrayList())); PhysicalHashAggregate expected = new PhysicalHashAggregate<>(Lists.newArrayList(), outputExpressionList1, new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.GATHER), child); + child); Assertions.assertEquals(expected, actual); List outputExpressionList2 = ImmutableList.of( @@ -266,7 +264,7 @@ void testPhysicalAggregate(@Mocked LogicalProperties logicalProperties) { PhysicalHashAggregate unexpected = new PhysicalHashAggregate<>(Lists.newArrayList(), outputExpressionList2, new AggregateParam(AggPhase.LOCAL, AggMode.INPUT_TO_RESULT), false, logicalProperties, - RequireProperties.of(PhysicalProperties.GATHER), child); + child); Assertions.assertNotEquals(unexpected, actual); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryCacheNormalizerTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryCacheNormalizerTest.java index ad39a2774873d2..8ddd9dc7410d54 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryCacheNormalizerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryCacheNormalizerTest.java @@ -146,10 +146,12 @@ public void testProjectOnOlapScan() throws Exception { Assertions.assertEquals(64, digest3.length()); } + // after add hint control agg phase, this case will be reopen @Test public void testProjectOnAggregate() throws Exception { connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION,TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT"); + .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setAggPhase(1); try { String digest1 = getDigest( "select k1 + 1, k2 + 2, sum(v1) + 3, sum(v2) + 4 as v from db1.non_part group by k1, k2" @@ -162,6 +164,7 @@ public void testProjectOnAggregate() throws Exception { } finally { connectContext.getSessionVariable() .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setAggPhase(0); } } @@ -203,7 +206,7 @@ public void testMultiLevelPartitionTable() throws Throwable { @Test public void testHaving() throws Throwable { - List normalizedPlanNodes = onePhaseAggWithoutDistinct(() -> normalizePlans( + List normalizedPlanNodes = phaseAgg(1, () -> normalizePlans( "select k1, sum(v1) as v from db1.part1 where dt='2024-05-01' group by 1 having v > 10")); Assertions.assertEquals(2, normalizedPlanNodes.size()); Assertions.assertTrue( @@ -301,7 +304,7 @@ public void testSelectFromWhereNoGroupBy() throws Throwable { @Test public void testSelectFromGroupBy() { - twoPhaseAggWithoutDistinct(() -> { + phaseAgg(2, () -> { TQueryCacheParam queryCacheParam1 = getQueryCacheParam("select k1, sum(v1) from db1.part1 group by k1"); TQueryCacheParam queryCacheParam2 = getQueryCacheParam("select k1, sum(v1) from db1.part1 group by k1 limit 10"); Assertions.assertNotEquals(queryCacheParam1.digest, queryCacheParam2.digest); @@ -319,38 +322,39 @@ public void testSelectFromGroupBy() { List plans2 = normalizePlans( "select sum(v1), k1 from db1.part1 where dt between '2024-04-10' and '2024-05-06' group by k1"); Assertions.assertEquals(plans1, plans2); + return null; }); } @Test public void phasesAgg() { - List onePhaseAggPlans = onePhaseAggWithoutDistinct(() -> normalizePlans( + List onePhaseAggPlans = phaseAgg(1, () -> normalizePlans( "select sum(v1), k1 from db1.part1 where dt = '2024-04-10' group by k1")); - List onePhaseAggPlans2 = onePhaseAggWithoutDistinct(() -> normalizePlans( + List onePhaseAggPlans2 = phaseAgg(1, () -> normalizePlans( "select k1, sum(v1) from db1.part1 where dt = '2024-04-10' group by k1")); Assertions.assertEquals(onePhaseAggPlans, onePhaseAggPlans2); - List twoPhaseAggPlans = twoPhaseAggWithoutDistinct(() -> normalizePlans( + List twoPhaseAggPlans = phaseAgg(2, () -> normalizePlans( "select sum(v1), k1 from db1.part1 where dt = '2024-04-10' group by k1")); Assertions.assertNotEquals(onePhaseAggPlans, twoPhaseAggPlans); } @Test public void phasesDistinctAgg() { - List noDistinctPlans = onePhaseAggWithoutDistinct(() -> normalizePlans( + List noDistinctPlans = phaseAgg(1, () -> normalizePlans( "select k1 from db1.part1 where dt = '2024-04-10' group by k1")); - List onePhaseAggPlans = onePhaseAggWithDistinct(() -> normalizePlans( + List onePhaseAggPlans = phaseAgg(1, () -> normalizePlans( "select distinct k1 from db1.part1 where dt = '2024-04-10'")); Assertions.assertEquals(noDistinctPlans, onePhaseAggPlans); - List twoPhaseAggPlans = twoPhaseAggWithDistinct(() -> normalizePlans( + List twoPhaseAggPlans = phaseAgg(2, () -> normalizePlans( "select distinct k1 from db1.part1 where dt = '2024-04-10'")); - Assertions.assertEquals(onePhaseAggPlans, twoPhaseAggPlans); + Assertions.assertNotEquals(onePhaseAggPlans, twoPhaseAggPlans); - List threePhaseAggPlans = threePhaseAggWithDistinct(() -> normalizePlans( - "select sum(distinct v1), k1 from db1.part1 where dt = '2024-04-10' group by k1")); - List fourPhaseAggPlans = fourPhaseAggWithDistinct(() -> normalizePlans( - "select sum(distinct v1), k1 from db1.part1 where dt = '2024-04-10' group by k1")); - Assertions.assertNotEquals(fourPhaseAggPlans, threePhaseAggPlans); + List threePhaseAggPlans = phaseAgg(3, () -> normalizePlans( + "select sum(distinct v1), k2 from db1.part1 where dt = '2024-04-10' group by k2")); + List fourPhaseAggPlans = phaseAgg(4, () -> normalizePlans( + "select sum(distinct v1), k2 from db1.part1 where dt = '2024-04-10' group by k2")); + Assertions.assertEquals(fourPhaseAggPlans, threePhaseAggPlans); } private String getDigest(String sql) throws Exception { @@ -390,115 +394,14 @@ private List normalizePlans(String sql) throws Exception { return normalizedPlans; } - private void onePhaseAggWithoutDistinct(Callback callback) { - onePhaseAggWithoutDistinct(() -> { - callback.run(); - return null; - }); - } - - private T onePhaseAggWithDistinct(ResultCallback callback) { - try { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION," - + "TWO_PHASE_AGGREGATE_WITH_DISTINCT," - + "THREE_PHASE_AGGREGATE_WITH_DISTINCT," - + "FOUR_PHASE_AGGREGATE_WITH_DISTINCT," - + "FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE," - + "TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI"); - return callback.run(); - } catch (Throwable e) { - throw new RuntimeException(e); - } finally { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - } - } - - private T twoPhaseAggWithDistinct(ResultCallback callback) { + private T phaseAgg(int phase, ResultCallback callback) { try { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION," - + "ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI," - + "THREE_PHASE_AGGREGATE_WITH_DISTINCT," - + "FOUR_PHASE_AGGREGATE_WITH_DISTINCT," - + "FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE," - + "ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI"); + connectContext.getSessionVariable().setAggPhase(phase); return callback.run(); } catch (Throwable e) { throw new RuntimeException(e); } finally { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - } - } - - private T threePhaseAggWithDistinct(ResultCallback callback) { - try { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION," - + "ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI," - + "FOUR_PHASE_AGGREGATE_WITH_DISTINCT," - + "FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE," - + "ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI," - + "TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI"); - return callback.run(); - } catch (Throwable e) { - throw new RuntimeException(e); - } finally { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - } - } - - private T fourPhaseAggWithDistinct(ResultCallback callback) { - try { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION," - + "ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI," - + "THREE_PHASE_AGGREGATE_WITH_DISTINCT," - + "ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI," - + "TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI"); - return callback.run(); - } catch (Throwable e) { - throw new RuntimeException(e); - } finally { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - } - } - - - private T onePhaseAggWithoutDistinct(ResultCallback callback) { - try { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION,TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT"); - return callback.run(); - } catch (Throwable e) { - throw new RuntimeException(e); - } finally { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - } - } - - private void twoPhaseAggWithoutDistinct(Callback callback) { - twoPhaseAggWithoutDistinct(() -> { - callback.run(); - return null; - }); - } - - private T twoPhaseAggWithoutDistinct(ResultCallback callback) { - try { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION,ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT"); - return callback.run(); - } catch (Throwable e) { - throw new RuntimeException(e); - } finally { - connectContext.getSessionVariable() - .setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setAggPhase(0); } } diff --git a/regression-test/data/empty_relation/eliminate_empty.out b/regression-test/data/empty_relation/eliminate_empty.out index ae3c08f4cd6125..804cbb486bd845 100644 --- a/regression-test/data/empty_relation/eliminate_empty.out +++ b/regression-test/data/empty_relation/eliminate_empty.out @@ -10,7 +10,7 @@ PhysicalResultSink -- !explain_union_empty_data -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[nation] @@ -38,7 +38,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalProject ----------PhysicalOlapScan[nation] @@ -77,7 +77,7 @@ PhysicalResultSink -- !null_explain_union_empty_data -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[nation] @@ -105,7 +105,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalProject ----------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_p0/hint/fix_leading.out b/regression-test/data/nereids_p0/hint/fix_leading.out index 86ed69ddaec5d5..73abf90eb058de 100644 --- a/regression-test/data/nereids_p0/hint/fix_leading.out +++ b/regression-test/data/nereids_p0/hint/fix_leading.out @@ -231,13 +231,12 @@ SyntaxError: -- !select4_3 -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------NestedLoopJoin[RIGHT_OUTER_JOIN](c3 > 500) ---------NestedLoopJoin[LEFT_OUTER_JOIN](c1 < 200)(c1 > 500) -----------PhysicalOlapScan[t1] -----------filter((t2.c2 > 500)) -------------PhysicalOlapScan[t2] ---------PhysicalOlapScan[t3] +----NestedLoopJoin[RIGHT_OUTER_JOIN](c3 > 500) +------NestedLoopJoin[LEFT_OUTER_JOIN](c1 < 200)(c1 > 500) +--------PhysicalOlapScan[t1] +--------filter((t2.c2 > 500)) +----------PhysicalOlapScan[t2] +------PhysicalOlapScan[t3] Hint log: Used: leading(t1 t2 t3 ) diff --git a/regression-test/data/nereids_p0/hint/multi_leading.out b/regression-test/data/nereids_p0/hint/multi_leading.out index df5b5f1bd30048..89e22b4524756a 100644 --- a/regression-test/data/nereids_p0/hint/multi_leading.out +++ b/regression-test/data/nereids_p0/hint/multi_leading.out @@ -305,9 +305,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------NestedLoopJoin[INNER_JOIN](cast(sum(c11) as DOUBLE) > 0.05 * avg(t1.c11)) --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecGather] ------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_rules_p0/adjust_nullable/test_adjust_nullable.out b/regression-test/data/nereids_rules_p0/adjust_nullable/test_adjust_nullable.out index 05ca7f01e20dd5..09626347b110d7 100644 --- a/regression-test/data/nereids_rules_p0/adjust_nullable/test_adjust_nullable.out +++ b/regression-test/data/nereids_rules_p0/adjust_nullable/test_adjust_nullable.out @@ -1,9 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !avg_shape -- -PhysicalResultSink ---PhysicalProject[(cast(sum(DISTINCT b) as DOUBLE) / cast(count(DISTINCT b) as DOUBLE)) AS `AVG(distinct b)`, non_nullable((cast(sum(DISTINCT a) as DOUBLE) / cast(count(DISTINCT a) as DOUBLE))) AS `AVG(distinct a)`] -----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[test_adjust_nullable_t] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[test_adjust_nullable_t] +--PhysicalResultSink +----PhysicalProject[(cast(sum(DISTINCT b) as DOUBLE) / cast(count(DISTINCT b) as DOUBLE)) AS `AVG(distinct b)`, non_nullable((cast(sum(DISTINCT a) as DOUBLE) / cast(count(DISTINCT a) as DOUBLE))) AS `AVG(distinct a)`] +------hashJoin[INNER_JOIN colocated] hashCondition=((c <=> .c)) otherCondition=() +--------PhysicalProject[c AS `c`, count(DISTINCT a) AS `count(DISTINCT a)`, sum(DISTINCT a) AS `sum(DISTINCT a)`] +----------hashAgg[DISTINCT_GLOBAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------PhysicalProject[.c, count(DISTINCT b) AS `count(DISTINCT b)`, sum(DISTINCT b) AS `sum(DISTINCT b)`] +----------hashAgg[DISTINCT_GLOBAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_rules_p0/adjust_nullable/test_agg_nullable.out b/regression-test/data/nereids_rules_p0/adjust_nullable/test_agg_nullable.out index 7ec09edaf5d94a..4891c8d8ad0b49 100644 --- a/regression-test/data/nereids_rules_p0/adjust_nullable/test_agg_nullable.out +++ b/regression-test/data/nereids_rules_p0/adjust_nullable/test_agg_nullable.out @@ -6,6 +6,5 @@ PhysicalResultSink --PhysicalProject[AND[k IS NULL,NULL] AS `k > 10 and k < 5`] ----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------PhysicalEmptyRelation +------PhysicalEmptyRelation diff --git a/regression-test/data/nereids_rules_p0/adjust_nullable/test_subquery_nullable.out b/regression-test/data/nereids_rules_p0/adjust_nullable/test_subquery_nullable.out index 843b19fd4d8f3d..348814d39f1149 100644 --- a/regression-test/data/nereids_rules_p0/adjust_nullable/test_subquery_nullable.out +++ b/regression-test/data/nereids_rules_p0/adjust_nullable/test_subquery_nullable.out @@ -115,7 +115,7 @@ PhysicalResultSink --------PhysicalAssertNumRows ----------PhysicalDistribute[DistributionSpecGather] ------------PhysicalProject[count(x)] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject[test_subquery_nullable_t2.x] ------------------filter((test_subquery_nullable_t2.x > 1000)) --------------------PhysicalOlapScan[test_subquery_nullable_t2] @@ -135,7 +135,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject[cast(x as BIGINT) AS `cast(x as BIGINT)`, count(x)] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject[test_subquery_nullable_t2.x] --------------------filter((test_subquery_nullable_t2.x > 1000)) ----------------------PhysicalOlapScan[test_subquery_nullable_t2] diff --git a/regression-test/data/nereids_rules_p0/agg_skew_rewrite/agg_skew_rewrite.out b/regression-test/data/nereids_rules_p0/agg_skew_rewrite/agg_skew_rewrite.out index 6462609e3877a2..360052e0f2a12a 100644 --- a/regression-test/data/nereids_rules_p0/agg_skew_rewrite/agg_skew_rewrite.out +++ b/regression-test/data/nereids_rules_p0/agg_skew_rewrite/agg_skew_rewrite.out @@ -352,10 +352,9 @@ PhysicalResultSink -- !shape_hint_other_agg_func -- PhysicalResultSink ---hashAgg[DISTINCT_LOCAL] +--hashAgg[GLOBAL] ----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------PhysicalOlapScan[test_skew_hint] +------PhysicalOlapScan[test_skew_hint] -- !shape_hint_other_agg_func_expr -- PhysicalResultSink @@ -368,8 +367,9 @@ PhysicalResultSink -- !shape_hint_same_column_with_group_by -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalOlapScan[test_skew_hint] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalOlapScan[test_skew_hint] -- !shape_hint_same_column_with_group_by_expr -- PhysicalResultSink @@ -391,9 +391,10 @@ PhysicalResultSink -- !shape_hint_other_agg_func_grouping -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalRepeat ---------PhysicalOlapScan[test_skew_hint] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalRepeat +----------PhysicalOlapScan[test_skew_hint] -- !shape_hint_other_agg_func_expr_grouping -- PhysicalResultSink @@ -424,10 +425,9 @@ PhysicalResultSink -- !shape_hint_multi_column -- PhysicalResultSink ---hashAgg[DISTINCT_LOCAL] +--hashAgg[GLOBAL] ----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------PhysicalOlapScan[test_skew_hint] +------PhysicalOlapScan[test_skew_hint] -- !shape_hint_sum -- PhysicalResultSink @@ -541,10 +541,21 @@ PhysicalResultSink ------------------PhysicalOlapScan[test_skew_hint] -- !shape_hint_multi_count_distinct -- -PhysicalResultSink ---hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalOlapScan[test_skew_hint] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[test_skew_hint] +--PhysicalResultSink +----hashJoin[INNER_JOIN] hashCondition=((a <=> .a)) otherCondition=() +------hashAgg[DISTINCT_GLOBAL] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !shape_test_min_hint -- PhysicalResultSink @@ -556,12 +567,12 @@ PhysicalResultSink PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] -------PhysicalOlapScan[test_skew_hint] +------hashAgg[GLOBAL] +--------PhysicalOlapScan[test_skew_hint] -- !shape_not_rewrite -- PhysicalResultSink ---hashAgg[DISTINCT_LOCAL] +--hashAgg[GLOBAL] ----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------PhysicalOlapScan[test_skew_hint] +------PhysicalOlapScan[test_skew_hint] diff --git a/regression-test/data/nereids_rules_p0/agg_strategy/agg_strategy.out b/regression-test/data/nereids_rules_p0/agg_strategy/agg_strategy.out new file mode 100644 index 00000000000000..e63c5951f4be36 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/agg_strategy/agg_strategy.out @@ -0,0 +1,911 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !non_agg_func -- +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 + +-- !agg_func -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 + +-- !agg_distinct_with_gby_key -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 + +-- !agg_distinct_satisfy_gby_key -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 + +-- !agg_distinct_satisfy_dst_key -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !agg_distinct_with_gby_key_with_other_func -- +1 1 27 27.0 +1 2 76 76.0 +1 3 42 42.0 +1 4 64 64.0 +1 5 18 18.0 +1 6 91 91.0 +1 7 13 13.0 +1 8 33 33.0 +1 9 55 55.0 +1 10 100 100.0 + +-- !agg_distinct_satisfy_gby_key_with_other_func -- +1 1 42 42.0 +1 2 18 18.0 +1 3 76 76.0 +1 4 33 33.0 +1 5 91 91.0 +1 6 27 27.0 +1 7 64 64.0 +1 8 55 55.0 +1 9 13 13.0 +1 10 100 100.0 + +-- !agg_distinct_satisfy_dst_key_with_other_func -- +1 13 13.0 +1 18 18.0 +1 27 27.0 +1 33 33.0 +1 42 42.0 +1 55 55.0 +1 64 64.0 +1 76 76.0 +1 91 91.0 +1 100 100.0 + +-- !agg_distinct_without_gby_key -- +10 + +-- !agg_distinct_without_gby_key_satisfy_dst_key -- +10 + +-- !agg_distinct_without_gby_key_with_other_func -- +10 55 + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func -- +10 5.5 + +-- !non_agg_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_with_gby_key -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_gby_key -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_dst_key -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_with_gby_key_with_other_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_gby_key_with_other_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_dst_key_with_other_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[GLOBAL] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key_with_other_func -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !non_agg_func -- +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 + +-- !agg_func -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 + +-- !agg_distinct_with_gby_key -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 + +-- !agg_distinct_satisfy_gby_key -- +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 + +-- !agg_distinct_satisfy_dst_key -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !agg_distinct_with_gby_key_with_other_func -- +1 1 27 27.0 +1 2 76 76.0 +1 3 42 42.0 +1 4 64 64.0 +1 5 18 18.0 +1 6 91 91.0 +1 7 13 13.0 +1 8 33 33.0 +1 9 55 55.0 +1 10 100 100.0 + +-- !agg_distinct_satisfy_gby_key_with_other_func -- +1 1 42 42.0 +1 2 18 18.0 +1 3 76 76.0 +1 4 33 33.0 +1 5 91 91.0 +1 6 27 27.0 +1 7 64 64.0 +1 8 55 55.0 +1 9 13 13.0 +1 10 100 100.0 + +-- !agg_distinct_satisfy_dst_key_with_other_func -- +1 13 13.0 +1 18 18.0 +1 27 27.0 +1 33 33.0 +1 42 42.0 +1 55 55.0 +1 64 64.0 +1 76 76.0 +1 91 91.0 +1 100 100.0 + +-- !agg_distinct_without_gby_key -- +10 + +-- !agg_distinct_without_gby_key_satisfy_dst_key -- +10 + +-- !agg_distinct_without_gby_key_with_other_func -- +10 55 + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func -- +10 5.5 + +-- !non_agg_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_with_gby_key -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_gby_key -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_dst_key -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_with_gby_key_with_other_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_gby_key_with_other_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_satisfy_dst_key_with_other_func -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[GLOBAL] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key_with_other_func -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id] + +-- !count_multi_expr -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !count_multi_expr_multi_count -- +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 + +-- !non_agg_func_low_ndv -- +0 +1 + +-- !agg_func_low_ndv -- +15 0 +15 1 + +-- !agg_distinct_with_gby_key_low_ndv -- +2 0 +2 1 + +-- !agg_distinct_satisfy_gby_key_low_ndv -- +2 0 +2 1 + +-- !agg_distinct_satisfy_dst_key_low_ndv -- +2 +2 + +-- !agg_distinct_with_gby_key_with_other_func_low_ndv -- +2 0 60 4.0 +2 1 75 5.0 + +-- !agg_distinct_satisfy_gby_key_with_other_func_low_ndv -- +2 0 69 4.3125 +2 1 66 4.714285714285714 + +-- !agg_distinct_satisfy_dst_key_with_other_func_low_ndv -- +2 60 4.0 +2 75 5.0 + +-- !agg_distinct_without_gby_key_low_ndv -- +2 + +-- !agg_distinct_without_gby_key_satisfy_dst_key_low_ndv -- +2 + +-- !agg_distinct_without_gby_key_with_other_func_low_ndv -- +2 15 + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func_low_ndv -- +2 0.5 + +-- !non_agg_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_with_gby_key_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_gby_key_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_dst_key_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_with_gby_key_with_other_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_gby_key_with_other_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_dst_key_with_other_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_low_ndv -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key_low_ndv -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[GLOBAL] +--------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_with_other_func_low_ndv -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func_low_ndv -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !non_agg_func_low_ndv -- +0 +1 + +-- !agg_func_low_ndv -- +15 0 +15 1 + +-- !agg_distinct_with_gby_key_low_ndv -- +2 0 +2 1 + +-- !agg_distinct_satisfy_gby_key_low_ndv -- +2 0 +2 1 + +-- !agg_distinct_satisfy_dst_key_low_ndv -- +2 +2 + +-- !agg_distinct_with_gby_key_with_other_func_low_ndv -- +2 0 60 4.0 +2 1 75 5.0 + +-- !agg_distinct_satisfy_gby_key_with_other_func_low_ndv -- +2 0 69 4.3125 +2 1 66 4.714285714285714 + +-- !agg_distinct_satisfy_dst_key_with_other_func_low_ndv -- +2 60 4.0 +2 75 5.0 + +-- !agg_distinct_without_gby_key_low_ndv -- +2 + +-- !agg_distinct_without_gby_key_satisfy_dst_key_low_ndv -- +2 + +-- !agg_distinct_without_gby_key_with_other_func_low_ndv -- +2 15 + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func_low_ndv -- +2 0.5 + +-- !non_agg_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_with_gby_key_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_gby_key_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_dst_key_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_with_gby_key_with_other_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_gby_key_with_other_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_satisfy_dst_key_with_other_func_low_ndv -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalDistribute[DistributionSpecGather] +------PhysicalQuickSort[LOCAL_SORT] +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_low_ndv -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key_low_ndv -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[GLOBAL] +--------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_with_other_func_low_ndv -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !agg_distinct_without_gby_key_satisfy_dst_key_with_other_func_low_ndv -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalOlapScan[t_gbykey_2_dstkey_10_30_id] + +-- !with_gby_split_in_rewrite -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !with_gby_split_in_cascades -- +1 13.0 +1 18.0 +1 27.0 +1 33.0 +1 42.0 +1 55.0 +1 64.0 +1 76.0 +1 91.0 +1 100.0 + +-- !without_gby -- +10 + +-- !without_gby_satisfy -- +10 + +-- !group_concat_with_order_by_without_gby_with_distinct -- +7,9,4,1,3,5,8,2,6,10 + +-- !group_concat_with_order_by_with_gby_with_distinct -- +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + +-- !group_concat_with_order_by_without_gby -- +7,9,4,1,3,5,8,2,6,10 + +-- !group_concat_with_order_by_with_gby -- +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + +-- !group_concat_with_multi_order_by_without_gby_with_distinct -- +7,9,4,1,3,5,8,2,6,10 + +-- !group_concat_with_multi_order_by_with_gby_with_distinct -- +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + +-- !final_multi_distinct_sum0_count -- +10 10 1 55 55 10 + +-- !final_multi_distinct_sum0_sum -- +55 10 1 55 55 10 + +-- !final_multi_distinct_sum0_sum0 -- +55 10 1 55 55 10 + +-- !agg_recieve_hash_request_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalWindow +------PhysicalQuickSort[LOCAL_SORT] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[GLOBAL] +------------hashJoin[LEFT_OUTER_JOIN broadcast] hashCondition=((lineitem_left.l_orderkey = orders_left.o_orderkey)) otherCondition=() +--------------PhysicalOlapScan[lineitem_left] +--------------PhysicalOlapScan[orders_left] + +-- !agg_recieve_hash_request -- +2024-01-02 1 2024-01-02 1 + +-- !group_concat_distinct_key_is_varchar_and_distribute_key -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_dst_key1] + diff --git a/regression-test/data/nereids_rules_p0/agg_strategy/distinct_agg_rewriter.out b/regression-test/data/nereids_rules_p0/agg_strategy/distinct_agg_rewriter.out new file mode 100644 index 00000000000000..18088719ce3af2 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/agg_strategy/distinct_agg_rewriter.out @@ -0,0 +1,40 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !use_multi_phase1 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t1000_2] + +-- !use_multi_phase2 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[t1000_2] + +-- !use_multi_phase3 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t1000_2] + +-- !use_multi_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[t1000_2] + diff --git a/regression-test/data/nereids_rules_p0/agg_strategy/distinct_agg_strategy_selector.out b/regression-test/data/nereids_rules_p0/agg_strategy/distinct_agg_strategy_selector.out new file mode 100644 index 00000000000000..ea5361dc350ca8 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/agg_strategy/distinct_agg_strategy_selector.out @@ -0,0 +1,120 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !should_use_cte -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t1000] +--PhysicalResultSink +----NestedLoopJoin[CROSS_JOIN] +------NestedLoopJoin[CROSS_JOIN] +--------NestedLoopJoin[CROSS_JOIN] +----------hashAgg[DISTINCT_GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------hashAgg[DISTINCT_GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[DISTINCT_GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !should_use_multi_distinct -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[LOCAL] +--------PhysicalOlapScan[t1000] + +-- !should_use_cte_with_group_by -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t1000] +--PhysicalResultSink +----PhysicalDistribute[DistributionSpecGather] +------hashJoin[INNER_JOIN colocated] hashCondition=((d_20 <=> .d_20)) otherCondition=() +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !should_use_multi_distinct_with_group_by -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[t1000] + +-- !no_stats_should_use_cte -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t1000] +--PhysicalResultSink +----NestedLoopJoin[CROSS_JOIN] +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] +--------PhysicalDistribute[DistributionSpecGather] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !no_stats_should_use_cte_with_group_by -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t1000] +--PhysicalResultSink +----PhysicalDistribute[DistributionSpecGather] +------hashJoin[INNER_JOIN colocated] hashCondition=((a_1 <=> .a_1)) otherCondition=() +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) + diff --git a/regression-test/data/nereids_rules_p0/agg_strategy/physical_agg_regulator.out b/regression-test/data/nereids_rules_p0/agg_strategy/physical_agg_regulator.out new file mode 100644 index 00000000000000..85c8a56e2713e4 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/agg_strategy/physical_agg_regulator.out @@ -0,0 +1,81 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !skew -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t1025_skew5000] +--PhysicalResultSink +----PhysicalDistribute[DistributionSpecGather] +------hashJoin[INNER_JOIN colocated] hashCondition=((d_1025 <=> .d_1025)) otherCondition=() +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !not_skew -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t1025] +--PhysicalResultSink +----PhysicalDistribute[DistributionSpecGather] +------hashJoin[INNER_JOIN colocated] hashCondition=((d_1025 <=> .d_1025)) otherCondition=() +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !request_deriver_parent_ndv_high -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[t1025] + +-- !request_deriver_parent_ndv_low -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t1025] + +-- !split_multi_agg_use_three_phase -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[DISTINCT_GLOBAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[t1025] + +-- !split_multi_agg_use_four_phase -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[DISTINCT_GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t1025] + diff --git a/regression-test/data/nereids_rules_p0/agg_strategy/test_variables.out b/regression-test/data/nereids_rules_p0/agg_strategy/test_variables.out new file mode 100644 index 00000000000000..4c9866e9568385 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/agg_strategy/test_variables.out @@ -0,0 +1,256 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !agg_phase_1 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !agg_phase_2 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !agg_phase_3 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[DISTINCT_GLOBAL] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !agg_phase_4 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[DISTINCT_GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_phase_1_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_phase_1 -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !distinct_phase_2_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_phase_2 -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !distinct_phase_1_satisfy_gby_key_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_phase_1_satisfy_gby_key -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !distinct_phase_2_satisfy_gby_key_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_phase_2_satisfy_gby_key -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !distinct_and_other_phase_1_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_and_other_phase_1 -- +1 13 +1 18 +1 27 +1 33 +1 42 +1 55 +1 64 +1 76 +1 91 +1 100 + +-- !distinct_and_other_phase_2_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_and_other_phase_2 -- +1 13 +1 18 +1 27 +1 33 +1 42 +1 55 +1 64 +1 76 +1 91 +1 100 + +-- !distinct_and_other_phase_1_satisfy_gby_key_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_and_other_phase_1_satisfy_gby_key -- +1 13 +1 18 +1 27 +1 33 +1 42 +1 55 +1 64 +1 76 +1 91 +1 100 + +-- !distinct_and_other_phase_2_satisfy_gby_key_shape -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_and_other_phase_2_satisfy_gby_key -- +1 13 +1 18 +1 27 +1 33 +1 42 +1 55 +1 64 +1 76 +1 91 +1 100 + +-- !use_multi_phase_distinct_key_satisfy -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[DISTINCT_LOCAL] +--------hashAgg[GLOBAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !distinct_key_not_satisfy_use_final_multi -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !multi_distinct_strategy_1 -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] + +-- !multi_distinct_strategy_2 -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] +--PhysicalResultSink +----PhysicalDistribute[DistributionSpecGather] +------hashJoin[INNER_JOIN colocated] hashCondition=((gby_key <=> .gby_key)) otherCondition=() +--------hashJoin[INNER_JOIN colocated] hashCondition=((gby_key <=> .gby_key)) otherCondition=() +----------hashAgg[GLOBAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------hashAgg[GLOBAL] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !agg_phase4_and_multi_distinct_strategy2 -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[t_gbykey_10_dstkey_10_1000_id_2] +--PhysicalResultSink +----PhysicalDistribute[DistributionSpecGather] +------hashJoin[INNER_JOIN colocated] hashCondition=((gby_key <=> .gby_key)) otherCondition=() +--------hashJoin[INNER_JOIN colocated] hashCondition=((gby_key <=> .gby_key)) otherCondition=() +----------hashAgg[DISTINCT_GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------hashAgg[DISTINCT_GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------PhysicalCteConsumer ( cteId=CTEId#0 ) + diff --git a/regression-test/data/nereids_rules_p0/distinct_split/disitinct_split.out b/regression-test/data/nereids_rules_p0/distinct_split/disitinct_split.out index ede0fb5259cc62..a8bdae2200e23d 100644 --- a/regression-test/data/nereids_rules_p0/distinct_split/disitinct_split.out +++ b/regression-test/data/nereids_rules_p0/distinct_split/disitinct_split.out @@ -366,14 +366,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----PhysicalOlapScan[test_distinct_multi] --PhysicalResultSink ----NestedLoopJoin[CROSS_JOIN] -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !multi_count_mulitcols_with_gby -- PhysicalCteAnchor ( cteId=CTEId#0 ) @@ -381,14 +383,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----PhysicalOlapScan[test_distinct_multi] --PhysicalResultSink ----hashJoin[INNER_JOIN] hashCondition=((d <=> .d)) otherCondition=() -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !three_count_mulitcols_without_gby -- PhysicalCteAnchor ( cteId=CTEId#0 ) @@ -397,18 +401,21 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalResultSink ----NestedLoopJoin[CROSS_JOIN] ------NestedLoopJoin[CROSS_JOIN] +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[DISTINCT_GLOBAL] +----------hashAgg[DISTINCT_LOCAL] +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] --------hashAgg[DISTINCT_LOCAL] ----------hashAgg[GLOBAL] ------------hashAgg[LOCAL] --------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------hashAgg[DISTINCT_LOCAL] -----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !four_count_mulitcols_with_gby -- PhysicalCteAnchor ( cteId=CTEId#0 ) @@ -418,22 +425,26 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----hashJoin[INNER_JOIN] hashCondition=((d <=> .d)) otherCondition=() ------hashJoin[INNER_JOIN] hashCondition=((d <=> .d)) otherCondition=() --------hashJoin[INNER_JOIN] hashCondition=((d <=> .d)) otherCondition=() -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] ------------hashAgg[GLOBAL] --------------hashAgg[LOCAL] ----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------hashAgg[DISTINCT_LOCAL] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] ----------hashAgg[GLOBAL] ------------hashAgg[LOCAL] --------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !has_other_func -- PhysicalCteAnchor ( cteId=CTEId#0 ) @@ -464,40 +475,144 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((c <=> .c)) otherCondition=() -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------hashAgg[DISTINCT_LOCAL] -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] +----------------hashAgg[LOCAL] +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !multi_count_with_gby -- -PhysicalResultSink ---hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalOlapScan[test_distinct_multi] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[test_distinct_multi] +--PhysicalResultSink +----hashJoin[INNER_JOIN] hashCondition=((c <=> .c)) otherCondition=() +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !multi_sum_with_gby -- -PhysicalResultSink ---hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalOlapScan[test_distinct_multi] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[test_distinct_multi] +--PhysicalResultSink +----hashJoin[INNER_JOIN] hashCondition=((c <=> .c)) otherCondition=() +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !sum_count_with_gby -- -PhysicalResultSink ---hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalOlapScan[test_distinct_multi] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[test_distinct_multi] +--PhysicalResultSink +----hashJoin[INNER_JOIN] hashCondition=((a <=> .a)) otherCondition=() +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[GLOBAL] +--------hashAgg[GLOBAL] +----------hashAgg[LOCAL] +------------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !has_grouping -- -PhysicalResultSink ---hashAgg[GLOBAL] -----hashAgg[LOCAL] -------PhysicalRepeat ---------PhysicalOlapScan[test_distinct_multi] +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalRepeat +------PhysicalOlapScan[test_distinct_multi] +--PhysicalResultSink +----hashJoin[INNER_JOIN] hashCondition=((GROUPING_ID <=> .GROUPING_ID) and (a <=> .a) and (b <=> .b) and (c <=> .c)) otherCondition=() +------hashAgg[GLOBAL] +--------hashAgg[GLOBAL] +----------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[GLOBAL] +--------hashAgg[GLOBAL] +----------PhysicalCteConsumer ( cteId=CTEId#0 ) -- !null_hash -- 1 \N 0 0.0 +-- !same_distinct_arg -- +2 1 + +-- !same_distinct_arg_2group -- +1 3 2 + +-- !same_distinct_arg_shape -- +PhysicalResultSink +--hashAgg[DISTINCT_GLOBAL] +----hashAgg[DISTINCT_LOCAL] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------PhysicalOlapScan[test_distinct_multi] + +-- !same_distinct_arg_2group_shape -- +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) +----PhysicalOlapScan[test_distinct_multi] +--PhysicalResultSink +----NestedLoopJoin[CROSS_JOIN] +------hashAgg[DISTINCT_GLOBAL] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------hashAgg[DISTINCT_GLOBAL] +--------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] +------------hashAgg[LOCAL] +--------------PhysicalCteConsumer ( cteId=CTEId#0 ) + +-- !use_multi_distinct -- +\N \N \N 12 10 15500.00 1291.6666 12 +\N \N Clothing 5 5 3900.00 780.0000 5 +\N \N Electronics 7 5 11600.00 1657.1428 7 +East \N Clothing 1 1 850.00 850.0000 1 +East \N Electronics 1 1 1300.00 1300.0000 1 +East Nanjing \N 2 2 2150.00 1075.0000 2 +North \N Clothing 2 2 1400.00 700.0000 2 +North \N Electronics 3 2 4700.00 1566.6666 3 +North Beijing \N 3 2 3500.00 1166.6666 3 +North Tianjin \N 2 2 2600.00 1300.0000 2 +South \N Clothing 2 2 1650.00 825.0000 2 +South \N Electronics 3 2 5600.00 1866.6666 3 +South Guangzhou \N 2 2 2950.00 1475.0000 2 +South Shanghai \N 3 2 4300.00 1433.3333 3 + +-- !use_cte_split -- +\N \N \N 12 10 15500.00 1291.6666 12 +\N \N Clothing 5 5 3900.00 780.0000 5 +\N \N Electronics 7 5 11600.00 1657.1428 7 +East \N Clothing 1 1 850.00 850.0000 1 +East \N Electronics 1 1 1300.00 1300.0000 1 +East Nanjing \N 2 2 2150.00 1075.0000 2 +North \N Clothing 2 2 1400.00 700.0000 2 +North \N Electronics 3 2 4700.00 1566.6666 3 +North Beijing \N 3 2 3500.00 1166.6666 3 +North Tianjin \N 2 2 2600.00 1300.0000 2 +South \N Clothing 2 2 1650.00 825.0000 2 +South \N Electronics 3 2 5600.00 1866.6666 3 +South Guangzhou \N 2 2 2950.00 1475.0000 2 +South Shanghai \N 3 2 4300.00 1433.3333 3 + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out index dfde066aefd866..dbc9885c2704c9 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out @@ -40,10 +40,10 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------filter((a.event_id = 'ad_click')) ------------PhysicalOlapScan[com_dd_library] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------filter((cast(experiment_id as DOUBLE) = 37.0)) ------------PhysicalOlapScan[shunt_log_com_dd_library] @@ -57,7 +57,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[com_dd_library] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library] @@ -72,7 +72,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[com_dd_library] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library] @@ -87,7 +87,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[com_dd_library] --------PhysicalOlapScan[shunt_log_com_dd_library] diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out index 9eb9068143011e..7a26912914ec02 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out @@ -40,10 +40,10 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------filter((a.event_id = 'ad_click')) ------------PhysicalOlapScan[com_dd_library_one_side] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------filter((cast(experiment_id as DOUBLE) = 37.0)) ------------PhysicalOlapScan[shunt_log_com_dd_library_one_side] @@ -57,7 +57,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[com_dd_library_one_side] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] @@ -72,7 +72,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[com_dd_library_one_side] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] @@ -87,7 +87,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[com_dd_library_one_side] --------PhysicalOlapScan[shunt_log_com_dd_library_one_side] diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out index 076731ffa2ff77..48f4292fcd760b 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out @@ -109,12 +109,11 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join_1 -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t] -----------PhysicalOlapScan[count_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] --------PhysicalOlapScan[count_t] +------PhysicalOlapScan[count_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -321,12 +320,11 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join_2 -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t] -----------PhysicalOlapScan[count_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] --------PhysicalOlapScan[count_t] +------PhysicalOlapScan[count_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -574,12 +572,11 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join_1 -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t] -----------PhysicalOlapScan[count_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] --------PhysicalOlapScan[count_t] +------PhysicalOlapScan[count_t] Hint log: Used: @@ -911,12 +908,11 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join_2 -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t] -----------PhysicalOlapScan[count_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] --------PhysicalOlapScan[count_t] +------PhysicalOlapScan[count_t] Hint log: Used: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index 74c9c3e281770e..1cbde8708f396c 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -109,12 +109,11 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] +------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -321,12 +320,11 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[count_t_one_side] -----------PhysicalOlapScan[count_t_one_side] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] +------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -405,8 +403,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -459,8 +456,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -487,8 +483,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -516,9 +511,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((count_t_one_side.score > 10)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((count_t_one_side.score > 10)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -545,9 +539,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((count_t_one_side.score > 10)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((count_t_one_side.score > 10)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -561,8 +554,7 @@ PhysicalResultSink ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] ----------PhysicalOlapScan[count_t_one_side] Hint log: @@ -576,8 +568,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -588,16 +579,14 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------------hashAgg[GLOBAL] -----------------hashAgg[LOCAL] -------------------PhysicalOlapScan[count_t_one_side] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------PhysicalOlapScan[count_t_one_side] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------hashAgg[GLOBAL] --------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -612,8 +601,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[count_t_one_side] +--------------PhysicalOlapScan[count_t_one_side] ------------PhysicalOlapScan[count_t_one_side] Hint log: @@ -627,8 +615,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -642,11 +629,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -659,8 +644,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -674,11 +658,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -692,9 +674,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((t1.score > 50)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((t1.score > 50)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -722,8 +703,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[count_t_one_side] +--------------PhysicalOlapScan[count_t_one_side] ------------PhysicalOlapScan[count_t_one_side] Hint log: @@ -737,8 +717,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -752,8 +731,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -782,9 +760,8 @@ PhysicalResultSink --------filter((count_t_one_side.id < 100)) ----------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((count_t_one_side.score > 20) and (t1.id < 100)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((count_t_one_side.score > 20) and (t1.id < 100)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -797,8 +774,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -851,8 +827,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -879,8 +854,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -894,8 +868,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -910,9 +883,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((count_t_one_side.score > 10)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((count_t_one_side.score > 10)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -939,9 +911,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((count_t_one_side.score > 10)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((count_t_one_side.score > 10)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -955,8 +926,7 @@ PhysicalResultSink ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] ----------PhysicalOlapScan[count_t_one_side] Hint log: @@ -967,16 +937,14 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------PhysicalOlapScan[count_t_one_side] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------------hashAgg[GLOBAL] -----------------hashAgg[LOCAL] -------------------PhysicalOlapScan[count_t_one_side] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------PhysicalOlapScan[count_t_one_side] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------hashAgg[GLOBAL] --------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -991,8 +959,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[count_t_one_side] +--------------PhysicalOlapScan[count_t_one_side] ------------PhysicalOlapScan[count_t_one_side] Hint log: @@ -1006,8 +973,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -1021,8 +987,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -1037,9 +1002,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((t1.score > 50)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((t1.score > 50)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -1067,8 +1031,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[count_t_one_side] +--------------PhysicalOlapScan[count_t_one_side] ------------PhysicalOlapScan[count_t_one_side] Hint log: @@ -1082,8 +1045,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] Hint log: @@ -1099,9 +1061,8 @@ PhysicalResultSink --------filter((count_t_one_side.id < 100)) ----------PhysicalOlapScan[count_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((count_t_one_side.score > 20) and (t1.id < 100)) ---------------PhysicalOlapScan[count_t_one_side] +----------filter((count_t_one_side.score > 20) and (t1.id < 100)) +------------PhysicalOlapScan[count_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out index 7cf9813e5b7181..b5d05241012022 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out @@ -225,8 +225,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -279,8 +278,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -307,8 +305,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -336,9 +333,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((max_t.score > 10)) ---------------PhysicalOlapScan[max_t] +----------filter((max_t.score > 10)) +------------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -365,9 +361,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((max_t.score > 10)) ---------------PhysicalOlapScan[max_t] +----------filter((max_t.score > 10)) +------------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -381,9 +376,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((t1.score > 100)) ---------------PhysicalOlapScan[max_t] +----------filter((t1.score > 100)) +------------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -396,8 +390,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -408,16 +401,14 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------PhysicalOlapScan[max_t] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------------hashAgg[GLOBAL] -----------------hashAgg[LOCAL] -------------------PhysicalOlapScan[max_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------PhysicalOlapScan[max_t] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------hashAgg[GLOBAL] --------------PhysicalOlapScan[max_t] +------------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -432,8 +423,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[max_t] +--------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] Hint log: @@ -447,8 +437,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -462,11 +451,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -479,8 +466,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -494,11 +480,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -512,9 +496,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((t1.score > 50)) ---------------PhysicalOlapScan[max_t] +----------filter((t1.score > 50)) +------------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -542,8 +525,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[max_t] +--------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] Hint log: @@ -557,8 +539,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -572,8 +553,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] Hint log: @@ -602,9 +582,8 @@ PhysicalResultSink --------filter((max_t.id < 100)) ----------PhysicalOlapScan[max_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((max_t.score > 20) and (t1.id < 100)) ---------------PhysicalOlapScan[max_t] +----------filter((max_t.score > 20) and (t1.id < 100)) +------------PhysicalOlapScan[max_t] Hint log: Used: use_push_down_agg_through_join_one_side diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out index 18769b1ed1afd7..b5517743ee8d31 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out @@ -225,8 +225,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -279,8 +278,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -307,8 +305,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -335,9 +332,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((min_t.score > 10)) ---------------PhysicalOlapScan[min_t] +----------filter((min_t.score > 10)) +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -364,9 +360,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((min_t.score > 10)) ---------------PhysicalOlapScan[min_t] +----------filter((min_t.score > 10)) +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -381,8 +376,7 @@ PhysicalResultSink ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[min_t] +------------PhysicalOlapScan[min_t] ----------PhysicalOlapScan[min_t] Hint log: @@ -396,8 +390,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -408,16 +401,14 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------------hashAgg[GLOBAL] -----------------hashAgg[LOCAL] -------------------PhysicalOlapScan[min_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------hashAgg[GLOBAL] --------------PhysicalOlapScan[min_t] ---------PhysicalOlapScan[min_t] +------------PhysicalOlapScan[min_t] +------PhysicalOlapScan[min_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -432,8 +423,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[min_t] +--------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] Hint log: @@ -447,8 +437,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -462,11 +451,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -479,8 +466,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -494,11 +480,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] Hint log: Used: use_push_down_agg_through_join_one_side @@ -511,9 +495,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((t1.score > 50)) ---------------PhysicalOlapScan[min_t] +----------filter((t1.score > 50)) +------------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -542,8 +525,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[min_t] +--------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] Hint log: @@ -557,8 +539,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -572,8 +553,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] Hint log: @@ -600,9 +580,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((min_t.score > 20) and (t1.id < 100)) ---------------PhysicalOlapScan[min_t] +----------filter((min_t.score > 20) and (t1.id < 100)) +------------PhysicalOlapScan[min_t] --------filter((min_t.id < 100)) ----------PhysicalOlapScan[min_t] diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out index e933630ef70723..88420a14b13e15 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out @@ -109,12 +109,11 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t] -----------PhysicalOlapScan[sum_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] --------PhysicalOlapScan[sum_t] +------PhysicalOlapScan[sum_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -394,12 +393,11 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t] -----------PhysicalOlapScan[sum_t] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] --------PhysicalOlapScan[sum_t] +------PhysicalOlapScan[sum_t] Hint log: Used: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out index 109b8ab25bfe66..3fce9ec9b1ab94 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out @@ -109,12 +109,11 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[sum_t_one_side] -----------PhysicalOlapScan[sum_t_one_side] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] +------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -225,8 +224,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------filter((t2.score < 100)) ----------PhysicalOlapScan[sum_t_one_side] @@ -280,8 +278,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -308,8 +305,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -337,9 +333,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[sum_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((sum_t_one_side.score > 10)) ---------------PhysicalOlapScan[sum_t_one_side] +----------filter((sum_t_one_side.score > 10)) +------------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -366,9 +361,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[sum_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((sum_t_one_side.score > 10)) ---------------PhysicalOlapScan[sum_t_one_side] +----------filter((sum_t_one_side.score > 10)) +------------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -382,8 +376,7 @@ PhysicalResultSink ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------hashAgg[GLOBAL] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t_one_side] +------------PhysicalOlapScan[sum_t_one_side] ----------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -397,8 +390,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -409,16 +401,14 @@ SyntaxError: -- !with_hint_groupby_pushdown_multi_table_join -- PhysicalResultSink --hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() ---------PhysicalOlapScan[sum_t_one_side] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------------hashAgg[GLOBAL] -----------------hashAgg[LOCAL] -------------------PhysicalOlapScan[sum_t_one_side] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------PhysicalOlapScan[sum_t_one_side] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------hashAgg[GLOBAL] --------------PhysicalOlapScan[sum_t_one_side] +------------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -433,8 +423,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[sum_t_one_side] +--------------PhysicalOlapScan[sum_t_one_side] ------------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -448,8 +437,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -463,11 +451,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -480,8 +466,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -495,11 +480,9 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -513,9 +496,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[sum_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((t1.score > 50)) ---------------PhysicalOlapScan[sum_t_one_side] +----------filter((t1.score > 50)) +------------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side @@ -543,8 +525,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[sum_t_one_side] +--------------PhysicalOlapScan[sum_t_one_side] ------------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -558,8 +539,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -573,8 +553,7 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] Hint log: @@ -603,9 +582,8 @@ PhysicalResultSink --------filter((sum_t_one_side.id < 100)) ----------PhysicalOlapScan[sum_t_one_side] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------filter((sum_t_one_side.score > 20) and (t1.id < 100)) ---------------PhysicalOlapScan[sum_t_one_side] +----------filter((sum_t_one_side.score > 20) and (t1.id < 100)) +------------PhysicalOlapScan[sum_t_one_side] Hint log: Used: use_push_down_agg_through_join_one_side diff --git a/regression-test/data/nereids_rules_p0/eliminate_aggregate_casewhen/eliminate_aggregate_casewhen.out b/regression-test/data/nereids_rules_p0/eliminate_aggregate_casewhen/eliminate_aggregate_casewhen.out index 46bfb749845d7a..8205077a801fb5 100644 --- a/regression-test/data/nereids_rules_p0/eliminate_aggregate_casewhen/eliminate_aggregate_casewhen.out +++ b/regression-test/data/nereids_rules_p0/eliminate_aggregate_casewhen/eliminate_aggregate_casewhen.out @@ -17,7 +17,7 @@ PhysicalResultSink -- !basic_3 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t1] @@ -39,7 +39,7 @@ PhysicalResultSink -- !basic_6 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t1] @@ -61,7 +61,7 @@ PhysicalResultSink -- !basic_2_3 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t2] @@ -83,7 +83,7 @@ PhysicalResultSink -- !basic_2_6 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t2] @@ -105,7 +105,7 @@ PhysicalResultSink -- !basic_3_3 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t3] @@ -127,7 +127,7 @@ PhysicalResultSink -- !basic_3_6 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t3] @@ -149,7 +149,7 @@ PhysicalResultSink -- !basic_4_4 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t4] @@ -171,7 +171,7 @@ PhysicalResultSink -- !basic_4_6 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----PhysicalProject ------PhysicalOlapScan[t4] diff --git a/regression-test/data/nereids_rules_p0/eliminate_aggregate_constant/eliminate_aggregate_constant.out b/regression-test/data/nereids_rules_p0/eliminate_aggregate_constant/eliminate_aggregate_constant.out index 3e1ee7196118a6..9321daf78652ca 100644 --- a/regression-test/data/nereids_rules_p0/eliminate_aggregate_constant/eliminate_aggregate_constant.out +++ b/regression-test/data/nereids_rules_p0/eliminate_aggregate_constant/eliminate_aggregate_constant.out @@ -2,7 +2,7 @@ -- !basic_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -16,7 +16,7 @@ PhysicalResultSink -- !basic_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -30,7 +30,7 @@ PhysicalResultSink -- !basic_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -44,7 +44,7 @@ PhysicalResultSink -- !basic_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -58,7 +58,7 @@ PhysicalResultSink -- !basic_2_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -72,7 +72,7 @@ PhysicalResultSink -- !basic_2_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -86,7 +86,7 @@ PhysicalResultSink -- !basic_2_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -100,7 +100,7 @@ PhysicalResultSink -- !basic_2_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -114,7 +114,7 @@ PhysicalResultSink -- !basic_3_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -128,7 +128,7 @@ PhysicalResultSink -- !basic_3_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -142,7 +142,7 @@ PhysicalResultSink -- !basic_3_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -156,7 +156,7 @@ PhysicalResultSink -- !basic_3_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -170,7 +170,7 @@ PhysicalResultSink -- !basic_4_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -184,7 +184,7 @@ PhysicalResultSink -- !basic_4_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -198,7 +198,7 @@ PhysicalResultSink -- !basic_4_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -212,7 +212,7 @@ PhysicalResultSink -- !basic_4_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -226,7 +226,7 @@ PhysicalResultSink -- !basic_add_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -241,7 +241,7 @@ PhysicalResultSink -- !basic_add_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -256,7 +256,7 @@ PhysicalResultSink -- !basic_add_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -271,7 +271,7 @@ PhysicalResultSink -- !basic_add_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -286,7 +286,7 @@ PhysicalResultSink -- !basic_add_2_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -301,7 +301,7 @@ PhysicalResultSink -- !basic_add_2_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -316,7 +316,7 @@ PhysicalResultSink -- !basic_add_2_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -331,7 +331,7 @@ PhysicalResultSink -- !basic_add_2_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -346,7 +346,7 @@ PhysicalResultSink -- !basic_add_3_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -361,7 +361,7 @@ PhysicalResultSink -- !basic_add_3_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -376,7 +376,7 @@ PhysicalResultSink -- !basic_add_3_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -391,7 +391,7 @@ PhysicalResultSink -- !basic_add_3_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -406,7 +406,7 @@ PhysicalResultSink -- !basic_add_4_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -421,7 +421,7 @@ PhysicalResultSink -- !basic_add_4_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -436,7 +436,7 @@ PhysicalResultSink -- !basic_add_4_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -451,7 +451,7 @@ PhysicalResultSink -- !basic_add_4_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -466,7 +466,7 @@ PhysicalResultSink -- !add_sum_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -481,7 +481,7 @@ PhysicalResultSink -- !add_sum_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -496,7 +496,7 @@ PhysicalResultSink -- !add_sum_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -511,7 +511,7 @@ PhysicalResultSink -- !add_sum_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t1] @@ -526,7 +526,7 @@ PhysicalResultSink -- !add_sum_2_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -541,7 +541,7 @@ PhysicalResultSink -- !add_sum_2_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -556,7 +556,7 @@ PhysicalResultSink -- !add_sum_2_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -571,7 +571,7 @@ PhysicalResultSink -- !add_sum_2_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t2] @@ -586,7 +586,7 @@ PhysicalResultSink -- !add_sum_3_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -601,7 +601,7 @@ PhysicalResultSink -- !add_sum_3_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -616,7 +616,7 @@ PhysicalResultSink -- !add_sum_3_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -631,7 +631,7 @@ PhysicalResultSink -- !add_sum_3_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t3] @@ -646,7 +646,7 @@ PhysicalResultSink -- !add_sum_4_1 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -661,7 +661,7 @@ PhysicalResultSink -- !add_sum_4_3 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -676,7 +676,7 @@ PhysicalResultSink -- !add_sum_4_5 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] @@ -691,7 +691,7 @@ PhysicalResultSink -- !add_sum_4_7 -- PhysicalResultSink --PhysicalProject -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------PhysicalOlapScan[t4] diff --git a/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out b/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out index 877492a2eed6ad..60fa2fe2c30d9f 100644 --- a/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out +++ b/regression-test/data/nereids_rules_p0/eliminate_not_null/eliminate_not_null.out @@ -21,7 +21,7 @@ PhysicalResultSink -- !eliminate_not_null_aggregate -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----filter((t.score > 0)) ------PhysicalOlapScan[t] diff --git a/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out b/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out index 5e9e62a74eb546..68d116d02ade20 100644 --- a/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out +++ b/regression-test/data/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.out @@ -160,7 +160,7 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject ------filter((count(id) > 1)) ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalProject --------------PhysicalOlapScan[t] @@ -215,7 +215,7 @@ PhysicalResultSink -- !right_outer_aggregate -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject --------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((t1.id = t2.id)) otherCondition=() ----------PhysicalProject diff --git a/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out b/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out index d823a843eb2bfd..f543d077dde4e1 100644 --- a/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out +++ b/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out @@ -229,7 +229,7 @@ PhysicalResultSink -- !filter_aggregation_group_set -- PhysicalResultSink --filter((cast(msg as DOUBLE) = 1.0)) -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalRepeat --------filter((t1.id > 10)) ----------PhysicalOlapScan[t1] @@ -237,7 +237,7 @@ PhysicalResultSink -- !filter_aggregation_group_set -- PhysicalResultSink --filter(OR[(t1.id > 10),(cast(msg as DOUBLE) = 1.0)]) -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalRepeat --------PhysicalOlapScan[t1] @@ -360,13 +360,6 @@ PhysicalResultSink ------filter((t1.id <= 5)) --------PhysicalOlapScan[t1] --- !filter_window_row_number_complex_predicate -- -PhysicalResultSink ---PhysicalWindow -----PhysicalQuickSort[LOCAL_SORT] -------filter(((cast(id as DOUBLE) + cast(msg as DOUBLE)) = cast('' as DOUBLE))) ---------PhysicalOlapScan[t1] - -- !filter_multi_window -- PhysicalResultSink --PhysicalWindow diff --git a/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out b/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out index d145061b1c1366..73e7df6bfa6986 100644 --- a/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out +++ b/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out @@ -52,7 +52,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------filter((t1.id > 100)) ----------PhysicalOlapScan[t1] @@ -313,12 +313,12 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[t2] @@ -330,10 +330,10 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t2] Hint log: @@ -349,12 +349,12 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------filter((t1.score > 10)) ------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------filter((t2.name = 'Test')) ----------------------PhysicalOlapScan[t2] @@ -372,13 +372,13 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -391,10 +391,10 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalExcept ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[t1] ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[t2] Hint log: @@ -406,7 +406,7 @@ SyntaxError: PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------filter((t1.id > 100)) ----------PhysicalOlapScan[t1] @@ -420,10 +420,10 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalIntersect ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[t1] ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[t2] Hint log: @@ -458,23 +458,23 @@ SyntaxError: PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalIntersect -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalExcept ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalUnion ------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------hashAgg[LOCAL] +--------------------hashAgg[GLOBAL] ----------------------PhysicalOlapScan[t1] ------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------hashAgg[LOCAL] +--------------------hashAgg[GLOBAL] ----------------------PhysicalOlapScan[t2] ----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] +------------hashAgg[GLOBAL] --------------PhysicalOlapScan[t3] ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[t4] Hint log: @@ -490,14 +490,14 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((t1.id = t2.id)) otherCondition=() --------------------PhysicalOlapScan[t1] --------------------PhysicalProject ----------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -513,11 +513,11 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------filter((t1.score > 10)) ------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------filter((t2.score < 5)) ------------------PhysicalOlapScan[t2] @@ -534,16 +534,16 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t4] Hint log: @@ -559,7 +559,7 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t1] ------------hashAgg[GLOBAL] --------------PhysicalDistribute[DistributionSpecHash] @@ -582,14 +582,14 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((t1.id = t2.id)) otherCondition=() --------------------PhysicalOlapScan[t1] --------------------PhysicalProject ----------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -605,14 +605,14 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[LEFT_OUTER_JOIN broadcast] hashCondition=((t1.id = t2.id)) otherCondition=() --------------------PhysicalOlapScan[t1] --------------------PhysicalProject ----------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -636,7 +636,7 @@ PhysicalResultSink ------------------------PhysicalOlapScan[t2] ----------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -660,7 +660,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -676,13 +676,13 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((t1.id = t2.id)) otherCondition=() ------------------PhysicalOlapScan[t1] ------------------PhysicalProject --------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t3] Hint log: @@ -695,10 +695,10 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalExcept ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalOlapScan[t1] ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------filter((t2.score > 10)) ------------PhysicalOlapScan[t2] @@ -773,11 +773,11 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalExcept ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------PhysicalOlapScan[t1] ------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------filter((t2.score > 20)) --------------PhysicalOlapScan[t2] @@ -796,12 +796,12 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[t2] @@ -883,7 +883,7 @@ SyntaxError: PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------filter((t1.id > 100)) ----------PhysicalOlapScan[t1] @@ -1234,12 +1234,12 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecExecutionAny] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[t2] diff --git a/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out b/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out index 20f72b3d6a18aa..fbed0a9f889fab 100644 --- a/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out +++ b/regression-test/data/nereids_rules_p0/limit_push_down/order_push_down.out @@ -94,21 +94,21 @@ PhysicalResultSink PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalTopN[LOCAL_SORT] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalOlapScan[t1] -- !limit_offset_agg_having -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalTopN[LOCAL_SORT] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalOlapScan[t1] -- !limit_offset_sort_agg_having -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalTopN[LOCAL_SORT] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalOlapScan[t1] -- !limit_distinct -- @@ -580,7 +580,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalTopN[LOCAL_SORT] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalOlapScan[t1] -- !limit_cross_join -- @@ -661,7 +661,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalTopN[LOCAL_SORT] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------hashJoin[INNER_JOIN] hashCondition=((subq2.id = t3.id)) otherCondition=() ----------hashJoin[INNER_JOIN] hashCondition=((subq2.id = t2.id)) otherCondition=() ------------PhysicalOlapScan[t1] @@ -672,7 +672,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalTopN[LOCAL_SORT] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------hashJoin[INNER_JOIN] hashCondition=((subq2.id = t3.id)) otherCondition=() ----------hashJoin[INNER_JOIN] hashCondition=((subq2.id = t2.id)) otherCondition=() ------------filter((t1.id > 100)) diff --git a/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out b/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out index b85ca62f7ffade..9a1df01358d918 100644 --- a/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out +++ b/regression-test/data/nereids_rules_p0/max_min_filter_push_down/max_min_filter_push_down.out @@ -3,8 +3,7 @@ PhysicalResultSink --filter((min(value1) < 20)) ----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------PhysicalEmptyRelation +------PhysicalEmptyRelation -- !min -- PhysicalResultSink diff --git a/regression-test/data/nereids_rules_p0/merge_aggregate/merge_aggregate.out b/regression-test/data/nereids_rules_p0/merge_aggregate/merge_aggregate.out index d7103bfed9f686..51002fa7feb92c 100644 --- a/regression-test/data/nereids_rules_p0/merge_aggregate/merge_aggregate.out +++ b/regression-test/data/nereids_rules_p0/merge_aggregate/merge_aggregate.out @@ -116,7 +116,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashAgg[LOCAL] +----------hashAgg[GLOBAL] ------------PhysicalOlapScan[mal_test1] -- !inner_agg_has_distinct_same_keys_shape -- @@ -125,10 +125,9 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashAgg[DISTINCT_LOCAL] +----------hashAgg[GLOBAL] ------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[mal_test1] +--------------PhysicalOlapScan[mal_test1] -- !middle_project_has_expression_cannot_merge_shape1 -- PhysicalResultSink @@ -140,7 +139,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------PhysicalOlapScan[mal_test1] @@ -154,7 +153,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalOlapScan[mal_test1] -- !maxGroupKey_minGroupKey_sumGroupKey_cannot_merge_shape -- @@ -164,7 +163,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] -------------hashAgg[LOCAL] +------------hashAgg[GLOBAL] --------------PhysicalProject ----------------PhysicalOlapScan[mal_test1] @@ -176,7 +175,7 @@ PhysicalResultSink --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[mal_test1] -- !group_key_not_contain_cannot_merge_shape -- @@ -187,7 +186,7 @@ PhysicalResultSink --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[mal_test1] -- !outer_agg_has_distinct_cannot_merge_shape -- @@ -197,11 +196,12 @@ PhysicalResultSink ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject ----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashAgg[LOCAL] ---------------------PhysicalOlapScan[mal_test1] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalOlapScan[mal_test1] -- !inner_agg_has_distinct_cannot_merge_shape -- PhysicalResultSink @@ -213,10 +213,9 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[DISTINCT_LOCAL] +------------------hashAgg[GLOBAL] --------------------hashAgg[GLOBAL] -----------------------hashAgg[LOCAL] -------------------------PhysicalOlapScan[mal_test1] +----------------------PhysicalOlapScan[mal_test1] -- !agg_with_expr_cannot_merge_shape1 -- PhysicalResultSink @@ -228,7 +227,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------PhysicalOlapScan[mal_test1] @@ -242,7 +241,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------PhysicalOlapScan[mal_test1] @@ -256,10 +255,11 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------PhysicalOlapScan[mal_test_merge_agg] +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------PhysicalOlapScan[mal_test_merge_agg] -- !test_distinct_expr_transform -- -1 @@ -282,10 +282,11 @@ PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject ------hashAgg[GLOBAL] ---------PhysicalDistribute[DistributionSpecHash] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalOlapScan[mal_test_merge_agg] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------PhysicalOlapScan[mal_test_merge_agg] -- !test_sum_empty_table -- \N \N \N diff --git a/regression-test/data/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.out b/regression-test/data/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.out index 56585d5a214741..57003fed91173c 100644 --- a/regression-test/data/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.out +++ b/regression-test/data/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.out @@ -10,7 +10,7 @@ -- !shape2_0_after -- PhysicalResultSink ---hashAgg[DISTINCT_LOCAL] +--hashAgg[DISTINCT_GLOBAL] ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------filter((mv2_0.o_orderkey = 1) and (mv2_0.o_orderstatus = 'o')) diff --git a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out index 5e28fcf402fed5..2915904c46fecf 100644 --- a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out +++ b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out @@ -66,7 +66,7 @@ with_pk_col PhysicalResultSink --hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt_not_null.fk)) otherCondition=() ----PhysicalOlapScan[pkt] -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalOlapScan[fkt_not_null] -- !res -- @@ -226,7 +226,7 @@ with_pk_col PhysicalResultSink --hashJoin[INNER_JOIN] hashCondition=((pkt.pk = fkt.fk)) otherCondition=() ----PhysicalOlapScan[pkt] -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalOlapScan[fkt] -- !res -- diff --git a/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out b/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out index 0a9c4293a9d17f..e0ff329e8e46d5 100644 --- a/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out +++ b/regression-test/data/nereids_rules_p0/predicate_infer/infer_predicate.out @@ -119,7 +119,7 @@ PhysicalResultSink -- !infer_predicate_with_aggregate -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------filter((t1.id > 70)) --------PhysicalOlapScan[t] @@ -170,7 +170,7 @@ PhysicalResultSink -- !infer_predicate_join_with_aggregate_having -- PhysicalResultSink --filter((sum(score) > 140)) -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[t] --------PhysicalOlapScan[t] @@ -186,7 +186,7 @@ PhysicalResultSink -- !infer_predicate_join_with_distinct -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------filter((t1.score > 160)) --------PhysicalOlapScan[t] @@ -219,7 +219,7 @@ PhysicalResultSink -- !infer_predicate_aggregate_subquery -- PhysicalResultSink --filter((t_agg.total > 30)) -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalOlapScan[t] -- !infer_predicate_join_with_function -- @@ -250,7 +250,7 @@ PhysicalResultSink -- !infer_predicate_multi_join_subquery_aggregate -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.id = t.id)) otherCondition=() ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() @@ -283,7 +283,7 @@ PhysicalResultSink -- !infer_predicate_multi_join_with_having_clause -- PhysicalResultSink --filter((sum(score) > 150)) -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------PhysicalOlapScan[t] diff --git a/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out b/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out index e33d0033967767..d86d0cdc082bad 100644 --- a/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out +++ b/regression-test/data/nereids_rules_p0/pull_up_join_from_union/pull_up_join_from_union.out @@ -72,9 +72,9 @@ PhysicalResultSink PhysicalResultSink --hashJoin[INNER_JOIN] hashCondition=((id = id)) otherCondition=() ----PhysicalUnion -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalOlapScan[table_b] -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalOlapScan[table_c] ----PhysicalOlapScan[table_a] diff --git a/regression-test/data/nereids_rules_p0/push_count_into_union_all/push_count_into_union_all.out b/regression-test/data/nereids_rules_p0/push_count_into_union_all/push_count_into_union_all.out index cfdef2ebfec5a2..5e094079a73ac9 100644 --- a/regression-test/data/nereids_rules_p0/push_count_into_union_all/push_count_into_union_all.out +++ b/regression-test/data/nereids_rules_p0/push_count_into_union_all/push_count_into_union_all.out @@ -180,13 +180,13 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------PhysicalUnion -------------hashAgg[LOCAL] +------------hashAgg[GLOBAL] --------------filter((mal_test_push_count.a > 1)) ----------------PhysicalOlapScan[mal_test_push_count] -------------hashAgg[LOCAL] +------------hashAgg[GLOBAL] --------------filter((mal_test_push_count.a < 100)) ----------------PhysicalOlapScan[mal_test_push_count] -------------hashAgg[LOCAL] +------------hashAgg[GLOBAL] --------------filter((mal_test_push_count.a = 1)) ----------------PhysicalOlapScan[mal_test_push_count] diff --git a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out index 4c5c60acd31a8e..f205e7bbfdab66 100644 --- a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !basic_not -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------PhysicalOlapScan[t] ------PhysicalOlapScan[t] @@ -13,9 +13,9 @@ SyntaxError: -- !basic -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------PhysicalOlapScan[t] ----------PhysicalOlapScan[t] diff --git a/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out index 9ffe9520387501..9ac0d2a4732bc7 100644 --- a/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct_through_join.out @@ -7,9 +7,9 @@ PhysicalResultSink --------hashAgg[LOCAL] ----------PhysicalUnion ------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t] ------------PhysicalLimit[LOCAL] ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalOlapScan[t] diff --git a/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.out index ef877b24e97317..c5f90edd139ce5 100644 --- a/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table_join] ------------PhysicalOlapScan[table_join] @@ -31,7 +31,7 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table_join] ------------PhysicalStorageLayerAggregate[table_join] diff --git a/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_union.out b/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_union.out index 5874800e24f406..fdbc9f780efc3b 100644 --- a/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_union.out +++ b/regression-test/data/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_union.out @@ -8,11 +8,11 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] -- !push_down_topn_union_with_conditions -- @@ -24,17 +24,17 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------filter((t1.score > 10)) --------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------filter((t2.name = 'Test')) --------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------filter((t3.id < 5)) --------------------PhysicalOlapScan[table2] @@ -47,15 +47,15 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] -- !push_down_topn_nested_union -- @@ -67,19 +67,19 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] -- !push_down_topn_union_after_join -- @@ -91,13 +91,13 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------------------PhysicalOlapScan[table2] --------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] -- !push_down_topn_union_different_projections -- @@ -127,12 +127,12 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------filter((table2.score > 20)) --------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalOlapScan[table2] -- !push_down_topn_union_with_limit -- @@ -166,12 +166,12 @@ PhysicalResultSink ----------PhysicalUnion ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------filter((t1.name = 'Test') and (t1.score > 10)) --------------------PhysicalOlapScan[table2] ------------PhysicalTopN[MERGE_SORT] --------------PhysicalTopN[LOCAL_SORT] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------filter((t2.id < 5) and (t2.score < 20)) --------------------PhysicalOlapScan[table2] diff --git a/regression-test/data/nereids_rules_p0/simplify_window_expression/simplify_window_expression.out b/regression-test/data/nereids_rules_p0/simplify_window_expression/simplify_window_expression.out index 7c72e1c31e70a0..0356836ffad9da 100644 --- a/regression-test/data/nereids_rules_p0/simplify_window_expression/simplify_window_expression.out +++ b/regression-test/data/nereids_rules_p0/simplify_window_expression/simplify_window_expression.out @@ -260,7 +260,7 @@ PhysicalResultSink PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject -------hashAgg[LOCAL] +------hashAgg[GLOBAL] --------PhysicalProject ----------filter((mal_test_simplify_window.__DORIS_DELETE_SIGN__ = 0)) ------------PhysicalOlapScan[mal_test_simplify_window] diff --git a/regression-test/data/nereids_rules_p0/test_load/test1.out b/regression-test/data/nereids_rules_p0/test_load/test1.out new file mode 100644 index 00000000000000..08ec59eef9ea10 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/test_load/test1.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_count_col -- +\N 0 0 +1 1 1 +1 1 1 +2 1 1 +2 1 1 +2 1 1 +3 1 1 +3 1 1 +4 1 1 +6 1 1 +6 1 1 + +-- !select_rank -- +\N 1 1 +1 1 1 +1 1 1 +2 1 1 +2 1 1 +2 1 1 +3 1 1 +3 1 1 +4 1 1 +6 1 1 +6 1 1 + diff --git a/regression-test/data/nereids_rules_p0/test_load/test2.out b/regression-test/data/nereids_rules_p0/test_load/test2.out new file mode 100644 index 00000000000000..08ec59eef9ea10 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/test_load/test2.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_count_col -- +\N 0 0 +1 1 1 +1 1 1 +2 1 1 +2 1 1 +2 1 1 +3 1 1 +3 1 1 +4 1 1 +6 1 1 +6 1 1 + +-- !select_rank -- +\N 1 1 +1 1 1 +1 1 1 +2 1 1 +2 1 1 +2 1 1 +3 1 1 +3 1 1 +4 1 1 +6 1 1 +6 1 1 + diff --git a/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out b/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out index f1002f9599dc49..03cb3ea3d428ee 100644 --- a/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out +++ b/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !groupby_positive_case -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[LEFT_SEMI_JOIN] hashCondition=((T3.a = T2.a)) otherCondition=() ------filter((T1.__DORIS_DELETE_SIGN__ = 0)) --------PhysicalOlapScan[T1] @@ -11,7 +11,7 @@ PhysicalResultSink -- !groupby_negative_case -- PhysicalResultSink --hashJoin[LEFT_SEMI_JOIN] hashCondition=((T3.D = expr_cast(a as BIGINT))) otherCondition=() -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------filter((T1.__DORIS_DELETE_SIGN__ = 0)) --------PhysicalOlapScan[T1] ----filter((T2.__DORIS_DELETE_SIGN__ = 0)) @@ -19,7 +19,7 @@ PhysicalResultSink -- !grouping_positive_case -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[LEFT_SEMI_JOIN] hashCondition=((T3.a = T2.a)) otherCondition=() ------PhysicalRepeat --------filter((T1.__DORIS_DELETE_SIGN__ = 0)) @@ -40,7 +40,7 @@ PhysicalResultSink -- !groupby_positive_case2 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[LEFT_SEMI_JOIN] hashCondition=((T3.a = T2.a)) otherCondition=() ------filter((T1.__DORIS_DELETE_SIGN__ = 0)) --------PhysicalOlapScan[T1] @@ -50,7 +50,7 @@ PhysicalResultSink -- !groupby_negative_case2 -- PhysicalResultSink --hashJoin[LEFT_SEMI_JOIN] hashCondition=((T3.D = expr_cast(a as BIGINT))) otherCondition=() -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------filter((T1.__DORIS_DELETE_SIGN__ = 0)) --------PhysicalOlapScan[T1] ----filter((T2.__DORIS_DELETE_SIGN__ = 0)) @@ -58,7 +58,7 @@ PhysicalResultSink -- !grouping_positive_case2 -- PhysicalResultSink ---hashAgg[LOCAL] +--hashAgg[GLOBAL] ----hashJoin[LEFT_SEMI_JOIN] hashCondition=((T3.a = T2.a)) otherCondition=() ------PhysicalRepeat --------filter((T1.__DORIS_DELETE_SIGN__ = 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 b57fe2df5cb445..b0c1d9ed83b2fe 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 @@ -49,9 +49,8 @@ PhysicalResultSink ----PhysicalWindow ------PhysicalProject[random() AS `random()`, random() AS `random()`, sum(random()), sum(random())] --------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalProject[random() AS `random()`, random() AS `random()`] ---------------PhysicalOneRowRelation +----------PhysicalProject[random() AS `random()`, random() AS `random()`] +------------PhysicalOneRowRelation -- !check_equal_project_to_agg_1_shape -- PhysicalResultSink @@ -96,9 +95,8 @@ PhysicalResultSink PhysicalResultSink --PhysicalProject[random() AS `random()`, random() AS `random()`, sum(random()), sum(random())] ----hashAgg[GLOBAL] -------hashAgg[LOCAL] ---------PhysicalProject[random() AS `random()`, random() AS `random()`] -----------PhysicalOneRowRelation +------PhysicalProject[random() AS `random()`, random() AS `random()`] +--------PhysicalOneRowRelation -- !check_equal_distinct_to_agg_3_shape -- PhysicalResultSink diff --git a/regression-test/data/nereids_rules_p0/unique_function/push_down_filter_through_agg_with_unique_function.out b/regression-test/data/nereids_rules_p0/unique_function/push_down_filter_through_agg_with_unique_function.out index 9f31c0faa33276..94f7250b53f8b4 100644 --- a/regression-test/data/nereids_rules_p0/unique_function/push_down_filter_through_agg_with_unique_function.out +++ b/regression-test/data/nereids_rules_p0/unique_function/push_down_filter_through_agg_with_unique_function.out @@ -2,7 +2,7 @@ -- !push_down_filter_through_agg -- PhysicalResultSink --filter(((cast(id as BIGINT) + random(1, 10)) > 6)) -----hashAgg[LOCAL] +----hashAgg[GLOBAL] ------PhysicalProject[t1.id] --------filter(((cast(id as BIGINT) + random(1, 10)) > 5)) ----------PhysicalOlapScan[t1] diff --git a/regression-test/data/nereids_syntax_p0/agg_4_phase.out b/regression-test/data/nereids_syntax_p0/agg_4_phase.out index a939738ea183d5..2d732f0c95d5e9 100644 --- a/regression-test/data/nereids_syntax_p0/agg_4_phase.out +++ b/regression-test/data/nereids_syntax_p0/agg_4_phase.out @@ -1,6 +1,6 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !4phase -- -3 +3 8 -- !phase4_multi_distinct -- 1 -10,-10 1 a 1 diff --git a/regression-test/data/nereids_syntax_p0/aggregate_strategies.out b/regression-test/data/nereids_syntax_p0/aggregate_strategies.out index 2aed31af87d401..d53cea4e6579af 100644 --- a/regression-test/data/nereids_syntax_p0/aggregate_strategies.out +++ b/regression-test/data/nereids_syntax_p0/aggregate_strategies.out @@ -248,10 +248,3 @@ name_4 1 4 1 1 --- !sql_distinct_same_col2 -- -1 -1 -1 -1 -1 - diff --git a/regression-test/data/nereids_syntax_p0/mv/newMv/single_slot.out b/regression-test/data/nereids_syntax_p0/mv/newMv/single_slot.out index 16fe69c3ff6a8f..1fa5b13b3fe126 100644 --- a/regression-test/data/nereids_syntax_p0/mv/newMv/single_slot.out +++ b/regression-test/data/nereids_syntax_p0/mv/newMv/single_slot.out @@ -1,13 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select_star -- --4 -4 -4 d -1 1 1 a -2 2 2 b -3 -3 \N c +1 2 1 a +1 3 2 b +2 -4 -4 d +2 5 \N c -- !select_mv -- -2 2 -3 3 -4 2 -5 3 +2 7 +3 9 diff --git a/regression-test/data/shape_check/clickbench/query10.out b/regression-test/data/shape_check/clickbench/query10.out index c784056436912a..ae9174ce1c1551 100644 --- a/regression-test/data/shape_check/clickbench/query10.out +++ b/regression-test/data/shape_check/clickbench/query10.out @@ -4,9 +4,10 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] +--------hashAgg[DISTINCT_GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalOlapScan[hits] +------------hashAgg[DISTINCT_LOCAL] +--------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query11.out b/regression-test/data/shape_check/clickbench/query11.out index 4b5e4486d3f4cc..856e55187f92a7 100644 --- a/regression-test/data/shape_check/clickbench/query11.out +++ b/regression-test/data/shape_check/clickbench/query11.out @@ -7,7 +7,8 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(( not (MobilePhoneModel = ''))) -------------------PhysicalOlapScan[hits] +--------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------filter(( not (MobilePhoneModel = ''))) +--------------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query12.out b/regression-test/data/shape_check/clickbench/query12.out index 10928363a83c02..d47a7e129e39bd 100644 --- a/regression-test/data/shape_check/clickbench/query12.out +++ b/regression-test/data/shape_check/clickbench/query12.out @@ -7,7 +7,8 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(( not (MobilePhoneModel = ''))) -------------------PhysicalOlapScan[hits] +--------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------filter(( not (MobilePhoneModel = ''))) +--------------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query14.out b/regression-test/data/shape_check/clickbench/query14.out index 35eedce41b927a..54afcc6268ce39 100644 --- a/regression-test/data/shape_check/clickbench/query14.out +++ b/regression-test/data/shape_check/clickbench/query14.out @@ -7,7 +7,8 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(( not (SearchPhrase = ''))) -------------------PhysicalOlapScan[hits] +--------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------filter(( not (SearchPhrase = ''))) +--------------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query16.out b/regression-test/data/shape_check/clickbench/query16.out index a229f5310dfc2d..b1ebc087fdf6c6 100644 --- a/regression-test/data/shape_check/clickbench/query16.out +++ b/regression-test/data/shape_check/clickbench/query16.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query17.out b/regression-test/data/shape_check/clickbench/query17.out index 78635481d04652..e5c1dcd3922419 100644 --- a/regression-test/data/shape_check/clickbench/query17.out +++ b/regression-test/data/shape_check/clickbench/query17.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query18.out b/regression-test/data/shape_check/clickbench/query18.out index 6af4a027d886c3..05c32be0dead95 100644 --- a/regression-test/data/shape_check/clickbench/query18.out +++ b/regression-test/data/shape_check/clickbench/query18.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query19.out b/regression-test/data/shape_check/clickbench/query19.out index 7540225b393218..bd06fbe318f798 100644 --- a/regression-test/data/shape_check/clickbench/query19.out +++ b/regression-test/data/shape_check/clickbench/query19.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query23.out b/regression-test/data/shape_check/clickbench/query23.out index 76a91b3ad49968..5c6ed877934180 100644 --- a/regression-test/data/shape_check/clickbench/query23.out +++ b/regression-test/data/shape_check/clickbench/query23.out @@ -7,7 +7,8 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(( not (SearchPhrase = '')) and ( not (URL like '%.google.%')) and (Title like '%Google%')) -------------------PhysicalOlapScan[hits] +--------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------filter(( not (SearchPhrase = '')) and ( not (URL like '%.google.%')) and (Title like '%Google%')) +--------------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query5.out b/regression-test/data/shape_check/clickbench/query5.out index 94b8f2ad28f023..5d7de093868c4c 100644 --- a/regression-test/data/shape_check/clickbench/query5.out +++ b/regression-test/data/shape_check/clickbench/query5.out @@ -1,11 +1,9 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ckbench_shape_5 -- PhysicalResultSink ---hashAgg[DISTINCT_GLOBAL] +--hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] -------hashAgg[DISTINCT_LOCAL] ---------hashAgg[GLOBAL] -----------hashAgg[LOCAL] -------------PhysicalProject ---------------PhysicalOlapScan[hits] +------hashAgg[GLOBAL] +--------PhysicalProject +----------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/clickbench/query9.out b/regression-test/data/shape_check/clickbench/query9.out index dcece9f0ce72d7..b35cb2e2a808e9 100644 --- a/regression-test/data/shape_check/clickbench/query9.out +++ b/regression-test/data/shape_check/clickbench/query9.out @@ -7,6 +7,7 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalOlapScan[hits] +--------------hashAgg[GLOBAL] +----------------PhysicalProject +------------------PhysicalOlapScan[hits] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query12.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query12.out index b4c126ae67aebf..6d9e989b19298d 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query12.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query12.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[ws_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Books', 'Men', 'Sports')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1998-05-06') and (date_dim.d_date >= '1998-04-06')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[ws_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Men', 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1998-05-06') and (date_dim.d_date >= '1998-04-06')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query20.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query20.out index fa360d9c6fdc83..3d0b0590dab75e 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query20.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query20.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Books', 'Shoes', 'Women')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-02-25') and (date_dim.d_date >= '2002-01-26')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Shoes', 'Women')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2002-02-25') and (date_dim.d_date >= '2002-01-26')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query28.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query28.out index 7a6bdd8868ef00..7a488abb6dabd0 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query28.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query34.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query34.out index 15e7650bae15c3..a12df581f106ae 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query34.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query34.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query38.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query38.out index 550170a1536437..5a103e684ebdcb 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query38.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query38.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalIntersect RFV2: RF6[c_last_name->c_last_name] RF7[c_last_name->c_last_name] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -35,7 +35,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] RFV2: RF6 ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query46.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query46.out index 48e2eae68f5c61..26ac77a46ec8ac 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query46.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query46.out @@ -7,19 +7,23 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=(( not (ca_city = bought_city))) ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------PhysicalProject -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF3 hd_demo_sk->[ss_hdemo_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] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer_address] ------------------------------------PhysicalProject --------------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] @@ -29,8 +33,6 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject ------------------PhysicalOlapScan[customer] ------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query47.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query47.out index 5995dc531259c5..c04dc536e6bbef 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query47.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalProject +--------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query51.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query51.out index 11fecfc9825a58..9b4742d60618ee 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query51.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query51.out @@ -14,29 +14,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query53.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query53.out index 89dc632eb527c4..6cc3c447d13449 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query53.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query53.out @@ -8,25 +8,24 @@ PhysicalResultSink ----------filter((if((avg_quarterly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_quarterly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_quarterly_sales), NULL) > 0.100000)) ------------PhysicalWindow --------------PhysicalQuickSort[LOCAL_SORT] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------PhysicalProject -----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +----------------PhysicalProject +------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +------------------------------------PhysicalProject +--------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query54.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query54.out index e612687075e9b6..46ff85eba23a08 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query54.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query54.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= d_month_seq+3) ------------------------PhysicalProject @@ -25,7 +25,7 @@ PhysicalResultSink --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 --------------------------------------------PhysicalProject -----------------------------------------------hashAgg[LOCAL] +----------------------------------------------hashAgg[GLOBAL] ------------------------------------------------PhysicalProject --------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() ----------------------------------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query57.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query57.out index 96d8f68090e5de..12bdb0c331739c 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query57.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalProject +--------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[call_center] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query63.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query63.out index 9653f6c52199aa..94e35cb7458980 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query63.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query63.out @@ -8,25 +8,24 @@ PhysicalResultSink ----------filter((if((avg_monthly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) ------------PhysicalWindow --------------PhysicalQuickSort[LOCAL_SORT] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------PhysicalProject -----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +----------------PhysicalProject +------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +------------------------------------PhysicalProject +--------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query64.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query64.out index 2cff316c5a22aa..80addfcc7ff5f6 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query64.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query64.out @@ -27,13 +27,13 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------PhysicalProject --------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() +------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() --------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() --------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------------------------------------------------------------------PhysicalProject @@ -41,7 +41,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------------------------------PhysicalProject ------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() --------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF6 RF19 +----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF8 RF19 --------------------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 ----------------------------------------------------------------------------PhysicalProject @@ -58,14 +58,14 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------------------------PhysicalProject --------------------------------------------------------------------------PhysicalOlapScan[customer] --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------filter(d_year IN (2001, 2002)) -------------------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------------filter(d_year IN (2001, 2002)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------------PhysicalProject -----------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------------PhysicalOlapScan[store] ----------------------------------------------------PhysicalProject ------------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query65.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query65.out index 58ec73205f4728..b2df7074aeec9b 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query65.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query65.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalDistribute[DistributionSpecGather] ----------PhysicalTopN[LOCAL_SORT] ------------PhysicalProject ---------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] +--------------hashJoin[INNER_JOIN colocated] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() --------------------PhysicalProject @@ -27,17 +27,15 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] ----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_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 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1232) and (date_dim.d_month_seq >= 1221)) ---------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1232) and (date_dim.d_month_seq >= 1221)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query68.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query68.out index fa40195e0fcefa..327930cde7b3c0 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query68.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query68.out @@ -9,19 +9,23 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=(( not (ca_city = bought_city))) ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF3 hd_demo_sk->[ss_hdemo_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] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] ----------------------------------------PhysicalProject ------------------------------------------filter((date_dim.d_dom <= 2) and (date_dim.d_dom >= 1) and d_year IN (1998, 1999, 2000)) --------------------------------------------PhysicalOlapScan[date_dim] @@ -31,8 +35,6 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) ------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] ----------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query73.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query73.out index 8cfa2b3c64b2b7..ef94345e10145a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query73.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query73.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------filter((dj.cnt <= 5) and (dj.cnt >= 1)) --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query79.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query79.out index 8f9b721f08ee59..32006bcc40781c 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query79.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query87.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query87.out index 431debd06c471b..19f7e59f537b2b 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query87.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query87.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalExcept ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -20,7 +20,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query89.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query89.out index 2e9294bce91fd2..8b0e89ca99b508 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query89.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query89.out @@ -9,25 +9,24 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query98.out b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query98.out index 1f92ed1e36f204..a50cfba68476d5 100644 --- a/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query98.out +++ b/regression-test/data/shape_check/tpcds_sf100/noStatsRfPrune/query98.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Music', 'Shoes', 'Sports')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-06-19') and (date_dim.d_date >= '2002-05-20')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Music', 'Shoes', 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2002-06-19') and (date_dim.d_date >= '2002-05-20')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query12.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query12.out index b4c126ae67aebf..6d9e989b19298d 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query12.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query12.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[ws_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Books', 'Men', 'Sports')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1998-05-06') and (date_dim.d_date >= '1998-04-06')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[ws_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Men', 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1998-05-06') and (date_dim.d_date >= '1998-04-06')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query20.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query20.out index fa360d9c6fdc83..3d0b0590dab75e 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query20.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query20.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Books', 'Shoes', 'Women')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-02-25') and (date_dim.d_date >= '2002-01-26')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Shoes', 'Women')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2002-02-25') and (date_dim.d_date >= '2002-01-26')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query28.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query28.out index 7a6bdd8868ef00..7a488abb6dabd0 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query28.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query34.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query34.out index a19fe778a57647..ed1f30d2c00dbe 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query34.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query34.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query38.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query38.out index 6dff99a09a656a..ab623089a49f20 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query38.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query38.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalIntersect RFV2: RF6[c_last_name->c_last_name] RF7[c_last_name->c_last_name] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -35,7 +35,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] RFV2: RF6 ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query46.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query46.out index 802c3969afc6e1..73faa771d22a1b 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query46.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query46.out @@ -7,19 +7,23 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=(( not (ca_city = bought_city))) build RFs:RF5 ca_address_sk->[c_current_addr_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] ----------------PhysicalProject -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ss_addr_sk] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF3 hd_demo_sk->[ss_hdemo_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] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ss_addr_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer_address] ------------------------------------PhysicalProject --------------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) ----------------------------------------PhysicalOlapScan[date_dim] @@ -29,8 +33,6 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------filter(OR[(household_demographics.hd_dep_count = 6),(household_demographics.hd_vehicle_count = 0)]) --------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject ------------------PhysicalOlapScan[customer] apply RFs: RF5 ------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query47.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query47.out index 645a46743f3d10..2f76e9211ef4df 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query47.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalProject +--------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query51.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query51.out index 11fecfc9825a58..9b4742d60618ee 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query51.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query51.out @@ -14,29 +14,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query53.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query53.out index 04920e65ac6894..00c45d333ec013 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query53.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query53.out @@ -8,25 +8,24 @@ PhysicalResultSink ----------filter((if((avg_quarterly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_quarterly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_quarterly_sales), NULL) > 0.100000)) ------------PhysicalWindow --------------PhysicalQuickSort[LOCAL_SORT] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------PhysicalProject -----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +----------------PhysicalProject +------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------------------PhysicalProject +--------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query54.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query54.out index 8bcb06bc8e6231..0be42521255ae6 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query54.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query54.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= d_month_seq+3) ------------------------PhysicalProject @@ -25,7 +25,7 @@ PhysicalResultSink --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF7 --------------------------------------------PhysicalProject -----------------------------------------------hashAgg[LOCAL] +----------------------------------------------hashAgg[GLOBAL] ------------------------------------------------PhysicalProject --------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[cs_bill_customer_sk,ws_bill_customer_sk] ----------------------------------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query57.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query57.out index 88777bc1ff548d..697bd284f5701e 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query57.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalProject +--------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[call_center] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query63.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query63.out index d4fb4990da98b8..210a97832febf1 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query63.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query63.out @@ -8,25 +8,24 @@ PhysicalResultSink ----------filter((if((avg_monthly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) ------------PhysicalWindow --------------PhysicalQuickSort[LOCAL_SORT] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalProject ---------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------PhysicalProject -----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +----------------PhysicalProject +------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------------------PhysicalProject +--------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +----------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query64.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query64.out index d1d49f78ba895b..634b564c010c97 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query64.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query64.out @@ -27,13 +27,13 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------PhysicalProject --------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF10 cd_demo_sk->[ss_cdemo_sk] ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[c_first_shipto_date_sk] +------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] --------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[c_first_sales_date_sk] +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF7 s_store_sk->[ss_store_sk] +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[c_first_shipto_date_sk] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[c_first_sales_date_sk] --------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] ------------------------------------------------------------------------PhysicalProject @@ -41,7 +41,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------------------------------------------------------------------PhysicalProject ------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_item_sk->[ss_item_sk];RF3 sr_ticket_number->[ss_ticket_number] --------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 RF10 RF12 RF13 RF15 RF19 +----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF5 RF8 RF9 RF10 RF12 RF13 RF15 RF19 --------------------------------------------------------------------------------PhysicalProject ----------------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 ----------------------------------------------------------------------------PhysicalProject @@ -56,16 +56,16 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------------------------------------------PhysicalProject --------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF19 ------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF8 RF9 RF11 RF14 RF16 +--------------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF6 RF7 RF11 RF14 RF16 --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------filter(d_year IN (2001, 2002)) -------------------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------------filter(d_year IN (2001, 2002)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------------PhysicalProject -----------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------------PhysicalOlapScan[store] ----------------------------------------------------PhysicalProject ------------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query65.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query65.out index c521ab31e4f4f0..ec9fa3ea438f4c 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query65.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query65.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalDistribute[DistributionSpecGather] ----------PhysicalTopN[LOCAL_SORT] ------------PhysicalProject ---------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] +--------------hashJoin[INNER_JOIN colocated] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] --------------------PhysicalProject @@ -27,17 +27,15 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] ----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_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 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1232) and (date_dim.d_month_seq >= 1221)) ---------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1232) and (date_dim.d_month_seq >= 1221)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query68.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query68.out index a417594ba5b43a..3e4c771ecc20c6 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query68.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query68.out @@ -9,19 +9,23 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk)) otherCondition=(( not (ca_city = bought_city))) build RFs:RF5 ca_address_sk->[c_current_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] --------------------PhysicalProject -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ss_addr_sk] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF3 hd_demo_sk->[ss_hdemo_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] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[ss_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] ----------------------------------------PhysicalProject ------------------------------------------filter((date_dim.d_dom <= 2) and (date_dim.d_dom >= 1) and d_year IN (1998, 1999, 2000)) --------------------------------------------PhysicalOlapScan[date_dim] @@ -31,8 +35,6 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------filter(OR[(household_demographics.hd_dep_count = 8),(household_demographics.hd_vehicle_count = -1)]) ------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] apply RFs: RF5 ----------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query73.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query73.out index a7655a2c3f268a..969743c9f9f833 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query73.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query73.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------filter((dj.cnt <= 5) and (dj.cnt >= 1)) --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query79.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query79.out index 1a19308d991441..070baafb43c25a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query79.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query87.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query87.out index a71098a021052f..10a3c4a3e35495 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query87.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query87.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalExcept ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -20,7 +20,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query89.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query89.out index e4d2ae3435f174..552fbbd3aaf5ae 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query89.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query89.out @@ -9,25 +9,24 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query98.out b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query98.out index 1f92ed1e36f204..a50cfba68476d5 100644 --- a/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query98.out +++ b/regression-test/data/shape_check/tpcds_sf100/no_stats_shape/query98.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Music', 'Shoes', 'Sports')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-06-19') and (date_dim.d_date >= '2002-05-20')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Music', 'Shoes', 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2002-06-19') and (date_dim.d_date >= '2002-05-20')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query28.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query28.out index 7a6bdd8868ef00..7a488abb6dabd0 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query28.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query34.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query34.out index 79c46ccb77bbb8..b1fdb6d566299a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query34.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query34.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +----------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] ------------PhysicalProject --------------PhysicalOlapScan[customer] apply RFs: RF3 ------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query38.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query38.out index b06a00731068b8..5a103e684ebdcb 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query38.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query38.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalIntersect RFV2: RF6[c_last_name->c_last_name] RF7[c_last_name->c_last_name] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -35,7 +35,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] RFV2: RF6 ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query46.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query46.out index 36270285709c58..2b48783b05a5a4 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query46.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query46.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() ------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query47.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query47.out index 048f93392595ec..0a70cbcf51c3a2 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query47.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------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 -----------------------------------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(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------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 +--------------------------------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(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query51.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query51.out index 11fecfc9825a58..9b4742d60618ee 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query51.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query51.out @@ -14,29 +14,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query57.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query57.out index 4f23fac89cf958..ca1f63bfb07616 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query57.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[call_center] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query65.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query65.out index 9b233769312c7a..9852100dbb9d6f 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query65.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query65.out @@ -11,7 +11,7 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = sc.ss_store_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() ------------------------hashAgg[GLOBAL] --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query68.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query68.out index f93e4120b2382a..9d7c3b294815ab 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query68.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query68.out @@ -15,7 +15,7 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] apply RFs: RF4 --------------------PhysicalProject -----------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query75.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query75.out index 921d754e533285..89334acaef5f92 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query75.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query75.out @@ -3,63 +3,61 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalDistribute[DistributionSpecExecutionAny] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalUnion +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] --------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +----------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ----------------------------PhysicalProject -------------------------------filter(d_year IN (1998, 1999)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +------------------------------filter((item.i_category = 'Home')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------filter(d_year IN (1998, 1999)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ----------------------------PhysicalProject -------------------------------filter(d_year IN (1998, 1999)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +------------------------------filter((item.i_category = 'Home')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------------filter(d_year IN (1998, 1999)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ----------------------------PhysicalProject -------------------------------filter(d_year IN (1998, 1999)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_category = 'Home')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter(d_year IN (1998, 1999)) +----------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query79.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query79.out index f57418546e7fb9..89ea85a6285475 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query79.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query87.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query87.out index 96d989c4955353..477b102ee79e8a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query87.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query87.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalExcept RFV2: RF6[c_last_name->c_last_name] RF7[c_last_name->c_last_name] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -20,7 +20,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] RFV2: RF6 ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query89.out b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query89.out index 2e9294bce91fd2..8b0e89ca99b508 100644 --- a/regression-test/data/shape_check/tpcds_sf100/rf_prune/query89.out +++ b/regression-test/data/shape_check/tpcds_sf100/rf_prune/query89.out @@ -9,25 +9,24 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query28.out b/regression-test/data/shape_check/tpcds_sf100/shape/query28.out index 7a6bdd8868ef00..7a488abb6dabd0 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query28.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 131.00),(store_sales.ss_list_price <= 141.00)],AND[(store_sales.ss_coupon_amt >= 16798.00),(store_sales.ss_coupon_amt <= 17798.00)],AND[(store_sales.ss_wholesale_cost >= 25.00),(store_sales.ss_wholesale_cost <= 45.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 145.00),(store_sales.ss_list_price <= 155.00)],AND[(store_sales.ss_coupon_amt >= 14792.00),(store_sales.ss_coupon_amt <= 15792.00)],AND[(store_sales.ss_wholesale_cost >= 46.00),(store_sales.ss_wholesale_cost <= 66.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 150.00),(store_sales.ss_list_price <= 160.00)],AND[(store_sales.ss_coupon_amt >= 6600.00),(store_sales.ss_coupon_amt <= 7600.00)],AND[(store_sales.ss_wholesale_cost >= 9.00),(store_sales.ss_wholesale_cost <= 29.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 91.00),(store_sales.ss_list_price <= 101.00)],AND[(store_sales.ss_coupon_amt >= 13493.00),(store_sales.ss_coupon_amt <= 14493.00)],AND[(store_sales.ss_wholesale_cost >= 36.00),(store_sales.ss_wholesale_cost <= 56.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 0.00),(store_sales.ss_list_price <= 10.00)],AND[(store_sales.ss_coupon_amt >= 7629.00),(store_sales.ss_coupon_amt <= 8629.00)],AND[(store_sales.ss_wholesale_cost >= 6.00),(store_sales.ss_wholesale_cost <= 26.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 15257.00),(store_sales.ss_coupon_amt <= 16257.00)],AND[(store_sales.ss_wholesale_cost >= 31.00),(store_sales.ss_wholesale_cost <= 51.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query34.out b/regression-test/data/shape_check/tpcds_sf100/shape/query34.out index 79c46ccb77bbb8..b1fdb6d566299a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query34.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query34.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +----------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] ------------PhysicalProject --------------PhysicalOlapScan[customer] apply RFs: RF3 ------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query38.out b/regression-test/data/shape_check/tpcds_sf100/shape/query38.out index 055f3f812dd0ba..ab623089a49f20 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query38.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query38.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalIntersect RFV2: RF6[c_last_name->c_last_name] RF7[c_last_name->c_last_name] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -35,7 +35,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] RFV2: RF6 ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query46.out b/regression-test/data/shape_check/tpcds_sf100/shape/query46.out index 71a8b55bb24018..13bf6421c7ed3a 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query46.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query46.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ss_addr_sk] ------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query47.out b/regression-test/data/shape_check/tpcds_sf100/shape/query47.out index ecc4777c22f664..fba9743e0f2353 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query47.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------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 -----------------------------------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(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------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 +--------------------------------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(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query51.out b/regression-test/data/shape_check/tpcds_sf100/shape/query51.out index 11fecfc9825a58..9b4742d60618ee 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query51.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query51.out @@ -14,29 +14,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query57.out b/regression-test/data/shape_check/tpcds_sf100/shape/query57.out index 2cab4f33e1358d..152c2c884779f7 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query57.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[call_center] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query65.out b/regression-test/data/shape_check/tpcds_sf100/shape/query65.out index d79e026da9ce01..85952f534a908d 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query65.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query65.out @@ -11,7 +11,7 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = sc.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------hashAgg[GLOBAL] --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query68.out b/regression-test/data/shape_check/tpcds_sf100/shape/query68.out index f93e4120b2382a..9d7c3b294815ab 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query68.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query68.out @@ -15,7 +15,7 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] apply RFs: RF4 --------------------PhysicalProject -----------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query75.out b/regression-test/data/shape_check/tpcds_sf100/shape/query75.out index 921d754e533285..89334acaef5f92 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query75.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query75.out @@ -3,63 +3,61 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalDistribute[DistributionSpecExecutionAny] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalUnion +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] --------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +----------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ----------------------------PhysicalProject -------------------------------filter(d_year IN (1998, 1999)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +------------------------------filter((item.i_category = 'Home')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------filter(d_year IN (1998, 1999)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ----------------------------PhysicalProject -------------------------------filter(d_year IN (1998, 1999)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +------------------------------filter((item.i_category = 'Home')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------------filter(d_year IN (1998, 1999)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ----------------------------PhysicalProject -------------------------------filter(d_year IN (1998, 1999)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_category = 'Home')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter(d_year IN (1998, 1999)) +----------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query79.out b/regression-test/data/shape_check/tpcds_sf100/shape/query79.out index 974fb1da39b734..ca85e61ee3240e 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query79.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query87.out b/regression-test/data/shape_check/tpcds_sf100/shape/query87.out index 61eb6848d23bfc..bd1f7347fd39f8 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query87.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query87.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalExcept RFV2: RF6[c_last_name->c_last_name] RF7[c_last_name->c_last_name] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -20,7 +20,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] RFV2: RF6 ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf100/shape/query89.out b/regression-test/data/shape_check/tpcds_sf100/shape/query89.out index e4d2ae3435f174..552fbbd3aaf5ae 100644 --- a/regression-test/data/shape_check/tpcds_sf100/shape/query89.out +++ b/regression-test/data/shape_check/tpcds_sf100/shape/query89.out @@ -9,25 +9,24 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Electronics', 'Jewelry', 'Shoes'),i_class IN ('athletic', 'portable', 'semi-precious')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'maternity', 'rock')]] and i_category IN ('Electronics', 'Jewelry', 'Men', 'Music', 'Shoes', 'Women') and i_class IN ('accessories', 'athletic', 'maternity', 'portable', 'rock', 'semi-precious')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query68.out b/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query68.out index bcbc6dfb42e77e..9c9c215f8d46fc 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query68.out +++ b/regression-test/data/shape_check/tpcds_sf1000/bs_downgrade_shape/query68.out @@ -15,7 +15,7 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] apply RFs: RF4 --------------------PhysicalProject -----------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query28.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query28.out index 36ec7305d96abb..c1d2341a2a776b 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query28.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query38.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query38.out index dc794c95f97b2f..5b01f5ff8b4586 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query38.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query38.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalIntersect ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -35,7 +35,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query46.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query46.out index 1d221460f2e13b..1b99c6d506b15e 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query46.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query46.out @@ -11,7 +11,7 @@ PhysicalResultSink ----------------PhysicalProject ------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ss_addr_sk] ------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query47.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query47.out index 065060bcec4716..9a2258d852cebf 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query47.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------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: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(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------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: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(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query51.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query51.out index 37d8b6fb556a40..54d305dd4b0141 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query51.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query51.out @@ -14,31 +14,29 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) +------------------------------------------PhysicalOlapScan[date_dim] Hint log: Used: leading(web_sales date_dim ) leading(store_sales date_dim ) leading(web store ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query57.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query57.out index a534db2063336b..a2a31216a3d458 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query57.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------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:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[call_center] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------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:RF2 i_item_sk->[cs_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[call_center] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query65.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query65.out index 4e8a4eba0fe0f6..025ce3c8c448a7 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query65.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query65.out @@ -9,9 +9,9 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = sc.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = sc.ss_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------PhysicalProject --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecHash] @@ -24,9 +24,9 @@ PhysicalResultSink --------------------------------------filter((date_dim.d_month_seq <= 1187) and (date_dim.d_month_seq >= 1176)) ----------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] apply RFs: RF4 +--------------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] --------------------PhysicalProject -----------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] +----------------------PhysicalOlapScan[store] apply RFs: RF4 ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query68.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query68.out index eabeb2d1088126..e0c2b87640c4be 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query68.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query68.out @@ -15,7 +15,7 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] apply RFs: RF4 --------------------PhysicalProject -----------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query75.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query75.out index c26b81b87791ba..8fcdeb91302f20 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query75.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query75.out @@ -3,63 +3,61 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalDistribute[DistributionSpecExecutionAny] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalUnion +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] --------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +----------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Sports')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ----------------------------PhysicalProject -------------------------------filter(d_year IN (2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +------------------------------filter((item.i_category = 'Sports')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------filter(d_year IN (2001, 2002)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Sports')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ----------------------------PhysicalProject -------------------------------filter(d_year IN (2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +------------------------------filter((item.i_category = 'Sports')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------------filter(d_year IN (2001, 2002)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Sports')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ----------------------------PhysicalProject -------------------------------filter(d_year IN (2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_category = 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter(d_year IN (2001, 2002)) +----------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query79.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query79.out index 1ade033847b1c9..8f352e5cb7aaeb 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query79.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query87.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query87.out index d21bcaff8bef43..b015e93babbe0c 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query87.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query87.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalExcept ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -20,7 +20,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf1000/hint/query89.out b/regression-test/data/shape_check/tpcds_sf1000/hint/query89.out index 97b06c9ab60a5b..a1e8e771fe13f5 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/hint/query89.out +++ b/regression-test/data/shape_check/tpcds_sf1000/hint/query89.out @@ -9,27 +9,26 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] Hint log: Used: leading(store_sales item date_dim store ) diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query28.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query28.out index 36ec7305d96abb..c1d2341a2a776b 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query28.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 107.00),(store_sales.ss_list_price <= 117.00)],AND[(store_sales.ss_coupon_amt >= 1319.00),(store_sales.ss_coupon_amt <= 2319.00)],AND[(store_sales.ss_wholesale_cost >= 60.00),(store_sales.ss_wholesale_cost <= 80.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 23.00),(store_sales.ss_list_price <= 33.00)],AND[(store_sales.ss_coupon_amt >= 825.00),(store_sales.ss_coupon_amt <= 1825.00)],AND[(store_sales.ss_wholesale_cost >= 43.00),(store_sales.ss_wholesale_cost <= 63.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 74.00),(store_sales.ss_list_price <= 84.00)],AND[(store_sales.ss_coupon_amt >= 4381.00),(store_sales.ss_coupon_amt <= 5381.00)],AND[(store_sales.ss_wholesale_cost >= 57.00),(store_sales.ss_wholesale_cost <= 77.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 89.00),(store_sales.ss_list_price <= 99.00)],AND[(store_sales.ss_coupon_amt >= 3117.00),(store_sales.ss_coupon_amt <= 4117.00)],AND[(store_sales.ss_wholesale_cost >= 68.00),(store_sales.ss_wholesale_cost <= 88.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 58.00),(store_sales.ss_list_price <= 68.00)],AND[(store_sales.ss_coupon_amt >= 9402.00),(store_sales.ss_coupon_amt <= 10402.00)],AND[(store_sales.ss_wholesale_cost >= 38.00),(store_sales.ss_wholesale_cost <= 58.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 64.00),(store_sales.ss_list_price <= 74.00)],AND[(store_sales.ss_coupon_amt >= 5792.00),(store_sales.ss_coupon_amt <= 6792.00)],AND[(store_sales.ss_wholesale_cost >= 73.00),(store_sales.ss_wholesale_cost <= 93.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query38.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query38.out index dc794c95f97b2f..5b01f5ff8b4586 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query38.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query38.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------PhysicalIntersect ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ws_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -22,7 +22,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] @@ -35,7 +35,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query46.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query46.out index cb7d73da5838ed..e9f7fea08a3864 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query46.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query46.out @@ -13,7 +13,7 @@ PhysicalResultSink ----------------PhysicalProject ------------------PhysicalOlapScan[customer] apply RFs: RF4 ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ss_addr_sk->[ca_address_sk] ------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query47.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query47.out index 0e9f713243773a..0428d0e8670918 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query47.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------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: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(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------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: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(OR[(date_dim.d_year = 2000),AND[(date_dim.d_year = 1999),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2001),(date_dim.d_moy = 1)]] and d_year IN (1999, 2000, 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query51.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query51.out index 9ffa695b733bf9..5acf6623389d03 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query51.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query51.out @@ -14,29 +14,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1223) and (date_dim.d_month_seq >= 1212)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query57.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query57.out index 2e8174812f69ea..00c01451579574 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query57.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------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:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[call_center] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------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:RF2 i_item_sk->[cs_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF1 cc_call_center_sk->[cs_call_center_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter(OR[(date_dim.d_year = 2001),AND[(date_dim.d_year = 2000),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2002),(date_dim.d_moy = 1)]] and d_year IN (2000, 2001, 2002)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[call_center] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query65.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query65.out index 3840e10df485e8..11de9c7149d706 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query65.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query65.out @@ -9,9 +9,9 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN broadcast] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = sc.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = sc.ss_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------hashAgg[GLOBAL] --------------------------PhysicalDistribute[DistributionSpecHash] ----------------------------hashAgg[LOCAL] @@ -23,9 +23,9 @@ PhysicalResultSink ------------------------------------filter((date_dim.d_month_seq <= 1187) and (date_dim.d_month_seq >= 1176)) --------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] apply RFs: RF4 +--------------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] --------------------PhysicalProject -----------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] +----------------------PhysicalOlapScan[store] apply RFs: RF4 ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query68.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query68.out index bcbc6dfb42e77e..9c9c215f8d46fc 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query68.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query68.out @@ -15,7 +15,7 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[customer lazySlots:(customer.c_first_name)] apply RFs: RF4 --------------------PhysicalProject -----------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ss_addr_sk->[ca_address_sk] ----------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query75.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query75.out index c26b81b87791ba..8fcdeb91302f20 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query75.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query75.out @@ -3,63 +3,61 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalUnion -------------------PhysicalDistribute[DistributionSpecExecutionAny] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] +------------PhysicalUnion +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] --------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +----------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Sports')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ----------------------------PhysicalProject -------------------------------filter(d_year IN (2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +------------------------------filter((item.i_category = 'Sports')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------------filter(d_year IN (2001, 2002)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF6 ss_ticket_number->[sr_ticket_number];RF7 ss_item_sk->[sr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Sports')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 ----------------------------PhysicalProject -------------------------------filter(d_year IN (2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalProject -----------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +------------------------------filter((item.i_category = 'Sports')) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------------filter(d_year IN (2001, 2002)) +----------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number)) otherCondition=() build RFs:RF10 ws_order_number->[wr_order_number];RF11 ws_item_sk->[wr_item_sk] +--------------------PhysicalProject +----------------------PhysicalOlapScan[web_returns] apply RFs: RF10 RF11 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = web_sales.ws_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ws_item_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Sports')) -------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF8 RF9 ----------------------------PhysicalProject -------------------------------filter(d_year IN (2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((item.i_category = 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter(d_year IN (2001, 2002)) +----------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query79.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query79.out index e3d8f3af326d91..605562961aa019 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query79.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query87.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query87.out index d21bcaff8bef43..b015e93babbe0c 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query87.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query87.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalExcept ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -20,7 +20,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] @@ -33,7 +33,7 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalOlapScan[customer] ------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] --------------------hashAgg[LOCAL] diff --git a/regression-test/data/shape_check/tpcds_sf1000/shape/query89.out b/regression-test/data/shape_check/tpcds_sf1000/shape/query89.out index b8751687a0ff29..e18adf036aa80e 100644 --- a/regression-test/data/shape_check/tpcds_sf1000/shape/query89.out +++ b/regression-test/data/shape_check/tpcds_sf1000/shape/query89.out @@ -9,25 +9,24 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('audio', 'history', 'school-uniforms')],AND[i_category IN ('Men', 'Shoes', 'Sports'),i_class IN ('pants', 'tennis', 'womens')]] and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Shoes', 'Sports') and i_class IN ('audio', 'history', 'pants', 'school-uniforms', 'tennis', 'womens')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query12.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query12.out index 31121aee38f722..82a217c4ab93cb 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query12.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query12.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[ws_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Books', 'Electronics', 'Women')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '1998-02-05') and (date_dim.d_date >= '1998-01-06')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[ws_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Electronics', 'Women')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '1998-02-05') and (date_dim.d_date >= '1998-01-06')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query20.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query20.out index 5ea9d277dedaba..ab15e339c3245e 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query20.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query20.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Children', 'Electronics', 'Shoes')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-04-13') and (date_dim.d_date >= '2001-03-14')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Children', 'Electronics', 'Shoes')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2001-04-13') and (date_dim.d_date >= '2001-03-14')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query23.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query23.out index e1bf14a3b8eec4..26c66518a58e08 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query23.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query23.out @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalProject --------NestedLoopJoin[INNER_JOIN](cast(ssales as DOUBLE) > cast((0.9500 * tpcds_cmax) as DOUBLE)) ----------PhysicalProject -------------hashAgg[LOCAL] +------------hashAgg[GLOBAL] --------------PhysicalProject ----------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] ------------------PhysicalProject @@ -35,7 +35,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute[DistributionSpecGather] ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashAgg[LOCAL] +--------------------hashAgg[GLOBAL] ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query28.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query28.out index fd2a30be06e564..59b29ba643d752 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query28.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query28.out @@ -13,45 +13,63 @@ PhysicalResultSink --------------------PhysicalLimit[LOCAL] ----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 73.00),(store_sales.ss_list_price <= 83.00)],AND[(store_sales.ss_coupon_amt >= 7826.00),(store_sales.ss_coupon_amt <= 8826.00)],AND[(store_sales.ss_wholesale_cost >= 70.00),(store_sales.ss_wholesale_cost <= 90.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0) and OR[AND[(store_sales.ss_list_price >= 73.00),(store_sales.ss_list_price <= 83.00)],AND[(store_sales.ss_coupon_amt >= 7826.00),(store_sales.ss_coupon_amt <= 8826.00)],AND[(store_sales.ss_wholesale_cost >= 70.00),(store_sales.ss_wholesale_cost <= 90.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalLimit[LOCAL] ---------------------------hashAgg[GLOBAL] +--------------------------hashAgg[DISTINCT_GLOBAL] ----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 152.00),(store_sales.ss_list_price <= 162.00)],AND[(store_sales.ss_coupon_amt >= 2196.00),(store_sales.ss_coupon_amt <= 3196.00)],AND[(store_sales.ss_wholesale_cost >= 56.00),(store_sales.ss_wholesale_cost <= 76.00)]]) -------------------------------------PhysicalOlapScan[store_sales] +------------------------------hashAgg[DISTINCT_LOCAL] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------filter((store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6) and OR[AND[(store_sales.ss_list_price >= 152.00),(store_sales.ss_list_price <= 162.00)],AND[(store_sales.ss_coupon_amt >= 2196.00),(store_sales.ss_coupon_amt <= 3196.00)],AND[(store_sales.ss_wholesale_cost >= 56.00),(store_sales.ss_wholesale_cost <= 76.00)]]) +------------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalLimit[LOCAL] -----------------------hashAgg[GLOBAL] +----------------------hashAgg[DISTINCT_GLOBAL] ------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 53.00),(store_sales.ss_list_price <= 63.00)],AND[(store_sales.ss_coupon_amt >= 3430.00),(store_sales.ss_coupon_amt <= 4430.00)],AND[(store_sales.ss_wholesale_cost >= 13.00),(store_sales.ss_wholesale_cost <= 33.00)]]) ---------------------------------PhysicalOlapScan[store_sales] +--------------------------hashAgg[DISTINCT_LOCAL] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11) and OR[AND[(store_sales.ss_list_price >= 53.00),(store_sales.ss_list_price <= 63.00)],AND[(store_sales.ss_coupon_amt >= 3430.00),(store_sales.ss_coupon_amt <= 4430.00)],AND[(store_sales.ss_wholesale_cost >= 13.00),(store_sales.ss_wholesale_cost <= 33.00)]]) +--------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalLimit[LOCAL] -------------------hashAgg[GLOBAL] +------------------hashAgg[DISTINCT_GLOBAL] --------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 182.00),(store_sales.ss_list_price <= 192.00)],AND[(store_sales.ss_coupon_amt >= 3262.00),(store_sales.ss_coupon_amt <= 4262.00)],AND[(store_sales.ss_wholesale_cost >= 20.00),(store_sales.ss_wholesale_cost <= 40.00)]]) -----------------------------PhysicalOlapScan[store_sales] +----------------------hashAgg[DISTINCT_LOCAL] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16) and OR[AND[(store_sales.ss_list_price >= 182.00),(store_sales.ss_list_price <= 192.00)],AND[(store_sales.ss_coupon_amt >= 3262.00),(store_sales.ss_coupon_amt <= 4262.00)],AND[(store_sales.ss_wholesale_cost >= 20.00),(store_sales.ss_wholesale_cost <= 40.00)]]) +----------------------------------PhysicalOlapScan[store_sales] ------------PhysicalLimit[LOCAL] ---------------hashAgg[GLOBAL] +--------------hashAgg[DISTINCT_GLOBAL] ----------------PhysicalDistribute[DistributionSpecGather] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 85.00),(store_sales.ss_list_price <= 95.00)],AND[(store_sales.ss_coupon_amt >= 3310.00),(store_sales.ss_coupon_amt <= 4310.00)],AND[(store_sales.ss_wholesale_cost >= 37.00),(store_sales.ss_wholesale_cost <= 57.00)]]) -------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[DISTINCT_LOCAL] +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21) and OR[AND[(store_sales.ss_list_price >= 85.00),(store_sales.ss_list_price <= 95.00)],AND[(store_sales.ss_coupon_amt >= 3310.00),(store_sales.ss_coupon_amt <= 4310.00)],AND[(store_sales.ss_wholesale_cost >= 37.00),(store_sales.ss_wholesale_cost <= 57.00)]]) +------------------------------PhysicalOlapScan[store_sales] --------PhysicalLimit[LOCAL] -----------hashAgg[GLOBAL] +----------hashAgg[DISTINCT_GLOBAL] ------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 180.00),(store_sales.ss_list_price <= 190.00)],AND[(store_sales.ss_coupon_amt >= 12592.00),(store_sales.ss_coupon_amt <= 13592.00)],AND[(store_sales.ss_wholesale_cost >= 22.00),(store_sales.ss_wholesale_cost <= 42.00)]]) ---------------------PhysicalOlapScan[store_sales] +--------------hashAgg[DISTINCT_LOCAL] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26) and OR[AND[(store_sales.ss_list_price >= 180.00),(store_sales.ss_list_price <= 190.00)],AND[(store_sales.ss_coupon_amt >= 12592.00),(store_sales.ss_coupon_amt <= 13592.00)],AND[(store_sales.ss_wholesale_cost >= 22.00),(store_sales.ss_wholesale_cost <= 42.00)]]) +--------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query34.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query34.out index 654477132bf75e..f5c833d21c1256 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query34.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query34.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query39.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query39.out index 7d25415833f93b..81c822f42d871d 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query39.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query39.out @@ -4,21 +4,23 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------filter((if((mean = 0.0), 0.0, (stdev / mean)) > 1.0)) ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffle] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[inv_date_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] -----------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF1 w_warehouse_sk->[inv_warehouse_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] ------------------PhysicalProject --------------------filter((date_dim.d_year = 2000) and d_moy IN (2, 3)) ----------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------PhysicalOlapScan[warehouse] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query46.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query46.out index effe4533d7debc..3675020a5eb76c 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query46.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query46.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] ----------------PhysicalProject -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ss_addr_sk] ------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query47.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query47.out index a04edf7f6b8f4a..d1c72cdf6f5766 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query47.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query47.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +--------------PhysicalProject +----------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalProject +--------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query51.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query51.out index 9f3b3b1b1a3495..a89b73db446b0a 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query51.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query51.out @@ -14,29 +14,27 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject ------------------------PhysicalWindow --------------------------PhysicalQuickSort[LOCAL_SORT] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query53.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query53.out index 382c6a18e33765..08f3ba6e090871 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query53.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query53.out @@ -8,25 +8,24 @@ PhysicalResultSink ----------filter((if((avg_quarterly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_quarterly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_quarterly_sales), NULL) > 0.100000)) ------------PhysicalWindow --------------PhysicalQuickSort[LOCAL_SORT] -----------------PhysicalDistribute[DistributionSpecHash] -------------------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: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(d_month_seq IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] -------------------------------PhysicalProject ---------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -----------------------------------PhysicalOlapScan[item] +----------------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: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(d_month_seq IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223)) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store] +----------------------------PhysicalProject +------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query54.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query54.out index 0211d74ab31beb..158e379109d710 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query54.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query54.out @@ -27,7 +27,7 @@ PhysicalResultSink ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF7 ------------------------------------------------PhysicalProject ---------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------hashAgg[GLOBAL] ----------------------------------------------------PhysicalProject ------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk,ws_sold_date_sk] --------------------------------------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query57.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query57.out index 88777bc1ff548d..697bd284f5701e 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query57.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query57.out @@ -7,26 +7,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalQuickSort[LOCAL_SORT] ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[call_center] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk)) otherCondition=() build RFs:RF2 cc_call_center_sk->[cs_call_center_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 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:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +------------------------------PhysicalProject +--------------------------------filter(OR[(date_dim.d_year = 1999),AND[(date_dim.d_year = 1998),(date_dim.d_moy = 12)],AND[(date_dim.d_year = 2000),(date_dim.d_moy = 1)]] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[call_center] --PhysicalResultSink ----PhysicalProject ------PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query63.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query63.out index 1dfce555f674d1..1ad6a72b1612f8 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query63.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query63.out @@ -8,25 +8,24 @@ PhysicalResultSink ----------filter((if((avg_monthly_sales > 0.0000), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) ------------PhysicalWindow --------------PhysicalQuickSort[LOCAL_SORT] -----------------PhysicalDistribute[DistributionSpecHash] -------------------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: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(d_month_seq IN (1211, 1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] -------------------------------PhysicalProject ---------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) -----------------------------------PhysicalOlapScan[item] +----------------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: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(d_month_seq IN (1211, 1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222)) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store] +----------------------------PhysicalProject +------------------------------filter(OR[AND[i_category IN ('Books', 'Children', 'Electronics'),i_class IN ('personal', 'portable', 'reference', 'self-help'),i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')],AND[i_category IN ('Men', 'Music', 'Women'),i_class IN ('accessories', 'classical', 'fragrances', 'pants'),i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')]] and i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'exportiunivamalg #9', 'importoamalg #1', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9') and i_category IN ('Books', 'Children', 'Electronics', 'Men', 'Music', 'Women') and i_class IN ('accessories', 'classical', 'fragrances', 'pants', 'personal', 'portable', 'reference', 'self-help')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query65.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query65.out index 9cf67b6f83a740..a7305a2c205a66 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query65.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query65.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalDistribute[DistributionSpecGather] ----------PhysicalTopN[LOCAL_SORT] ------------PhysicalProject ---------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] +--------------hashJoin[INNER_JOIN colocated] hashCondition=((sb.ss_store_sk = sc.ss_store_sk)) otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) build RFs:RF4 ss_store_sk->[s_store_sk,ss_store_sk] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = sc.ss_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] --------------------PhysicalProject @@ -27,17 +27,15 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalLazyMaterializeOlapScan[item lazySlots:(item.i_current_price,item.i_wholesale_cost,item.i_brand)] ----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_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 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) ---------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq <= 1197) and (date_dim.d_month_seq >= 1186)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query68.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query68.out index 8a3ad93d8b2cd1..630126cf643027 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query68.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query68.out @@ -11,7 +11,7 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] --------------------PhysicalProject -----------------------hashAgg[LOCAL] +----------------------hashAgg[GLOBAL] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[ss_addr_sk] ----------------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query73.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query73.out index 33a130844295c9..70d93fd9695e48 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query73.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query73.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------filter((dj.cnt <= 5) and (dj.cnt >= 1)) --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query79.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query79.out index 0239a35e7676cd..3de405496a1870 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query79.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query79.out @@ -5,7 +5,7 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query89.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query89.out index a24231e1dd660d..db4f8eeac80499 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query89.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query89.out @@ -9,25 +9,24 @@ PhysicalResultSink ------------filter((if(( not (avg_monthly_sales = 0.0000)), (cast(abs((sum_sales - cast(avg_monthly_sales as DECIMALV3(38, 2)))) as DECIMALV3(38, 10)) / avg_monthly_sales), NULL) > 0.100000)) --------------PhysicalWindow ----------------PhysicalQuickSort[LOCAL_SORT] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------------PhysicalProject -------------------------------------------filter(OR[AND[i_category IN ('Books', 'Home', 'Music'),i_class IN ('classical', 'fiction', 'glassware')],AND[i_category IN ('Jewelry', 'Sports', 'Women'),i_class IN ('baseball', 'dresses', 'semi-precious')]] and i_category IN ('Books', 'Home', 'Jewelry', 'Music', 'Sports', 'Women') and i_class IN ('baseball', 'classical', 'dresses', 'fiction', 'glassware', 'semi-precious')) ---------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------PhysicalProject +--------------------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_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------PhysicalProject +----------------------------------------filter(OR[AND[i_category IN ('Books', 'Home', 'Music'),i_class IN ('classical', 'fiction', 'glassware')],AND[i_category IN ('Jewelry', 'Sports', 'Women'),i_class IN ('baseball', 'dresses', 'semi-precious')]] and i_category IN ('Books', 'Home', 'Jewelry', 'Music', 'Sports', 'Women') and i_class IN ('baseball', 'classical', 'dresses', 'fiction', 'glassware', 'semi-precious')) +------------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query98.out b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query98.out index 65aec39f800674..dae24c00574632 100644 --- a/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query98.out +++ b/regression-test/data/shape_check/tpcds_sf10t_orc/shape/query98.out @@ -7,20 +7,19 @@ PhysicalResultSink --------PhysicalProject ----------PhysicalWindow ------------PhysicalQuickSort[LOCAL_SORT] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 -------------------------------PhysicalProject ---------------------------------filter(i_category IN ('Home', 'Men', 'Sports')) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2002-02-04') and (date_dim.d_date >= '2002-01-05')) -------------------------------PhysicalOlapScan[date_dim] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Home', 'Men', 'Sports')) +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalProject +--------------------------filter((date_dim.d_date <= '2002-02-04') and (date_dim.d_date >= '2002-01-05')) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/shape_check/tpch_sf1000/hint/q11.out b/regression-test/data/shape_check/tpch_sf1000/hint/q11.out index 8967622e92b179..675739f76d1056 100644 --- a/regression-test/data/shape_check/tpch_sf1000/hint/q11.out +++ b/regression-test/data/shape_check/tpch_sf1000/hint/q11.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast(sum(ps_supplycost * ps_availqty) * 0.000002 as DOUBLE)) ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() --------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/hint/q13.out b/regression-test/data/shape_check/tpch_sf1000/hint/q13.out index 818c200e8f0a72..a04a56b1345f43 100644 --- a/regression-test/data/shape_check/tpch_sf1000/hint/q13.out +++ b/regression-test/data/shape_check/tpch_sf1000/hint/q13.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------hashJoin[RIGHT_OUTER_JOIN shuffle] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() ----------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/hint/q3.out b/regression-test/data/shape_check/tpch_sf1000/hint/q3.out index e21654d94b0a75..d4cf366b41a7d2 100644 --- a/regression-test/data/shape_check/tpch_sf1000/hint/q3.out +++ b/regression-test/data/shape_check/tpch_sf1000/hint/q3.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineitem.l_orderkey = orders.o_orderkey)) otherCondition=() --------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q11.out b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q11.out index dcb4189583a0f1..6afed88faa4802 100644 --- a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q11.out +++ b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q11.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast(sum(ps_supplycost * ps_availqty) * 0.000002 as DOUBLE)) ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF3 n_nationkey->[s_nationkey] --------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q13.out b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q13.out index 9aefb642529297..6165574e0ee265 100644 --- a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q13.out +++ b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q13.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------hashJoin[RIGHT_OUTER_JOIN shuffle] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() ----------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q16.out b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q16.out index 7b04caaf3e087a..1ed947eacabb04 100644 --- a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q16.out +++ b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q16.out @@ -5,17 +5,18 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +--------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] --------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -----------------------PhysicalOlapScan[part] +----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q18.out b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q18.out index f2dd337eadfe86..c84338debea515 100644 --- a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q18.out +++ b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q18.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 o_orderkey->[l_orderkey] --------------PhysicalProject @@ -16,7 +16,7 @@ PhysicalResultSink ----------------------PhysicalOlapScan[orders] apply RFs: RF0 --------------------PhysicalProject ----------------------filter((sum(l_quantity) > 300.00)) -------------------------hashAgg[LOCAL] +------------------------hashAgg[GLOBAL] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineitem] ------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q3.out b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q3.out index 2fa14f8a3abf28..5e1ce9d65f8b5d 100644 --- a/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q3.out +++ b/regression-test/data/shape_check/tpch_sf1000/nostats_rf_prune/q3.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineitem.l_orderkey = orders.o_orderkey)) otherCondition=() build RFs:RF1 o_orderkey->[l_orderkey] --------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q10.out b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q10.out index e47bc37df32324..860280d12dd4cb 100644 --- a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q10.out +++ b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q10.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_nationkey = nation.n_nationkey)) otherCondition=() --------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q11.out b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q11.out index 48fbc3d1a1fe22..2c16e9fff3c711 100644 --- a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q11.out +++ b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q11.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast(sum(ps_supplycost * ps_availqty) * 0.000002 as DOUBLE)) ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF3 s_suppkey->[ps_suppkey] --------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q13.out b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q13.out index 9aefb642529297..6165574e0ee265 100644 --- a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q13.out +++ b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q13.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------hashJoin[RIGHT_OUTER_JOIN shuffle] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() ----------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q16.out b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q16.out index f04b0bc766338b..5fe9babe975e0c 100644 --- a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q16.out +++ b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q16.out @@ -5,17 +5,18 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -------------------------PhysicalOlapScan[part] -----------------PhysicalProject -------------------filter((s_comment like '%Customer%Complaints%')) ---------------------PhysicalOlapScan[supplier] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +--------------------------PhysicalOlapScan[part] +------------------PhysicalProject +--------------------filter((s_comment like '%Customer%Complaints%')) +----------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q18.out b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q18.out index f2dd337eadfe86..c84338debea515 100644 --- a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q18.out +++ b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q18.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 o_orderkey->[l_orderkey] --------------PhysicalProject @@ -16,7 +16,7 @@ PhysicalResultSink ----------------------PhysicalOlapScan[orders] apply RFs: RF0 --------------------PhysicalProject ----------------------filter((sum(l_quantity) > 300.00)) -------------------------hashAgg[LOCAL] +------------------------hashAgg[GLOBAL] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineitem] ------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q3.out b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q3.out index 36d395afccaa2d..3474f8dcf010c3 100644 --- a/regression-test/data/shape_check/tpch_sf1000/rf_prune/q3.out +++ b/regression-test/data/shape_check/tpch_sf1000/rf_prune/q3.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN colocated] hashCondition=((lineitem.l_orderkey = orders.o_orderkey)) otherCondition=() build RFs:RF1 o_orderkey->[l_orderkey] --------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/runtime_filter/test_pushdown_setop.out b/regression-test/data/shape_check/tpch_sf1000/runtime_filter/test_pushdown_setop.out index 44db4c0470f7a6..1cb60a27a39445 100644 --- a/regression-test/data/shape_check/tpch_sf1000/runtime_filter/test_pushdown_setop.out +++ b/regression-test/data/shape_check/tpch_sf1000/runtime_filter/test_pushdown_setop.out @@ -13,7 +13,7 @@ PhysicalResultSink --------------------PhysicalProject ----------------------PhysicalOlapScan[lineitem] apply RFs: RF0 --------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------PhysicalOlapScan[orders] apply RFs: RF0 RFV2: RF1 ------------PhysicalProject @@ -34,7 +34,7 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[lineitem] apply RFs: RF0 ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] +------------------hashAgg[GLOBAL] --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] apply RFs: RF0 RFV2: RF1 ------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape/q10.out b/regression-test/data/shape_check/tpch_sf1000/shape/q10.out index c13794f49a1c6e..952ec22ca66935 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape/q10.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape/q10.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF2 n_nationkey->[c_nationkey] --------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape/q11.out b/regression-test/data/shape_check/tpch_sf1000/shape/q11.out index 48fbc3d1a1fe22..2c16e9fff3c711 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape/q11.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape/q11.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast(sum(ps_supplycost * ps_availqty) * 0.000002 as DOUBLE)) ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF3 s_suppkey->[ps_suppkey] --------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape/q13.out b/regression-test/data/shape_check/tpch_sf1000/shape/q13.out index 8da72977f78cf2..ab30427a698a90 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape/q13.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape/q13.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------hashJoin[RIGHT_OUTER_JOIN shuffle] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] ----------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape/q16.out b/regression-test/data/shape_check/tpch_sf1000/shape/q16.out index f04b0bc766338b..5fe9babe975e0c 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape/q16.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape/q16.out @@ -5,17 +5,18 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -------------------------PhysicalOlapScan[part] -----------------PhysicalProject -------------------filter((s_comment like '%Customer%Complaints%')) ---------------------PhysicalOlapScan[supplier] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +--------------------------PhysicalOlapScan[part] +------------------PhysicalProject +--------------------filter((s_comment like '%Customer%Complaints%')) +----------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/shape_check/tpch_sf1000/shape/q18.out b/regression-test/data/shape_check/tpch_sf1000/shape/q18.out index 580d66d2c717c5..44d1cdb0bbb8a2 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape/q18.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape/q18.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 o_orderkey->[l_orderkey] --------------PhysicalProject @@ -16,7 +16,7 @@ PhysicalResultSink ----------------------PhysicalOlapScan[orders] apply RFs: RF0 RF1 --------------------PhysicalProject ----------------------filter((sum(l_quantity) > 300.00)) -------------------------hashAgg[LOCAL] +------------------------hashAgg[GLOBAL] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineitem] ------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape/q3.out b/regression-test/data/shape_check/tpch_sf1000/shape/q3.out index 36d395afccaa2d..3474f8dcf010c3 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape/q3.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape/q3.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN colocated] hashCondition=((lineitem.l_orderkey = orders.o_orderkey)) otherCondition=() build RFs:RF1 o_orderkey->[l_orderkey] --------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q11.out b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q11.out index dcb4189583a0f1..6afed88faa4802 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q11.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q11.out @@ -7,7 +7,7 @@ PhysicalResultSink --------PhysicalProject ----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast(sum(ps_supplycost * ps_availqty) * 0.000002 as DOUBLE)) ------------PhysicalProject ---------------hashAgg[LOCAL] +--------------hashAgg[GLOBAL] ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((supplier.s_nationkey = nation.n_nationkey)) otherCondition=() build RFs:RF3 n_nationkey->[s_nationkey] --------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q13.out b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q13.out index 8da72977f78cf2..ab30427a698a90 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q13.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q13.out @@ -8,7 +8,7 @@ PhysicalResultSink ----------PhysicalDistribute[DistributionSpecHash] ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[LOCAL] +----------------hashAgg[GLOBAL] ------------------PhysicalProject --------------------hashJoin[RIGHT_OUTER_JOIN shuffle] hashCondition=((customer.c_custkey = orders.o_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] ----------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q16.out b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q16.out index 7b04caaf3e087a..1ed947eacabb04 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q16.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q16.out @@ -5,17 +5,18 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] --------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +--------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] --------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -----------------------PhysicalOlapScan[part] +----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q18.out b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q18.out index 580d66d2c717c5..44d1cdb0bbb8a2 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q18.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q18.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF2 o_orderkey->[l_orderkey] --------------PhysicalProject @@ -16,7 +16,7 @@ PhysicalResultSink ----------------------PhysicalOlapScan[orders] apply RFs: RF0 RF1 --------------------PhysicalProject ----------------------filter((sum(l_quantity) > 300.00)) -------------------------hashAgg[LOCAL] +------------------------hashAgg[GLOBAL] --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineitem] ------------------PhysicalProject diff --git a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q3.out b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q3.out index 2fa14f8a3abf28..5e1ce9d65f8b5d 100644 --- a/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q3.out +++ b/regression-test/data/shape_check/tpch_sf1000/shape_no_stats/q3.out @@ -4,7 +4,7 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[LOCAL] +--------hashAgg[GLOBAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineitem.l_orderkey = orders.o_orderkey)) otherCondition=() build RFs:RF1 o_orderkey->[l_orderkey] --------------PhysicalProject diff --git a/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy b/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy index 4a1bca9e6748cb..fe1e71705c4bf8 100644 --- a/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy +++ b/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy @@ -56,5 +56,4 @@ suite ("k1ap2spa") { sql """set enable_stats=false;""" mv_rewrite_success_without_check_chosen("select abs(k1)+1 t,sum(abs(k2+1)) from d_table group by t order by t;", "k1ap2spa") qt_select_mv "select abs(k1)+1 t,sum(abs(k2+1)) from d_table group by t order by t;" - } diff --git a/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy b/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy index 4f13ed56882e38..8409ded4d3210e 100644 --- a/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy +++ b/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy @@ -22,6 +22,7 @@ suite ("testCountDistinctToBitmap") { // this mv rewrite would not be rewritten in RBO phase, so set TRY_IN_RBO explicitly to make case stable sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" sql """set enable_nereids_planner=true;""" + sql "set disable_nereids_rules='DISTINCT_AGGREGATE_SPLIT';" sql """ DROP TABLE IF EXISTS user_tags; """ sql """ create table user_tags ( diff --git a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy index 727cedc322f149..0030b270413ae2 100644 --- a/regression-test/suites/nereids_function_p0/agg_function/agg.groovy +++ b/regression-test/suites/nereids_function_p0/agg_function/agg.groovy @@ -28,9 +28,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), any_value(kint) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), any_value(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), any_value(kint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), any_value(kint) from fn_test''' + select count(distinct id), any_value(kint) from fn_test''' sql ''' select any_value(kint) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -40,9 +40,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), any_value(kint) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), any_value(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), any_value(kint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), any_value(kint) from fn_test''' + select count(distinct id), any_value(kint) from fn_test''' // count_by_enum_string result is unstable in one line, so just check not core sql ''' select count_by_enum(kstr) from fn_test group by kbool order by kbool''' @@ -58,9 +58,9 @@ suite("nereids_agg_fn") { qt_sql_avg_TinyInt_agg_phase_2 ''' select count(distinct id), avg(ktint) from fn_test''' qt_sql_avg_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(ktint) from fn_test group by kbool order by kbool''' qt_sql_avg_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(ktint) from fn_test''' + select count(distinct id), avg(ktint) from fn_test''' qt_sql_avg_TinyInt_gb_notnull ''' select avg(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_TinyInt_notnull ''' @@ -70,9 +70,9 @@ suite("nereids_agg_fn") { qt_sql_avg_TinyInt_agg_phase_2_notnull ''' select count(distinct id), avg(ktint) from fn_test_not_nullable''' qt_sql_avg_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(ktint) from fn_test group by kbool order by kbool''' qt_sql_avg_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(ktint) from fn_test''' + select count(distinct id), avg(ktint) from fn_test''' qt_sql_avg_SmallInt_gb ''' select avg(ksint) from fn_test group by kbool order by kbool''' @@ -83,9 +83,9 @@ suite("nereids_agg_fn") { qt_sql_avg_SmallInt_agg_phase_2 ''' select count(distinct id), avg(ksint) from fn_test''' qt_sql_avg_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(ksint) from fn_test group by kbool order by kbool''' qt_sql_avg_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(ksint) from fn_test''' + select count(distinct id), avg(ksint) from fn_test''' qt_sql_avg_SmallInt_gb_notnull ''' select avg(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_SmallInt_notnull ''' @@ -95,9 +95,9 @@ suite("nereids_agg_fn") { qt_sql_avg_SmallInt_agg_phase_2_notnull ''' select count(distinct id), avg(ksint) from fn_test_not_nullable''' qt_sql_avg_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(ksint) from fn_test group by kbool order by kbool''' qt_sql_avg_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(ksint) from fn_test''' + select count(distinct id), avg(ksint) from fn_test''' qt_sql_avg_Integer_gb ''' select avg(kint) from fn_test group by kbool order by kbool''' @@ -108,9 +108,9 @@ suite("nereids_agg_fn") { qt_sql_avg_Integer_agg_phase_2 ''' select count(distinct id), avg(kint) from fn_test''' qt_sql_avg_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kint) from fn_test group by kbool order by kbool''' qt_sql_avg_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kint) from fn_test''' + select count(distinct id), avg(kint) from fn_test''' qt_sql_avg_Integer_gb_notnull ''' select avg(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_Integer_notnull ''' @@ -120,9 +120,9 @@ suite("nereids_agg_fn") { qt_sql_avg_Integer_agg_phase_2_notnull ''' select count(distinct id), avg(kint) from fn_test_not_nullable''' qt_sql_avg_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kint) from fn_test group by kbool order by kbool''' qt_sql_avg_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kint) from fn_test''' + select count(distinct id), avg(kint) from fn_test''' qt_sql_avg_BigInt_gb ''' select avg(kbint) from fn_test group by kbool order by kbool''' @@ -133,9 +133,9 @@ suite("nereids_agg_fn") { qt_sql_avg_BigInt_agg_phase_2 ''' select count(distinct id), avg(kbint) from fn_test''' qt_sql_avg_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kbint) from fn_test group by kbool order by kbool''' qt_sql_avg_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kbint) from fn_test''' + select count(distinct id), avg(kbint) from fn_test''' qt_sql_avg_BigInt_gb_notnull ''' select avg(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_BigInt_notnull ''' @@ -145,9 +145,9 @@ suite("nereids_agg_fn") { qt_sql_avg_BigInt_agg_phase_2_notnull ''' select count(distinct id), avg(kbint) from fn_test_not_nullable''' qt_sql_avg_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kbint) from fn_test group by kbool order by kbool''' qt_sql_avg_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kbint) from fn_test''' + select count(distinct id), avg(kbint) from fn_test''' qt_sql_avg_Double_gb ''' select avg(kdbl) from fn_test group by kbool order by kbool''' @@ -158,9 +158,9 @@ suite("nereids_agg_fn") { qt_sql_avg_Double_agg_phase_2 ''' select count(distinct id), avg(kdbl) from fn_test''' qt_sql_avg_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kdbl) from fn_test''' + select count(distinct id), avg(kdbl) from fn_test''' qt_sql_avg_Double_gb_notnull ''' select avg(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_Double_notnull ''' @@ -170,9 +170,9 @@ suite("nereids_agg_fn") { qt_sql_avg_Double_agg_phase_2_notnull ''' select count(distinct id), avg(kdbl) from fn_test_not_nullable''' qt_sql_avg_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kdbl) from fn_test''' + select count(distinct id), avg(kdbl) from fn_test''' qt_sql_avg_DecimalV2_gb ''' select avg(kdcmls1) from fn_test group by kbool order by kbool''' @@ -183,9 +183,9 @@ suite("nereids_agg_fn") { qt_sql_avg_DecimalV2_agg_phase_2 ''' select count(distinct id), avg(kdcmls1) from fn_test''' qt_sql_avg_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_avg_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kdcmls1) from fn_test''' + select count(distinct id), avg(kdcmls1) from fn_test''' qt_sql_avg_DecimalV2_gb_notnull ''' select avg(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_DecimalV2_notnull ''' @@ -195,9 +195,9 @@ suite("nereids_agg_fn") { qt_sql_avg_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), avg(kdcmls1) from fn_test_not_nullable''' qt_sql_avg_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_avg_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg(kdcmls1) from fn_test''' + select count(distinct id), avg(kdcmls1) from fn_test''' qt_sql_avg_weighted_TinyInt_Double_gb ''' select avg_weighted(ktint, kdbl) from fn_test group by kbool order by kbool''' @@ -208,9 +208,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_TinyInt_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(ktint, kdbl) from fn_test''' qt_sql_avg_weighted_TinyInt_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(ktint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(ktint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_TinyInt_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(ktint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(ktint, kdbl) from fn_test''' qt_sql_avg_weighted_TinyInt_Double_gb_notnull ''' select avg_weighted(ktint, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_TinyInt_Double_notnull ''' @@ -220,9 +220,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_TinyInt_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(ktint, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_TinyInt_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(ktint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(ktint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_TinyInt_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(ktint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(ktint, kdbl) from fn_test''' qt_sql_avg_weighted_SmallInt_Double_gb ''' select avg_weighted(ksint, kdbl) from fn_test group by kbool order by kbool''' @@ -233,9 +233,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_SmallInt_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(ksint, kdbl) from fn_test''' qt_sql_avg_weighted_SmallInt_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(ksint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(ksint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_SmallInt_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(ksint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(ksint, kdbl) from fn_test''' qt_sql_avg_weighted_SmallInt_Double_gb_notnull ''' select avg_weighted(ksint, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_SmallInt_Double_notnull ''' @@ -245,9 +245,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_SmallInt_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(ksint, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_SmallInt_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(ksint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(ksint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_SmallInt_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(ksint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(ksint, kdbl) from fn_test''' qt_sql_avg_weighted_Integer_Double_gb ''' select avg_weighted(kint, kdbl) from fn_test group by kbool order by kbool''' @@ -258,9 +258,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_Integer_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(kint, kdbl) from fn_test''' qt_sql_avg_weighted_Integer_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_Integer_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kint, kdbl) from fn_test''' qt_sql_avg_weighted_Integer_Double_gb_notnull ''' select avg_weighted(kint, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_Integer_Double_notnull ''' @@ -270,9 +270,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_Integer_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(kint, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_Integer_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_Integer_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kint, kdbl) from fn_test''' qt_sql_avg_weighted_BigInt_Double_gb ''' select avg_weighted(kbint, kdbl) from fn_test group by kbool order by kbool''' @@ -283,9 +283,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_BigInt_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(kbint, kdbl) from fn_test''' qt_sql_avg_weighted_BigInt_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kbint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kbint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_BigInt_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kbint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kbint, kdbl) from fn_test''' qt_sql_avg_weighted_BigInt_Double_gb_notnull ''' select avg_weighted(kbint, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_BigInt_Double_notnull ''' @@ -295,9 +295,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_BigInt_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(kbint, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_BigInt_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kbint, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kbint, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_BigInt_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kbint, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kbint, kdbl) from fn_test''' qt_sql_avg_weighted_Float_Double_gb ''' select avg_weighted(kfloat, kdbl) from fn_test group by kbool order by kbool''' @@ -308,9 +308,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_Float_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(kfloat, kdbl) from fn_test''' qt_sql_avg_weighted_Float_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kfloat, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kfloat, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_Float_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kfloat, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kfloat, kdbl) from fn_test''' qt_sql_avg_weighted_Float_Double_gb_notnull ''' select avg_weighted(kfloat, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_Float_Double_notnull ''' @@ -320,9 +320,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_Float_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(kfloat, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_Float_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kfloat, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kfloat, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_Float_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kfloat, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kfloat, kdbl) from fn_test''' qt_sql_avg_weighted_Double_Double_gb ''' select avg_weighted(kdbl, kdbl) from fn_test group by kbool order by kbool''' @@ -333,9 +333,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_Double_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(kdbl, kdbl) from fn_test''' qt_sql_avg_weighted_Double_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kdbl, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kdbl, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_Double_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kdbl, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kdbl, kdbl) from fn_test''' qt_sql_avg_weighted_Double_Double_gb_notnull ''' select avg_weighted(kdbl, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_Double_Double_notnull ''' @@ -345,9 +345,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_Double_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(kdbl, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_Double_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kdbl, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kdbl, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_Double_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kdbl, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kdbl, kdbl) from fn_test''' qt_sql_avg_weighted_DecimalV2_Double_gb ''' select avg_weighted(kdcmls1, kdbl) from fn_test group by kbool order by kbool''' @@ -358,9 +358,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_DecimalV2_Double_agg_phase_2 ''' select count(distinct id), avg_weighted(kdcmls1, kdbl) from fn_test''' qt_sql_avg_weighted_DecimalV2_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kdcmls1, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kdcmls1, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_DecimalV2_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kdcmls1, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kdcmls1, kdbl) from fn_test''' qt_sql_avg_weighted_DecimalV2_Double_gb_notnull ''' select avg_weighted(kdcmls1, kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_avg_weighted_DecimalV2_Double_notnull ''' @@ -370,9 +370,9 @@ suite("nereids_agg_fn") { qt_sql_avg_weighted_DecimalV2_Double_agg_phase_2_notnull ''' select count(distinct id), avg_weighted(kdcmls1, kdbl) from fn_test_not_nullable''' qt_sql_avg_weighted_DecimalV2_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), avg_weighted(kdcmls1, kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), avg_weighted(kdcmls1, kdbl) from fn_test group by kbool order by kbool''' qt_sql_avg_weighted_DecimalV2_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), avg_weighted(kdcmls1, kdbl) from fn_test''' + select count(distinct id), avg_weighted(kdcmls1, kdbl) from fn_test''' qt_sql_bitmap_intersect_Bitmap_gb ''' select bitmap_intersect(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' @@ -383,9 +383,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_intersect_Bitmap_agg_phase_2 ''' select count(distinct id), bitmap_intersect(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_intersect_Bitmap_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_intersect(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_intersect(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_bitmap_intersect_Bitmap_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_intersect(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), bitmap_intersect(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_intersect_Bitmap_gb_notnull ''' select bitmap_intersect(bitmap_hash(kbint)) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_bitmap_intersect_Bitmap_notnull ''' @@ -395,9 +395,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_intersect_Bitmap_agg_phase_2_notnull ''' select count(distinct id), bitmap_intersect(bitmap_hash(kbint)) from fn_test_not_nullable''' qt_sql_bitmap_intersect_Bitmap_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_intersect(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_intersect(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_bitmap_intersect_Bitmap_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_intersect(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), bitmap_intersect(bitmap_hash(kbint)) from fn_test''' // bitmap_agg qt_sql_bitmap_agg_tinyint """ @@ -422,9 +422,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_Bitmap_agg_phase_2 ''' select count(distinct id), bitmap_union(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_union_Bitmap_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_Bitmap_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), bitmap_union(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_union_Bitmap_gb_notnull ''' select bitmap_union(bitmap_hash(kbint)) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_bitmap_union_Bitmap_notnull ''' @@ -434,9 +434,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_Bitmap_agg_phase_2_notnull ''' select count(distinct id), bitmap_union(bitmap_hash(kbint)) from fn_test_not_nullable''' qt_sql_bitmap_union_Bitmap_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_Bitmap_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), bitmap_union(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_union_count_Bitmap_gb ''' select bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' @@ -447,9 +447,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_count_Bitmap_agg_phase_2 ''' select count(distinct id), bitmap_union_count(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_union_count_Bitmap_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_count_Bitmap_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_count(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), bitmap_union_count(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_union_count_Bitmap_gb_notnull ''' select bitmap_union_count(bitmap_hash(kbint)) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_bitmap_union_count_Bitmap_notnull ''' @@ -459,9 +459,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_count_Bitmap_agg_phase_2_notnull ''' select count(distinct id), bitmap_union_count(bitmap_hash(kbint)) from fn_test_not_nullable''' qt_sql_bitmap_union_count_Bitmap_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_count_Bitmap_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_count(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), bitmap_union_count(bitmap_hash(kbint)) from fn_test''' qt_sql_bitmap_union_int_SmallInt_gb ''' select bitmap_union_int(ksint) from fn_test group by kbool order by kbool''' @@ -472,9 +472,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_int_SmallInt_agg_phase_2 ''' select count(distinct id), bitmap_union_int(ksint) from fn_test''' qt_sql_bitmap_union_int_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_int(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_int(ksint) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_int_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_int(ksint) from fn_test''' + select count(distinct id), bitmap_union_int(ksint) from fn_test''' qt_sql_bitmap_union_int_SmallInt_gb_notnull ''' select bitmap_union_int(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_bitmap_union_int_SmallInt_notnull ''' @@ -484,9 +484,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_int_SmallInt_agg_phase_2_notnull ''' select count(distinct id), bitmap_union_int(ksint) from fn_test_not_nullable''' qt_sql_bitmap_union_int_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_int(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_int(ksint) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_int_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_int(ksint) from fn_test''' + select count(distinct id), bitmap_union_int(ksint) from fn_test''' qt_sql_bitmap_union_int_TinyInt_gb ''' select bitmap_union_int(ktint) from fn_test group by kbool order by kbool''' @@ -497,9 +497,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_int_TinyInt_agg_phase_2 ''' select count(distinct id), bitmap_union_int(ktint) from fn_test''' qt_sql_bitmap_union_int_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_int(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_int(ktint) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_int_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_int(ktint) from fn_test''' + select count(distinct id), bitmap_union_int(ktint) from fn_test''' qt_sql_bitmap_union_int_TinyInt_gb_notnull ''' select bitmap_union_int(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_bitmap_union_int_TinyInt_notnull ''' @@ -509,9 +509,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_int_TinyInt_agg_phase_2_notnull ''' select count(distinct id), bitmap_union_int(ktint) from fn_test_not_nullable''' qt_sql_bitmap_union_int_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_int(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_int(ktint) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_int_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_int(ktint) from fn_test''' + select count(distinct id), bitmap_union_int(ktint) from fn_test''' qt_sql_bitmap_union_int_Integer_gb ''' select bitmap_union_int(kint) from fn_test group by kbool order by kbool''' @@ -522,9 +522,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_int_Integer_agg_phase_2 ''' select count(distinct id), bitmap_union_int(kint) from fn_test''' qt_sql_bitmap_union_int_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_int(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_int(kint) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_int_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_int(kint) from fn_test''' + select count(distinct id), bitmap_union_int(kint) from fn_test''' qt_sql_bitmap_union_int_Integer_gb_notnull ''' select bitmap_union_int(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_bitmap_union_int_Integer_notnull ''' @@ -534,9 +534,9 @@ suite("nereids_agg_fn") { qt_sql_bitmap_union_int_Integer_agg_phase_2_notnull ''' select count(distinct id), bitmap_union_int(kint) from fn_test_not_nullable''' qt_sql_bitmap_union_int_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), bitmap_union_int(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), bitmap_union_int(kint) from fn_test group by kbool order by kbool''' qt_sql_bitmap_union_int_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), bitmap_union_int(kint) from fn_test''' + select count(distinct id), bitmap_union_int(kint) from fn_test''' qt_sql_count_gb ''' select count() from fn_test group by kbool order by kbool''' @@ -547,9 +547,9 @@ suite("nereids_agg_fn") { qt_sql_count_agg_phase_2 ''' select count(distinct id), count() from fn_test''' qt_sql_count_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), count() from fn_test group by kbool order by kbool''' + select count(distinct id, kint), count() from fn_test group by kbool order by kbool''' qt_sql_count_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), count() from fn_test''' + select count(distinct id), count() from fn_test''' qt_sql_count_gb_notnull ''' select count() from fn_test_not_nullable group by kbool order by kbool''' qt_sql_count_notnull ''' @@ -559,9 +559,9 @@ suite("nereids_agg_fn") { qt_sql_count_agg_phase_2_notnull ''' select count(distinct id), count() from fn_test_not_nullable''' qt_sql_count_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), count() from fn_test group by kbool order by kbool''' + select count(distinct id, kint), count() from fn_test group by kbool order by kbool''' qt_sql_count_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), count() from fn_test''' + select count(distinct id), count() from fn_test''' qt_sql_count_array_gb ''' select count(kaint) from fn_test group by kbool order by kbool''' @@ -574,9 +574,9 @@ suite("nereids_agg_fn") { qt_sql_count_AnyData_agg_phase_2 ''' select count(distinct id), count(kint) from fn_test''' qt_sql_count_AnyData_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), count(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), count(kint) from fn_test group by kbool order by kbool''' qt_sql_count_AnyData_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), count(kint) from fn_test''' + select count(distinct id), count(kint) from fn_test''' qt_sql_count_AnyData_gb_notnull ''' select count(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_count_AnyData_notnull ''' @@ -586,9 +586,9 @@ suite("nereids_agg_fn") { qt_sql_count_AnyData_agg_phase_2_notnull ''' select count(distinct id), count(kint) from fn_test_not_nullable''' qt_sql_count_AnyData_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), count(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), count(kint) from fn_test group by kbool order by kbool''' qt_sql_count_AnyData_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), count(kint) from fn_test''' + select count(distinct id), count(kint) from fn_test''' // array intersect result is not stable, we comment them // qt_sql_group_array_intersect_array_bool ''' @@ -665,9 +665,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_TinyInt_agg_phase_2 ''' select count(distinct id), group_bit_and(ktint) from fn_test''' qt_sql_group_bit_and_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(ktint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(ktint) from fn_test''' + select count(distinct id), group_bit_and(ktint) from fn_test''' qt_sql_group_bit_and_TinyInt_gb_notnull ''' select group_bit_and(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_and_TinyInt_notnull ''' @@ -677,9 +677,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_TinyInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_and(ktint) from fn_test_not_nullable''' qt_sql_group_bit_and_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(ktint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(ktint) from fn_test''' + select count(distinct id), group_bit_and(ktint) from fn_test''' qt_sql_group_bit_and_SmallInt_gb ''' select group_bit_and(ksint) from fn_test group by kbool order by kbool''' @@ -690,9 +690,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_SmallInt_agg_phase_2 ''' select count(distinct id), group_bit_and(ksint) from fn_test''' qt_sql_group_bit_and_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(ksint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(ksint) from fn_test''' + select count(distinct id), group_bit_and(ksint) from fn_test''' qt_sql_group_bit_and_SmallInt_gb_notnull ''' select group_bit_and(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_and_SmallInt_notnull ''' @@ -702,9 +702,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_SmallInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_and(ksint) from fn_test_not_nullable''' qt_sql_group_bit_and_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(ksint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(ksint) from fn_test''' + select count(distinct id), group_bit_and(ksint) from fn_test''' qt_sql_group_bit_and_Integer_gb ''' select group_bit_and(kint) from fn_test group by kbool order by kbool''' @@ -715,9 +715,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_Integer_agg_phase_2 ''' select count(distinct id), group_bit_and(kint) from fn_test''' qt_sql_group_bit_and_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(kint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(kint) from fn_test''' + select count(distinct id), group_bit_and(kint) from fn_test''' qt_sql_group_bit_and_Integer_gb_notnull ''' select group_bit_and(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_and_Integer_notnull ''' @@ -727,9 +727,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_Integer_agg_phase_2_notnull ''' select count(distinct id), group_bit_and(kint) from fn_test_not_nullable''' qt_sql_group_bit_and_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(kint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(kint) from fn_test''' + select count(distinct id), group_bit_and(kint) from fn_test''' qt_sql_group_bit_and_BigInt_gb ''' select group_bit_and(kbint) from fn_test group by kbool order by kbool''' @@ -740,9 +740,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_BigInt_agg_phase_2 ''' select count(distinct id), group_bit_and(kbint) from fn_test''' qt_sql_group_bit_and_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(kbint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(kbint) from fn_test''' + select count(distinct id), group_bit_and(kbint) from fn_test''' qt_sql_group_bit_and_BigInt_gb_notnull ''' select group_bit_and(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_and_BigInt_notnull ''' @@ -752,9 +752,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_BigInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_and(kbint) from fn_test_not_nullable''' qt_sql_group_bit_and_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(kbint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(kbint) from fn_test''' + select count(distinct id), group_bit_and(kbint) from fn_test''' qt_sql_group_bit_and_LargeInt_gb ''' select group_bit_and(klint) from fn_test group by kbool order by kbool''' @@ -765,9 +765,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_LargeInt_agg_phase_2 ''' select count(distinct id), group_bit_and(klint) from fn_test''' qt_sql_group_bit_and_LargeInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(klint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_LargeInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(klint) from fn_test''' + select count(distinct id), group_bit_and(klint) from fn_test''' qt_sql_group_bit_and_LargeInt_gb_notnull ''' select group_bit_and(klint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_and_LargeInt_notnull ''' @@ -777,9 +777,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_and_LargeInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_and(klint) from fn_test_not_nullable''' qt_sql_group_bit_and_LargeInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_and(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_and(klint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_and_LargeInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_and(klint) from fn_test''' + select count(distinct id), group_bit_and(klint) from fn_test''' qt_sql_group_bit_or_TinyInt_gb ''' select group_bit_or(ktint) from fn_test group by kbool order by kbool''' @@ -790,9 +790,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_TinyInt_agg_phase_2 ''' select count(distinct id), group_bit_or(ktint) from fn_test''' qt_sql_group_bit_or_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(ktint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(ktint) from fn_test''' + select count(distinct id), group_bit_or(ktint) from fn_test''' qt_sql_group_bit_or_TinyInt_gb_notnull ''' select group_bit_or(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_or_TinyInt_notnull ''' @@ -802,9 +802,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_TinyInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_or(ktint) from fn_test_not_nullable''' qt_sql_group_bit_or_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(ktint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(ktint) from fn_test''' + select count(distinct id), group_bit_or(ktint) from fn_test''' qt_sql_group_bit_or_SmallInt_gb ''' select group_bit_or(ksint) from fn_test group by kbool order by kbool''' @@ -815,9 +815,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_SmallInt_agg_phase_2 ''' select count(distinct id), group_bit_or(ksint) from fn_test''' qt_sql_group_bit_or_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(ksint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(ksint) from fn_test''' + select count(distinct id), group_bit_or(ksint) from fn_test''' qt_sql_group_bit_or_SmallInt_gb_notnull ''' select group_bit_or(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_or_SmallInt_notnull ''' @@ -827,9 +827,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_SmallInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_or(ksint) from fn_test_not_nullable''' qt_sql_group_bit_or_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(ksint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(ksint) from fn_test''' + select count(distinct id), group_bit_or(ksint) from fn_test''' qt_sql_group_bit_or_Integer_gb ''' select group_bit_or(kint) from fn_test group by kbool order by kbool''' @@ -840,9 +840,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_Integer_agg_phase_2 ''' select count(distinct id), group_bit_or(kint) from fn_test''' qt_sql_group_bit_or_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(kint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(kint) from fn_test''' + select count(distinct id), group_bit_or(kint) from fn_test''' qt_sql_group_bit_or_Integer_gb_notnull ''' select group_bit_or(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_or_Integer_notnull ''' @@ -852,9 +852,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_Integer_agg_phase_2_notnull ''' select count(distinct id), group_bit_or(kint) from fn_test_not_nullable''' qt_sql_group_bit_or_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(kint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(kint) from fn_test''' + select count(distinct id), group_bit_or(kint) from fn_test''' qt_sql_group_bit_or_BigInt_gb ''' select group_bit_or(kbint) from fn_test group by kbool order by kbool''' @@ -865,9 +865,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_BigInt_agg_phase_2 ''' select count(distinct id), group_bit_or(kbint) from fn_test''' qt_sql_group_bit_or_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(kbint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(kbint) from fn_test''' + select count(distinct id), group_bit_or(kbint) from fn_test''' qt_sql_group_bit_or_BigInt_gb_notnull ''' select group_bit_or(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_or_BigInt_notnull ''' @@ -877,9 +877,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_BigInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_or(kbint) from fn_test_not_nullable''' qt_sql_group_bit_or_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(kbint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(kbint) from fn_test''' + select count(distinct id), group_bit_or(kbint) from fn_test''' qt_sql_group_bit_or_LargeInt_gb ''' select group_bit_or(klint) from fn_test group by kbool order by kbool''' @@ -890,9 +890,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_LargeInt_agg_phase_2 ''' select count(distinct id), group_bit_or(klint) from fn_test''' qt_sql_group_bit_or_LargeInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(klint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_LargeInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(klint) from fn_test''' + select count(distinct id), group_bit_or(klint) from fn_test''' qt_sql_group_bit_or_LargeInt_gb_notnull ''' select group_bit_or(klint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_or_LargeInt_notnull ''' @@ -902,9 +902,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_or_LargeInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_or(klint) from fn_test_not_nullable''' qt_sql_group_bit_or_LargeInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_or(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_or(klint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_or_LargeInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_or(klint) from fn_test''' + select count(distinct id), group_bit_or(klint) from fn_test''' qt_sql_group_bit_xor_TinyInt_gb ''' select group_bit_xor(ktint) from fn_test group by kbool order by kbool''' @@ -915,9 +915,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_TinyInt_agg_phase_2 ''' select count(distinct id), group_bit_xor(ktint) from fn_test''' qt_sql_group_bit_xor_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(ktint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(ktint) from fn_test''' + select count(distinct id), group_bit_xor(ktint) from fn_test''' qt_sql_group_bit_xor_TinyInt_gb_notnull ''' select group_bit_xor(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_xor_TinyInt_notnull ''' @@ -927,9 +927,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_TinyInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_xor(ktint) from fn_test_not_nullable''' qt_sql_group_bit_xor_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(ktint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(ktint) from fn_test''' + select count(distinct id), group_bit_xor(ktint) from fn_test''' qt_sql_group_bit_xor_SmallInt_gb ''' select group_bit_xor(ksint) from fn_test group by kbool order by kbool''' @@ -940,9 +940,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_SmallInt_agg_phase_2 ''' select count(distinct id), group_bit_xor(ksint) from fn_test''' qt_sql_group_bit_xor_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(ksint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(ksint) from fn_test''' + select count(distinct id), group_bit_xor(ksint) from fn_test''' qt_sql_group_bit_xor_SmallInt_gb_notnull ''' select group_bit_xor(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_xor_SmallInt_notnull ''' @@ -952,9 +952,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_SmallInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_xor(ksint) from fn_test_not_nullable''' qt_sql_group_bit_xor_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(ksint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(ksint) from fn_test''' + select count(distinct id), group_bit_xor(ksint) from fn_test''' qt_sql_group_bit_xor_Integer_gb ''' select group_bit_xor(kint) from fn_test group by kbool order by kbool''' @@ -965,9 +965,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_Integer_agg_phase_2 ''' select count(distinct id), group_bit_xor(kint) from fn_test''' qt_sql_group_bit_xor_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(kint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(kint) from fn_test''' + select count(distinct id), group_bit_xor(kint) from fn_test''' qt_sql_group_bit_xor_Integer_gb_notnull ''' select group_bit_xor(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_xor_Integer_notnull ''' @@ -977,9 +977,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_Integer_agg_phase_2_notnull ''' select count(distinct id), group_bit_xor(kint) from fn_test_not_nullable''' qt_sql_group_bit_xor_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(kint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(kint) from fn_test''' + select count(distinct id), group_bit_xor(kint) from fn_test''' qt_sql_group_bit_xor_BigInt_gb ''' select group_bit_xor(kbint) from fn_test group by kbool order by kbool''' @@ -990,9 +990,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_BigInt_agg_phase_2 ''' select count(distinct id), group_bit_xor(kbint) from fn_test''' qt_sql_group_bit_xor_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(kbint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(kbint) from fn_test''' + select count(distinct id), group_bit_xor(kbint) from fn_test''' qt_sql_group_bit_xor_BigInt_gb_notnull ''' select group_bit_xor(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_xor_BigInt_notnull ''' @@ -1002,9 +1002,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_BigInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_xor(kbint) from fn_test_not_nullable''' qt_sql_group_bit_xor_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(kbint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(kbint) from fn_test''' + select count(distinct id), group_bit_xor(kbint) from fn_test''' qt_sql_group_bit_xor_LargeInt_gb ''' select group_bit_xor(klint) from fn_test group by kbool order by kbool''' @@ -1015,9 +1015,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_LargeInt_agg_phase_2 ''' select count(distinct id), group_bit_xor(klint) from fn_test''' qt_sql_group_bit_xor_LargeInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(klint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_LargeInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(klint) from fn_test''' + select count(distinct id), group_bit_xor(klint) from fn_test''' qt_sql_group_bit_xor_LargeInt_gb_notnull ''' select group_bit_xor(klint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bit_xor_LargeInt_notnull ''' @@ -1027,9 +1027,9 @@ suite("nereids_agg_fn") { qt_sql_group_bit_xor_LargeInt_agg_phase_2_notnull ''' select count(distinct id), group_bit_xor(klint) from fn_test_not_nullable''' qt_sql_group_bit_xor_LargeInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bit_xor(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bit_xor(klint) from fn_test group by kbool order by kbool''' qt_sql_group_bit_xor_LargeInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bit_xor(klint) from fn_test''' + select count(distinct id), group_bit_xor(klint) from fn_test''' qt_sql_group_bitmap_xor_Bitmap_gb ''' select group_bitmap_xor(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' @@ -1040,9 +1040,9 @@ suite("nereids_agg_fn") { qt_sql_group_bitmap_xor_Bitmap_agg_phase_2 ''' select count(distinct id), group_bitmap_xor(bitmap_hash(kbint)) from fn_test''' qt_sql_group_bitmap_xor_Bitmap_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bitmap_xor(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bitmap_xor(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_group_bitmap_xor_Bitmap_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bitmap_xor(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), group_bitmap_xor(bitmap_hash(kbint)) from fn_test''' qt_sql_group_bitmap_xor_Bitmap_gb_notnull ''' select group_bitmap_xor(bitmap_hash(kbint)) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_group_bitmap_xor_Bitmap_notnull ''' @@ -1052,9 +1052,9 @@ suite("nereids_agg_fn") { qt_sql_group_bitmap_xor_Bitmap_agg_phase_2_notnull ''' select count(distinct id), group_bitmap_xor(bitmap_hash(kbint)) from fn_test_not_nullable''' qt_sql_group_bitmap_xor_Bitmap_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), group_bitmap_xor(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), group_bitmap_xor(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_group_bitmap_xor_Bitmap_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), group_bitmap_xor(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), group_bitmap_xor(bitmap_hash(kbint)) from fn_test''' // qt_sql_group_concat_Varchar_gb ''' // select group_concat(distinct cast(abs(kint) as varchar) order by abs(ksint)) from fn_test group by kbool order by kbool''' @@ -1104,9 +1104,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(ktint) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(ktint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(ktint) from fn_test''' + select count(distinct id), histogram(ktint) from fn_test''' sql ''' select histogram(ktint) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1116,9 +1116,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(ktint) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(ktint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(ktint) from fn_test''' + select count(distinct id), histogram(ktint) from fn_test''' sql ''' select histogram(ksint) from fn_test group by kbool order by kbool''' @@ -1129,9 +1129,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(ksint) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(ksint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(ksint) from fn_test''' + select count(distinct id), histogram(ksint) from fn_test''' sql ''' select histogram(ksint) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1141,9 +1141,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(ksint) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(ksint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(ksint) from fn_test''' + select count(distinct id), histogram(ksint) from fn_test''' sql ''' select histogram(kint) from fn_test group by kbool order by kbool''' @@ -1154,9 +1154,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kint) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kint) from fn_test''' + select count(distinct id), histogram(kint) from fn_test''' sql ''' select histogram(kint) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1166,9 +1166,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kint) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kint) from fn_test''' + select count(distinct id), histogram(kint) from fn_test''' sql ''' select histogram(kbint) from fn_test group by kbool order by kbool''' @@ -1179,9 +1179,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kbint) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kbint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kbint) from fn_test''' + select count(distinct id), histogram(kbint) from fn_test''' sql ''' select histogram(kbint) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1191,9 +1191,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kbint) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kbint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kbint) from fn_test''' + select count(distinct id), histogram(kbint) from fn_test''' sql ''' select histogram(klint) from fn_test group by kbool order by kbool''' @@ -1204,9 +1204,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(klint) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(klint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(klint) from fn_test''' + select count(distinct id), histogram(klint) from fn_test''' sql ''' select histogram(klint) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1216,9 +1216,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(klint) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(klint) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(klint) from fn_test''' + select count(distinct id), histogram(klint) from fn_test''' sql ''' select histogram(kfloat) from fn_test group by kbool order by kbool''' @@ -1229,9 +1229,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kfloat) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kfloat) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kfloat) from fn_test''' + select count(distinct id), histogram(kfloat) from fn_test''' sql ''' select histogram(kfloat) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1241,9 +1241,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kfloat) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kfloat) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kfloat) from fn_test''' + select count(distinct id), histogram(kfloat) from fn_test''' sql ''' select histogram(kdbl) from fn_test group by kbool order by kbool''' @@ -1254,9 +1254,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdbl) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdbl) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdbl) from fn_test''' + select count(distinct id), histogram(kdbl) from fn_test''' sql ''' select histogram(kdbl) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1266,9 +1266,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdbl) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdbl) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdbl) from fn_test''' + select count(distinct id), histogram(kdbl) from fn_test''' sql ''' select histogram(kdcmls1) from fn_test group by kbool order by kbool''' @@ -1279,9 +1279,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdcmls1) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdcmls1) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdcmls1) from fn_test''' + select count(distinct id), histogram(kdcmls1) from fn_test''' sql ''' select histogram(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1291,9 +1291,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdcmls1) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdcmls1) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdcmls1) from fn_test''' + select count(distinct id), histogram(kdcmls1) from fn_test''' sql ''' select histogram(kdt) from fn_test group by kbool order by kbool''' @@ -1304,9 +1304,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdt) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdt) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdt) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdt) from fn_test''' + select count(distinct id), histogram(kdt) from fn_test''' sql ''' select histogram(kdt) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1316,9 +1316,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdt) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdt) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdt) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdt) from fn_test''' + select count(distinct id), histogram(kdt) from fn_test''' sql ''' select histogram(kdtm) from fn_test group by kbool order by kbool''' @@ -1329,9 +1329,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdtm) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdtm) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdtm) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdtm) from fn_test''' + select count(distinct id), histogram(kdtm) from fn_test''' sql ''' select histogram(kdtm) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1341,9 +1341,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdtm) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdtm) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdtm) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdtm) from fn_test''' + select count(distinct id), histogram(kdtm) from fn_test''' sql ''' select histogram(kdtv2) from fn_test group by kbool order by kbool''' @@ -1354,9 +1354,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdtv2) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdtv2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdtv2) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdtv2) from fn_test''' + select count(distinct id), histogram(kdtv2) from fn_test''' sql ''' select histogram(kdtv2) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1366,9 +1366,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdtv2) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdtv2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdtv2) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdtv2) from fn_test''' + select count(distinct id), histogram(kdtv2) from fn_test''' sql ''' select histogram(kdtmv2s1) from fn_test group by kbool order by kbool''' @@ -1379,9 +1379,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdtmv2s1) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdtmv2s1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdtmv2s1) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdtmv2s1) from fn_test''' + select count(distinct id), histogram(kdtmv2s1) from fn_test''' sql ''' select histogram(kdtmv2s1) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1391,9 +1391,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kdtmv2s1) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kdtmv2s1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kdtmv2s1) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kdtmv2s1) from fn_test''' + select count(distinct id), histogram(kdtmv2s1) from fn_test''' sql ''' select histogram(kchrs1) from fn_test group by kbool order by kbool''' @@ -1404,9 +1404,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kchrs1) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kchrs1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kchrs1) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kchrs1) from fn_test''' + select count(distinct id), histogram(kchrs1) from fn_test''' sql ''' select histogram(kchrs1) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1416,9 +1416,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kchrs1) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kchrs1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kchrs1) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kchrs1) from fn_test''' + select count(distinct id), histogram(kchrs1) from fn_test''' sql ''' select histogram(kstr) from fn_test group by kbool order by kbool''' @@ -1429,9 +1429,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kstr) from fn_test''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kstr) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kstr) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kstr) from fn_test''' + select count(distinct id), histogram(kstr) from fn_test''' sql ''' select histogram(kstr) from fn_test_not_nullable group by kbool order by kbool''' sql ''' @@ -1441,9 +1441,9 @@ suite("nereids_agg_fn") { sql ''' select count(distinct id), histogram(kstr) from fn_test_not_nullable''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), histogram(kstr) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), histogram(kstr) from fn_test group by kbool order by kbool''' sql ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), histogram(kstr) from fn_test''' + select count(distinct id), histogram(kstr) from fn_test''' qt_sql_linear_histogram_TinyInt_gb ''' select linear_histogram(ktint, 2) from fn_test group by kbool order by kbool''' @@ -1456,9 +1456,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_TinyInt_agg_phase_2 ''' select count(distinct id), linear_histogram(ktint, 2) from fn_test''' qt_sql_linear_histogram_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(ktint, 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(ktint, 2) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(ktint, 2) from fn_test''' + select count(distinct id), linear_histogram(ktint, 2) from fn_test''' qt_sql_linear_histogram_TinyInt_gb_notnull ''' select linear_histogram(ktint, 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_TinyInt_notnull ''' @@ -1468,9 +1468,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_TinyInt_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(ktint, 2) from fn_test_not_nullable''' qt_sql_linear_histogram_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(ktint, 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(ktint, 2) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(ktint, 2) from fn_test''' + select count(distinct id), linear_histogram(ktint, 2) from fn_test''' qt_sql_linear_histogram_SmallInt_gb ''' select linear_histogram(ksint, 3) from fn_test group by kbool order by kbool''' @@ -1481,9 +1481,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_SmallInt_agg_phase_2 ''' select count(distinct id), linear_histogram(ksint, 3) from fn_test''' qt_sql_linear_histogram_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(ksint, 3) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(ksint, 3) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(ksint, 3) from fn_test''' + select count(distinct id), linear_histogram(ksint, 3) from fn_test''' qt_sql_linear_histogram_SmallInt_gb_notnull ''' select linear_histogram(ksint, 3) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_SmallInt_notnull ''' @@ -1493,9 +1493,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_SmallInt_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(ksint, 3) from fn_test_not_nullable''' qt_sql_linear_histogram_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(ksint, 3) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(ksint, 3) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(ksint, 3) from fn_test''' + select count(distinct id), linear_histogram(ksint, 3) from fn_test''' qt_sql_linear_histogram_Integer_gb ''' select linear_histogram(kint, 5) from fn_test group by kbool order by kbool''' @@ -1506,9 +1506,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_Integer_agg_phase_2 ''' select count(distinct id), linear_histogram(kint, 5) from fn_test''' qt_sql_linear_histogram_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kint, 5) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kint, 5) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kint, 5) from fn_test''' + select count(distinct id), linear_histogram(kint, 5) from fn_test''' qt_sql_linear_histogram_Integer_gb_notnull ''' select linear_histogram(kint, 5) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_Integer_notnull ''' @@ -1518,9 +1518,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_Integer_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(kint, 5) from fn_test_not_nullable''' qt_sql_linear_histogram_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kint, 5) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kint, 5) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kint, 5) from fn_test''' + select count(distinct id), linear_histogram(kint, 5) from fn_test''' qt_sql_linear_histogram_BigInt_gb ''' select linear_histogram(kbint, 10) from fn_test group by kbool order by kbool''' @@ -1531,9 +1531,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_BigInt_agg_phase_2 ''' select count(distinct id), linear_histogram(kbint, 10) from fn_test''' qt_sql_linear_histogram_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kbint, 10) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kbint, 10) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kbint, 10) from fn_test''' + select count(distinct id), linear_histogram(kbint, 10) from fn_test''' qt_sql_linear_histogram_BigInt_gb_notnull ''' select linear_histogram(kbint, 10) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_BigInt_notnull ''' @@ -1543,9 +1543,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_BigInt_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(kbint, 10) from fn_test_not_nullable''' qt_sql_linear_histogram_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kbint, 10) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kbint, 10) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kbint, 10) from fn_test''' + select count(distinct id), linear_histogram(kbint, 10) from fn_test''' qt_sql_linear_histogram_Double_gb ''' select linear_histogram(kdbl, 6.0) from fn_test group by kbool order by kbool''' @@ -1556,9 +1556,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_Double_agg_phase_2 ''' select count(distinct id), linear_histogram(kdbl, 6.0) from fn_test''' qt_sql_linear_histogram_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kdbl, 6.0) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kdbl, 6.0) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kdbl, 6.0) from fn_test''' + select count(distinct id), linear_histogram(kdbl, 6.0) from fn_test''' qt_sql_linear_histogram_Double_gb_notnull ''' select linear_histogram(kdbl, 6.0) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_Double_notnull ''' @@ -1568,9 +1568,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_Double_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(kdbl, 6.0) from fn_test_not_nullable''' qt_sql_linear_histogram_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kdbl, 6.0) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kdbl, 6.0) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kdbl, 6.0) from fn_test''' + select count(distinct id), linear_histogram(kdbl, 6.0) from fn_test''' qt_sql_linear_histogram_DecimalV2_gb ''' select linear_histogram(kdcmls1, 10) from fn_test group by kbool order by kbool''' @@ -1581,9 +1581,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_DecimalV2_agg_phase_2 ''' select count(distinct id), linear_histogram(kdcmls1, 10) from fn_test''' qt_sql_linear_histogram_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kdcmls1, 10) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kdcmls1, 10) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kdcmls1, 10) from fn_test''' + select count(distinct id), linear_histogram(kdcmls1, 10) from fn_test''' qt_sql_linear_histogram_DecimalV2_gb_notnull ''' select linear_histogram(kdcmls1, 10) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_DecimalV2_notnull ''' @@ -1593,9 +1593,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(kdcmls1, 10) from fn_test_not_nullable''' qt_sql_linear_histogram_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(kdcmls1, 10) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(kdcmls1, 10) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(kdcmls1, 10) from fn_test''' + select count(distinct id), linear_histogram(kdcmls1, 10) from fn_test''' qt_sql_linear_histogram_LargeInt_gb ''' select linear_histogram(klint, 10) from fn_test group by kbool order by kbool''' @@ -1606,9 +1606,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_LargeInt_agg_phase_2 ''' select count(distinct id), linear_histogram(klint, 10) from fn_test''' qt_sql_linear_histogram_LargeInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(klint, 10) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(klint, 10) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_LargeInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(klint, 10) from fn_test''' + select count(distinct id), linear_histogram(klint, 10) from fn_test''' qt_sql_linear_histogram_LargeInt_gb_notnull ''' select linear_histogram(klint, 10) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_linear_histogram_LargeInt_notnull ''' @@ -1618,9 +1618,9 @@ suite("nereids_agg_fn") { qt_sql_linear_histogram_LargeInt_agg_phase_2_notnull ''' select count(distinct id), linear_histogram(klint, 10) from fn_test_not_nullable''' qt_sql_linear_histogram_LargeInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), linear_histogram(klint, 10) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), linear_histogram(klint, 10) from fn_test group by kbool order by kbool''' qt_sql_linear_histogram_LargeInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), linear_histogram(klint, 10) from fn_test''' + select count(distinct id), linear_histogram(klint, 10) from fn_test''' qt_sql_max_by_AnyData_AnyData_gb ''' select max_by(kint, kint) from fn_test group by kbool order by kbool''' @@ -1631,9 +1631,9 @@ suite("nereids_agg_fn") { qt_sql_max_by_AnyData_AnyData_agg_phase_2 ''' select count(distinct id), max_by(kint, kint) from fn_test''' qt_sql_max_by_AnyData_AnyData_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), max_by(kint, kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), max_by(kint, kint) from fn_test group by kbool order by kbool''' qt_sql_max_by_AnyData_AnyData_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), max_by(kint, kint) from fn_test''' + select count(distinct id), max_by(kint, kint) from fn_test''' qt_sql_max_by_AnyData_AnyData_gb_notnull ''' select max_by(kint, kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_max_by_AnyData_AnyData_notnull ''' @@ -1643,9 +1643,9 @@ suite("nereids_agg_fn") { qt_sql_max_by_AnyData_AnyData_agg_phase_2_notnull ''' select count(distinct id), max_by(kint, kint) from fn_test_not_nullable''' qt_sql_max_by_AnyData_AnyData_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), max_by(kint, kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), max_by(kint, kint) from fn_test group by kbool order by kbool''' qt_sql_max_by_AnyData_AnyData_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), max_by(kint, kint) from fn_test''' + select count(distinct id), max_by(kint, kint) from fn_test''' qt_sql_min_by_AnyData_AnyData_gb ''' select min_by(kint, kint) from fn_test group by kbool order by kbool''' @@ -1656,9 +1656,9 @@ suite("nereids_agg_fn") { qt_sql_min_by_AnyData_AnyData_agg_phase_2 ''' select count(distinct id), min_by(kint, kint) from fn_test''' qt_sql_min_by_AnyData_AnyData_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), min_by(kint, kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), min_by(kint, kint) from fn_test group by kbool order by kbool''' qt_sql_min_by_AnyData_AnyData_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), min_by(kint, kint) from fn_test''' + select count(distinct id), min_by(kint, kint) from fn_test''' qt_sql_min_by_AnyData_AnyData_gb_notnull ''' select min_by(kint, kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_min_by_AnyData_AnyData_notnull ''' @@ -1668,9 +1668,9 @@ suite("nereids_agg_fn") { qt_sql_min_by_AnyData_AnyData_agg_phase_2_notnull ''' select count(distinct id), min_by(kint, kint) from fn_test_not_nullable''' qt_sql_min_by_AnyData_AnyData_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), min_by(kint, kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), min_by(kint, kint) from fn_test group by kbool order by kbool''' qt_sql_min_by_AnyData_AnyData_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), min_by(kint, kint) from fn_test''' + select count(distinct id), min_by(kint, kint) from fn_test''' qt_sql_ndv_AnyData_gb ''' select ndv(kint) from fn_test group by kbool order by kbool''' @@ -1681,9 +1681,9 @@ suite("nereids_agg_fn") { qt_sql_ndv_AnyData_agg_phase_2 ''' select count(distinct id), ndv(kint) from fn_test''' qt_sql_ndv_AnyData_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), ndv(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), ndv(kint) from fn_test group by kbool order by kbool''' qt_sql_ndv_AnyData_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), ndv(kint) from fn_test''' + select count(distinct id), ndv(kint) from fn_test''' qt_sql_ndv_AnyData_gb_notnull ''' select ndv(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_ndv_AnyData_notnull ''' @@ -1693,9 +1693,9 @@ suite("nereids_agg_fn") { qt_sql_ndv_AnyData_agg_phase_2_notnull ''' select count(distinct id), ndv(kint) from fn_test_not_nullable''' qt_sql_ndv_AnyData_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), ndv(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), ndv(kint) from fn_test group by kbool order by kbool''' qt_sql_ndv_AnyData_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), ndv(kint) from fn_test''' + select count(distinct id), ndv(kint) from fn_test''' // this function result is unstable // qt_sql_orthogonal_bitmap_union_count_Bitmap_gb ''' @@ -1707,9 +1707,9 @@ suite("nereids_agg_fn") { // qt_sql_orthogonal_bitmap_union_count_Bitmap_agg_phase_2 ''' // select count(distinct id), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test''' // qt_sql_orthogonal_bitmap_union_count_Bitmap_agg_phase_3 ''' - // select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + // select count(distinct id, kint), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_orthogonal_bitmap_union_count_Bitmap_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test''' // this function result is unstable // qt_sql_orthogonal_bitmap_union_count_Bitmap_gb_notnull ''' // select orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test_not_nullable group by kbool order by kbool''' @@ -1720,9 +1720,9 @@ suite("nereids_agg_fn") { // qt_sql_orthogonal_bitmap_union_count_Bitmap_agg_phase_2_notnull ''' // select count(distinct id), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test_not_nullable''' // qt_sql_orthogonal_bitmap_union_count_Bitmap_agg_phase_3_notnull ''' - // select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' + // select count(distinct id, kint), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test group by kbool order by kbool''' qt_sql_orthogonal_bitmap_union_count_Bitmap_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test''' + select count(distinct id), orthogonal_bitmap_union_count(bitmap_hash(kbint)) from fn_test''' qt_sql_percentile_BigInt_Double_gb ''' select percentile(kbint, 0.6) from fn_test group by kbool order by kbool''' @@ -1735,9 +1735,9 @@ suite("nereids_agg_fn") { qt_sql_percentile_BigInt_Double_agg_phase_2 ''' select count(distinct id), percentile(kbint, 0.6) from fn_test''' qt_sql_percentile_BigInt_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), percentile(kbint, 0.6) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), percentile(kbint, 0.6) from fn_test group by kbool order by kbool''' qt_sql_percentile_BigInt_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), percentile(kbint, 0.6) from fn_test''' + select count(distinct id), percentile(kbint, 0.6) from fn_test''' qt_sql_percentile_BigInt_Double_gb_notnull ''' select percentile(kbint, 0.6) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_percentile_BigInt_Double_notnull ''' @@ -1747,9 +1747,9 @@ suite("nereids_agg_fn") { qt_sql_percentile_BigInt_Double_agg_phase_2_notnull ''' select count(distinct id), percentile(kbint, 0.6) from fn_test_not_nullable''' qt_sql_percentile_BigInt_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), percentile(kbint, 0.6) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), percentile(kbint, 0.6) from fn_test group by kbool order by kbool''' qt_sql_percentile_BigInt_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), percentile(kbint, 0.6) from fn_test''' + select count(distinct id), percentile(kbint, 0.6) from fn_test''' qt_sql_percentile_approx_Double_Double_gb ''' select percentile_approx(kdbl, 0.6) from fn_test group by kbool order by kbool''' @@ -1760,9 +1760,9 @@ suite("nereids_agg_fn") { qt_sql_percentile_approx_Double_Double_agg_phase_2 ''' select count(distinct id), percentile_approx(kdbl, 0.6) from fn_test''' qt_sql_percentile_approx_Double_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), percentile_approx(kdbl, 0.6) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), percentile_approx(kdbl, 0.6) from fn_test group by kbool order by kbool''' qt_sql_percentile_approx_Double_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), percentile_approx(kdbl, 0.6) from fn_test''' + select count(distinct id), percentile_approx(kdbl, 0.6) from fn_test''' qt_sql_percentile_approx_Double_Double_gb_notnull ''' select percentile_approx(kdbl, 0.6) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_percentile_approx_Double_Double_notnull ''' @@ -1772,9 +1772,9 @@ suite("nereids_agg_fn") { qt_sql_percentile_approx_Double_Double_agg_phase_2_notnull ''' select count(distinct id), percentile_approx(kdbl, 0.6) from fn_test_not_nullable''' qt_sql_percentile_approx_Double_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), percentile_approx(kdbl, 0.6) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), percentile_approx(kdbl, 0.6) from fn_test group by kbool order by kbool''' qt_sql_percentile_approx_Double_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), percentile_approx(kdbl, 0.6) from fn_test''' + select count(distinct id), percentile_approx(kdbl, 0.6) from fn_test''' qt_sql_percentile_approx_Double_Double_Double_gb ''' select percentile_approx(kdbl, 0.6, 4096.0) from fn_test group by kbool order by kbool''' @@ -1785,9 +1785,9 @@ suite("nereids_agg_fn") { qt_sql_percentile_approx_Double_Double_Double_agg_phase_2 ''' select count(distinct id), percentile_approx(kdbl, 0.6, 4096.0) from fn_test''' qt_sql_percentile_approx_Double_Double_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), percentile_approx(kdbl, 0.6, 4096.0) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), percentile_approx(kdbl, 0.6, 4096.0) from fn_test group by kbool order by kbool''' qt_sql_percentile_approx_Double_Double_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), percentile_approx(kdbl, 0.6, 4096.0) from fn_test''' + select count(distinct id), percentile_approx(kdbl, 0.6, 4096.0) from fn_test''' qt_sql_percentile_approx_Double_Double_Double_gb_notnull ''' select percentile_approx(kdbl, 0.6, 4096.0) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_percentile_approx_Double_Double_Double_notnull ''' @@ -1797,9 +1797,9 @@ suite("nereids_agg_fn") { qt_sql_percentile_approx_Double_Double_Double_agg_phase_2_notnull ''' select count(distinct id), percentile_approx(kdbl, 0.6, 4096.0) from fn_test_not_nullable''' qt_sql_percentile_approx_Double_Double_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), percentile_approx(kdbl, 0.6, 4096.0) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), percentile_approx(kdbl, 0.6, 4096.0) from fn_test group by kbool order by kbool''' qt_sql_percentile_approx_Double_Double_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), percentile_approx(kdbl, 0.6, 4096.0) from fn_test''' + select count(distinct id), percentile_approx(kdbl, 0.6, 4096.0) from fn_test''' qt_sql_sequence_count_String_DateV2_Boolean_gb ''' select sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -1810,9 +1810,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_count_String_DateV2_Boolean_agg_phase_2 ''' select count(distinct id), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_count_String_DateV2_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_count_String_DateV2_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_count_String_DateV2_Boolean_gb_notnull ''' select sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sequence_count_String_DateV2_Boolean_notnull ''' @@ -1822,9 +1822,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_count_String_DateV2_Boolean_agg_phase_2_notnull ''' select count(distinct id), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_sequence_count_String_DateV2_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_count_String_DateV2_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_count('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_count_String_DateTime_Boolean_gb ''' select sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -1835,9 +1835,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_count_String_DateTime_Boolean_agg_phase_2 ''' select count(distinct id), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_count_String_DateTime_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_count_String_DateTime_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_count_String_DateTime_Boolean_gb_notnull ''' select sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sequence_count_String_DateTime_Boolean_notnull ''' @@ -1847,9 +1847,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_count_String_DateTime_Boolean_agg_phase_2_notnull ''' select count(distinct id), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_sequence_count_String_DateTime_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_count_String_DateTime_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_count('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_count_String_DateTimeV2_Boolean_gb ''' select sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test group by kbool order by kbool''' @@ -1860,9 +1860,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_count_String_DateTimeV2_Boolean_agg_phase_2 ''' select count(distinct id), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test''' qt_sql_sequence_count_String_DateTimeV2_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test group by kbool order by kbool''' qt_sql_sequence_count_String_DateTimeV2_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test''' + select count(distinct id), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test''' qt_sql_sequence_count_String_DateTimeV2_Boolean_gb_notnull ''' select sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sequence_count_String_DateTimeV2_Boolean_notnull ''' @@ -1872,9 +1872,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_count_String_DateTimeV2_Boolean_agg_phase_2_notnull ''' select count(distinct id), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test_not_nullable''' qt_sql_sequence_count_String_DateTimeV2_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test group by kbool order by kbool''' qt_sql_sequence_count_String_DateTimeV2_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test''' + select count(distinct id), sequence_count('(?1)(?2)', kdtmv2s1, kint = 1, kint = 5) from fn_test''' qt_sql_sequence_match_String_DateV2_Boolean_gb ''' select sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -1885,9 +1885,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_match_String_DateV2_Boolean_agg_phase_2 ''' select count(distinct id), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateV2_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_match_String_DateV2_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateV2_Boolean_gb_notnull ''' select sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sequence_match_String_DateV2_Boolean_notnull ''' @@ -1897,9 +1897,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_match_String_DateV2_Boolean_agg_phase_2_notnull ''' select count(distinct id), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_sequence_match_String_DateV2_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_match_String_DateV2_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_match('(?1)(?2)', kdtv2, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateTime_Boolean_gb ''' select sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -1910,9 +1910,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_match_String_DateTime_Boolean_agg_phase_2 ''' select count(distinct id), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateTime_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_match_String_DateTime_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateTime_Boolean_gb_notnull ''' select sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sequence_match_String_DateTime_Boolean_notnull ''' @@ -1922,9 +1922,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_match_String_DateTime_Boolean_agg_phase_2_notnull ''' select count(distinct id), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_sequence_match_String_DateTime_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_match_String_DateTime_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_match('(?1)(?2)', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateTimeV2_Boolean_gb ''' select sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -1935,9 +1935,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_match_String_DateTimeV2_Boolean_agg_phase_2 ''' select count(distinct id), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateTimeV2_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_match_String_DateTimeV2_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test''' qt_sql_sequence_match_String_DateTimeV2_Boolean_gb_notnull ''' select sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sequence_match_String_DateTimeV2_Boolean_notnull ''' @@ -1947,9 +1947,9 @@ suite("nereids_agg_fn") { qt_sql_sequence_match_String_DateTimeV2_Boolean_agg_phase_2_notnull ''' select count(distinct id), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_sequence_match_String_DateTimeV2_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_sequence_match_String_DateTimeV2_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test''' + select count(distinct id), sequence_match('(?1)(?2)', kdtmv2s1, kint = 1, kint = 2) from fn_test''' qt_sql_stddev_TinyInt_gb ''' select stddev(ktint) from fn_test group by kbool order by kbool''' @@ -1960,9 +1960,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_TinyInt_agg_phase_2 ''' select count(distinct id), stddev(ktint) from fn_test''' qt_sql_stddev_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(ktint) from fn_test group by kbool order by kbool''' qt_sql_stddev_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(ktint) from fn_test''' + select count(distinct id), stddev(ktint) from fn_test''' qt_sql_stddev_TinyInt_gb_notnull ''' select stddev(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_TinyInt_notnull ''' @@ -1972,9 +1972,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_TinyInt_agg_phase_2_notnull ''' select count(distinct id), stddev(ktint) from fn_test_not_nullable''' qt_sql_stddev_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(ktint) from fn_test group by kbool order by kbool''' qt_sql_stddev_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(ktint) from fn_test''' + select count(distinct id), stddev(ktint) from fn_test''' qt_sql_stddev_SmallInt_gb ''' select stddev(ksint) from fn_test group by kbool order by kbool''' @@ -1985,9 +1985,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_SmallInt_agg_phase_2 ''' select count(distinct id), stddev(ksint) from fn_test''' qt_sql_stddev_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(ksint) from fn_test group by kbool order by kbool''' qt_sql_stddev_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(ksint) from fn_test''' + select count(distinct id), stddev(ksint) from fn_test''' qt_sql_stddev_SmallInt_gb_notnull ''' select stddev(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_SmallInt_notnull ''' @@ -1997,9 +1997,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_SmallInt_agg_phase_2_notnull ''' select count(distinct id), stddev(ksint) from fn_test_not_nullable''' qt_sql_stddev_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(ksint) from fn_test group by kbool order by kbool''' qt_sql_stddev_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(ksint) from fn_test''' + select count(distinct id), stddev(ksint) from fn_test''' qt_sql_stddev_Integer_gb ''' select stddev(kint) from fn_test group by kbool order by kbool''' @@ -2010,9 +2010,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_Integer_agg_phase_2 ''' select count(distinct id), stddev(kint) from fn_test''' qt_sql_stddev_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kint) from fn_test group by kbool order by kbool''' qt_sql_stddev_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kint) from fn_test''' + select count(distinct id), stddev(kint) from fn_test''' qt_sql_stddev_Integer_gb_notnull ''' select stddev(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_Integer_notnull ''' @@ -2022,9 +2022,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_Integer_agg_phase_2_notnull ''' select count(distinct id), stddev(kint) from fn_test_not_nullable''' qt_sql_stddev_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kint) from fn_test group by kbool order by kbool''' qt_sql_stddev_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kint) from fn_test''' + select count(distinct id), stddev(kint) from fn_test''' qt_sql_stddev_BigInt_gb ''' select stddev(kbint) from fn_test group by kbool order by kbool''' @@ -2035,9 +2035,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_BigInt_agg_phase_2 ''' select count(distinct id), stddev(kbint) from fn_test''' qt_sql_stddev_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kbint) from fn_test group by kbool order by kbool''' qt_sql_stddev_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kbint) from fn_test''' + select count(distinct id), stddev(kbint) from fn_test''' qt_sql_stddev_BigInt_gb_notnull ''' select stddev(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_BigInt_notnull ''' @@ -2047,9 +2047,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_BigInt_agg_phase_2_notnull ''' select count(distinct id), stddev(kbint) from fn_test_not_nullable''' qt_sql_stddev_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kbint) from fn_test group by kbool order by kbool''' qt_sql_stddev_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kbint) from fn_test''' + select count(distinct id), stddev(kbint) from fn_test''' qt_sql_stddev_Float_gb ''' select stddev(kfloat) from fn_test group by kbool order by kbool''' @@ -2060,9 +2060,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_Float_agg_phase_2 ''' select count(distinct id), stddev(kfloat) from fn_test''' qt_sql_stddev_Float_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kfloat) from fn_test group by kbool order by kbool''' qt_sql_stddev_Float_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kfloat) from fn_test''' + select count(distinct id), stddev(kfloat) from fn_test''' qt_sql_stddev_Float_gb_notnull ''' select stddev(kfloat) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_Float_notnull ''' @@ -2072,9 +2072,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_Float_agg_phase_2_notnull ''' select count(distinct id), stddev(kfloat) from fn_test_not_nullable''' qt_sql_stddev_Float_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kfloat) from fn_test group by kbool order by kbool''' qt_sql_stddev_Float_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kfloat) from fn_test''' + select count(distinct id), stddev(kfloat) from fn_test''' qt_sql_stddev_Double_gb ''' select stddev(kdbl) from fn_test group by kbool order by kbool''' @@ -2085,9 +2085,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_Double_agg_phase_2 ''' select count(distinct id), stddev(kdbl) from fn_test''' qt_sql_stddev_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kdbl) from fn_test group by kbool order by kbool''' qt_sql_stddev_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kdbl) from fn_test''' + select count(distinct id), stddev(kdbl) from fn_test''' qt_sql_stddev_Double_gb_notnull ''' select stddev(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_Double_notnull ''' @@ -2097,9 +2097,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_Double_agg_phase_2_notnull ''' select count(distinct id), stddev(kdbl) from fn_test_not_nullable''' qt_sql_stddev_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kdbl) from fn_test group by kbool order by kbool''' qt_sql_stddev_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kdbl) from fn_test''' + select count(distinct id), stddev(kdbl) from fn_test''' qt_sql_stddev_DecimalV2_gb ''' select stddev(kdcmls1) from fn_test group by kbool order by kbool''' @@ -2110,9 +2110,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_DecimalV2_agg_phase_2 ''' select count(distinct id), stddev(kdcmls1) from fn_test''' qt_sql_stddev_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_stddev_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kdcmls1) from fn_test''' + select count(distinct id), stddev(kdcmls1) from fn_test''' qt_sql_stddev_DecimalV2_gb_notnull ''' select stddev(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' // qt_sql_stddev_DecimalV2_notnull ''' @@ -2122,9 +2122,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), stddev(kdcmls1) from fn_test_not_nullable''' qt_sql_stddev_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_stddev_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev(kdcmls1) from fn_test''' + select count(distinct id), stddev(kdcmls1) from fn_test''' qt_sql_stddev_samp_TinyInt_gb ''' select stddev_samp(ktint) from fn_test group by kbool order by kbool''' @@ -2135,9 +2135,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_TinyInt_agg_phase_2 ''' select count(distinct id), stddev_samp(ktint) from fn_test''' qt_sql_stddev_samp_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(ktint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(ktint) from fn_test''' + select count(distinct id), stddev_samp(ktint) from fn_test''' qt_sql_stddev_samp_TinyInt_gb_notnull ''' select stddev_samp(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_samp_TinyInt_notnull ''' @@ -2147,9 +2147,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_TinyInt_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(ktint) from fn_test_not_nullable''' qt_sql_stddev_samp_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(ktint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(ktint) from fn_test''' + select count(distinct id), stddev_samp(ktint) from fn_test''' qt_sql_stddev_samp_SmallInt_gb ''' select stddev_samp(ksint) from fn_test group by kbool order by kbool''' @@ -2160,9 +2160,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_SmallInt_agg_phase_2 ''' select count(distinct id), stddev_samp(ksint) from fn_test''' qt_sql_stddev_samp_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(ksint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(ksint) from fn_test''' + select count(distinct id), stddev_samp(ksint) from fn_test''' qt_sql_stddev_samp_SmallInt_gb_notnull ''' select stddev_samp(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_samp_SmallInt_notnull ''' @@ -2172,9 +2172,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_SmallInt_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(ksint) from fn_test_not_nullable''' qt_sql_stddev_samp_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(ksint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(ksint) from fn_test''' + select count(distinct id), stddev_samp(ksint) from fn_test''' qt_sql_stddev_samp_Integer_gb ''' select stddev_samp(kint) from fn_test group by kbool order by kbool''' @@ -2185,9 +2185,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_Integer_agg_phase_2 ''' select count(distinct id), stddev_samp(kint) from fn_test''' qt_sql_stddev_samp_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kint) from fn_test''' + select count(distinct id), stddev_samp(kint) from fn_test''' qt_sql_stddev_samp_Integer_gb_notnull ''' select stddev_samp(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_samp_Integer_notnull ''' @@ -2197,9 +2197,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_Integer_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(kint) from fn_test_not_nullable''' qt_sql_stddev_samp_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kint) from fn_test''' + select count(distinct id), stddev_samp(kint) from fn_test''' qt_sql_stddev_samp_BigInt_gb ''' select stddev_samp(kbint) from fn_test group by kbool order by kbool''' @@ -2210,9 +2210,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_BigInt_agg_phase_2 ''' select count(distinct id), stddev_samp(kbint) from fn_test''' qt_sql_stddev_samp_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kbint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kbint) from fn_test''' + select count(distinct id), stddev_samp(kbint) from fn_test''' qt_sql_stddev_samp_BigInt_gb_notnull ''' select stddev_samp(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_samp_BigInt_notnull ''' @@ -2222,9 +2222,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_BigInt_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(kbint) from fn_test_not_nullable''' qt_sql_stddev_samp_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kbint) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kbint) from fn_test''' + select count(distinct id), stddev_samp(kbint) from fn_test''' qt_sql_stddev_samp_Float_gb ''' select stddev_samp(kfloat) from fn_test group by kbool order by kbool''' @@ -2235,9 +2235,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_Float_agg_phase_2 ''' select count(distinct id), stddev_samp(kfloat) from fn_test''' qt_sql_stddev_samp_Float_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kfloat) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_Float_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kfloat) from fn_test''' + select count(distinct id), stddev_samp(kfloat) from fn_test''' qt_sql_stddev_samp_Float_gb_notnull ''' select stddev_samp(kfloat) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_samp_Float_notnull ''' @@ -2247,9 +2247,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_Float_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(kfloat) from fn_test_not_nullable''' qt_sql_stddev_samp_Float_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kfloat) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_Float_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kfloat) from fn_test''' + select count(distinct id), stddev_samp(kfloat) from fn_test''' qt_sql_stddev_samp_Double_gb ''' select stddev_samp(kdbl) from fn_test group by kbool order by kbool''' @@ -2260,9 +2260,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_Double_agg_phase_2 ''' select count(distinct id), stddev_samp(kdbl) from fn_test''' qt_sql_stddev_samp_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kdbl) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kdbl) from fn_test''' + select count(distinct id), stddev_samp(kdbl) from fn_test''' qt_sql_stddev_samp_Double_gb_notnull ''' select stddev_samp(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_stddev_samp_Double_notnull ''' @@ -2272,9 +2272,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_Double_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(kdbl) from fn_test_not_nullable''' qt_sql_stddev_samp_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kdbl) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kdbl) from fn_test''' + select count(distinct id), stddev_samp(kdbl) from fn_test''' // qt_sql_stddev_samp_DecimalV2_gb ''' // select stddev_samp(kdcmls1) from fn_test group by kbool order by kbool''' @@ -2285,9 +2285,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_DecimalV2_agg_phase_2 ''' select count(distinct id), stddev_samp(kdcmls1) from fn_test''' qt_sql_stddev_samp_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kdcmls1) from fn_test''' + select count(distinct id), stddev_samp(kdcmls1) from fn_test''' // qt_sql_stddev_samp_DecimalV2_gb_notnull ''' // select stddev_samp(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' // qt_sql_stddev_samp_DecimalV2_notnull ''' @@ -2297,9 +2297,9 @@ suite("nereids_agg_fn") { qt_sql_stddev_samp_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), stddev_samp(kdcmls1) from fn_test_not_nullable''' qt_sql_stddev_samp_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), stddev_samp(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), stddev_samp(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_stddev_samp_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), stddev_samp(kdcmls1) from fn_test''' + select count(distinct id), stddev_samp(kdcmls1) from fn_test''' qt_sql_sum_Boolean ''' select sum(kbool) from fn_test''' @@ -2314,9 +2314,9 @@ suite("nereids_agg_fn") { qt_sql_sum_TinyInt_agg_phase_2 ''' select count(distinct id), sum(ktint) from fn_test''' qt_sql_sum_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(ktint) from fn_test group by kbool order by kbool''' qt_sql_sum_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(ktint) from fn_test''' + select count(distinct id), sum(ktint) from fn_test''' qt_sql_sum_TinyInt_gb_notnull ''' select sum(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_TinyInt_notnull ''' @@ -2326,9 +2326,9 @@ suite("nereids_agg_fn") { qt_sql_sum_TinyInt_agg_phase_2_notnull ''' select count(distinct id), sum(ktint) from fn_test_not_nullable''' qt_sql_sum_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(ktint) from fn_test group by kbool order by kbool''' qt_sql_sum_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(ktint) from fn_test''' + select count(distinct id), sum(ktint) from fn_test''' qt_sql_sum_SmallInt_gb ''' select sum(ksint) from fn_test group by kbool order by kbool''' @@ -2339,9 +2339,9 @@ suite("nereids_agg_fn") { qt_sql_sum_SmallInt_agg_phase_2 ''' select count(distinct id), sum(ksint) from fn_test''' qt_sql_sum_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(ksint) from fn_test group by kbool order by kbool''' qt_sql_sum_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(ksint) from fn_test''' + select count(distinct id), sum(ksint) from fn_test''' qt_sql_sum_SmallInt_gb_notnull ''' select sum(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_SmallInt_notnull ''' @@ -2351,9 +2351,9 @@ suite("nereids_agg_fn") { qt_sql_sum_SmallInt_agg_phase_2_notnull ''' select count(distinct id), sum(ksint) from fn_test_not_nullable''' qt_sql_sum_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(ksint) from fn_test group by kbool order by kbool''' qt_sql_sum_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(ksint) from fn_test''' + select count(distinct id), sum(ksint) from fn_test''' qt_sql_sum_Integer_gb ''' select sum(kint) from fn_test group by kbool order by kbool''' @@ -2364,9 +2364,9 @@ suite("nereids_agg_fn") { qt_sql_sum_Integer_agg_phase_2 ''' select count(distinct id), sum(kint) from fn_test''' qt_sql_sum_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kint) from fn_test group by kbool order by kbool''' qt_sql_sum_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kint) from fn_test''' + select count(distinct id), sum(kint) from fn_test''' qt_sql_sum_Integer_gb_notnull ''' select sum(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_Integer_notnull ''' @@ -2376,9 +2376,9 @@ suite("nereids_agg_fn") { qt_sql_sum_Integer_agg_phase_2_notnull ''' select count(distinct id), sum(kint) from fn_test_not_nullable''' qt_sql_sum_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kint) from fn_test group by kbool order by kbool''' qt_sql_sum_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kint) from fn_test''' + select count(distinct id), sum(kint) from fn_test''' qt_sql_sum_BigInt_gb ''' select sum(kbint) from fn_test group by kbool order by kbool''' @@ -2389,9 +2389,9 @@ suite("nereids_agg_fn") { qt_sql_sum_BigInt_agg_phase_2 ''' select count(distinct id), sum(kbint) from fn_test''' qt_sql_sum_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kbint) from fn_test group by kbool order by kbool''' qt_sql_sum_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kbint) from fn_test''' + select count(distinct id), sum(kbint) from fn_test''' qt_sql_sum_BigInt_gb_notnull ''' select sum(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_BigInt_notnull ''' @@ -2401,9 +2401,9 @@ suite("nereids_agg_fn") { qt_sql_sum_BigInt_agg_phase_2_notnull ''' select count(distinct id), sum(kbint) from fn_test_not_nullable''' qt_sql_sum_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kbint) from fn_test group by kbool order by kbool''' qt_sql_sum_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kbint) from fn_test''' + select count(distinct id), sum(kbint) from fn_test''' //not cast float to double explain { @@ -2420,9 +2420,9 @@ suite("nereids_agg_fn") { qt_sql_sum_Double_agg_phase_2 ''' select count(distinct id), sum(kdbl) from fn_test''' qt_sql_sum_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kdbl) from fn_test group by kbool order by kbool''' qt_sql_sum_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kdbl) from fn_test''' + select count(distinct id), sum(kdbl) from fn_test''' qt_sql_sum_Double_gb_notnull ''' select sum(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_Double_notnull ''' @@ -2432,9 +2432,9 @@ suite("nereids_agg_fn") { qt_sql_sum_Double_agg_phase_2_notnull ''' select count(distinct id), sum(kdbl) from fn_test_not_nullable''' qt_sql_sum_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kdbl) from fn_test group by kbool order by kbool''' qt_sql_sum_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kdbl) from fn_test''' + select count(distinct id), sum(kdbl) from fn_test''' qt_sql_sum_DecimalV2_gb ''' select sum(kdcmls1) from fn_test group by kbool order by kbool''' @@ -2445,9 +2445,9 @@ suite("nereids_agg_fn") { qt_sql_sum_DecimalV2_agg_phase_2 ''' select count(distinct id), sum(kdcmls1) from fn_test''' qt_sql_sum_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_sum_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kdcmls1) from fn_test''' + select count(distinct id), sum(kdcmls1) from fn_test''' qt_sql_sum_DecimalV2_gb_notnull ''' select sum(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_DecimalV2_notnull ''' @@ -2457,9 +2457,9 @@ suite("nereids_agg_fn") { qt_sql_sum_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), sum(kdcmls1) from fn_test_not_nullable''' qt_sql_sum_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_sum_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(kdcmls1) from fn_test''' + select count(distinct id), sum(kdcmls1) from fn_test''' qt_sql_sum_LargeInt_gb ''' select sum(klint) from fn_test group by kbool order by kbool''' @@ -2470,9 +2470,9 @@ suite("nereids_agg_fn") { qt_sql_sum_LargeInt_agg_phase_2 ''' select count(distinct id), sum(klint) from fn_test''' qt_sql_sum_LargeInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(klint) from fn_test group by kbool order by kbool''' qt_sql_sum_LargeInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(klint) from fn_test''' + select count(distinct id), sum(klint) from fn_test''' qt_sql_sum_LargeInt_gb_notnull ''' select sum(klint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum_LargeInt_notnull ''' @@ -2482,9 +2482,9 @@ suite("nereids_agg_fn") { qt_sql_sum_LargeInt_agg_phase_2_notnull ''' select count(distinct id), sum(klint) from fn_test_not_nullable''' qt_sql_sum_LargeInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum(klint) from fn_test group by kbool order by kbool''' qt_sql_sum_LargeInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum(klint) from fn_test''' + select count(distinct id), sum(klint) from fn_test''' // sum on string like explain { @@ -2513,9 +2513,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_TinyInt_agg_phase_2 ''' select count(distinct id), sum0(ktint) from fn_test''' qt_sql_sum0_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(ktint) from fn_test group by kbool order by kbool''' qt_sql_sum0_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ktint) from fn_test''' + select count(distinct id), sum0(ktint) from fn_test''' qt_sql_sum0_TinyInt_gb_notnull ''' select sum0(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_TinyInt_notnull ''' @@ -2525,9 +2525,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_TinyInt_agg_phase_2_notnull ''' select count(distinct id), sum0(ktint) from fn_test_not_nullable''' qt_sql_sum0_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(ktint) from fn_test group by kbool order by kbool''' qt_sql_sum0_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ktint) from fn_test''' + select count(distinct id), sum0(ktint) from fn_test''' qt_sql_sum0_SmallInt_gb ''' select sum0(ksint) from fn_test group by kbool order by kbool''' @@ -2538,9 +2538,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_SmallInt_agg_phase_2 ''' select count(distinct id), sum0(ksint) from fn_test''' qt_sql_sum0_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(ksint) from fn_test group by kbool order by kbool''' qt_sql_sum0_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ksint) from fn_test''' + select count(distinct id), sum0(ksint) from fn_test''' qt_sql_sum0_SmallInt_gb_notnull ''' select sum0(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_SmallInt_notnull ''' @@ -2550,9 +2550,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_SmallInt_agg_phase_2_notnull ''' select count(distinct id), sum0(ksint) from fn_test_not_nullable''' qt_sql_sum0_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(ksint) from fn_test group by kbool order by kbool''' qt_sql_sum0_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(ksint) from fn_test''' + select count(distinct id), sum0(ksint) from fn_test''' qt_sql_sum0_Integer_gb ''' select sum0(kint) from fn_test group by kbool order by kbool''' @@ -2563,9 +2563,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_Integer_agg_phase_2 ''' select count(distinct id), sum0(kint) from fn_test''' qt_sql_sum0_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kint) from fn_test group by kbool order by kbool''' qt_sql_sum0_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kint) from fn_test''' + select count(distinct id), sum0(kint) from fn_test''' qt_sql_sum0_Integer_gb_notnull ''' select sum0(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_Integer_notnull ''' @@ -2575,9 +2575,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_Integer_agg_phase_2_notnull ''' select count(distinct id), sum0(kint) from fn_test_not_nullable''' qt_sql_sum0_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kint) from fn_test group by kbool order by kbool''' qt_sql_sum0_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kint) from fn_test''' + select count(distinct id), sum0(kint) from fn_test''' qt_sql_sum0_BigInt_gb ''' select sum0(kbint) from fn_test group by kbool order by kbool''' @@ -2588,9 +2588,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_BigInt_agg_phase_2 ''' select count(distinct id), sum0(kbint) from fn_test''' qt_sql_sum0_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kbint) from fn_test group by kbool order by kbool''' qt_sql_sum0_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kbint) from fn_test''' + select count(distinct id), sum0(kbint) from fn_test''' qt_sql_sum0_BigInt_gb_notnull ''' select sum0(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_BigInt_notnull ''' @@ -2600,9 +2600,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_BigInt_agg_phase_2_notnull ''' select count(distinct id), sum0(kbint) from fn_test_not_nullable''' qt_sql_sum0_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kbint) from fn_test group by kbool order by kbool''' qt_sql_sum0_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kbint) from fn_test''' + select count(distinct id), sum0(kbint) from fn_test''' //not cast float to double explain { @@ -2619,9 +2619,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_Double_agg_phase_2 ''' select count(distinct id), sum0(kdbl) from fn_test''' qt_sql_sum0_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kdbl) from fn_test group by kbool order by kbool''' qt_sql_sum0_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdbl) from fn_test''' + select count(distinct id), sum0(kdbl) from fn_test''' qt_sql_sum0_Double_gb_notnull ''' select sum0(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_Double_notnull ''' @@ -2631,9 +2631,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_Double_agg_phase_2_notnull ''' select count(distinct id), sum0(kdbl) from fn_test_not_nullable''' qt_sql_sum0_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kdbl) from fn_test group by kbool order by kbool''' qt_sql_sum0_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdbl) from fn_test''' + select count(distinct id), sum0(kdbl) from fn_test''' qt_sql_sum0_DecimalV2_gb ''' select sum0(kdcmls1) from fn_test group by kbool order by kbool''' @@ -2644,9 +2644,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_DecimalV2_agg_phase_2 ''' select count(distinct id), sum0(kdcmls1) from fn_test''' qt_sql_sum0_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_sum0_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdcmls1) from fn_test''' + select count(distinct id), sum0(kdcmls1) from fn_test''' qt_sql_sum0_DecimalV2_gb_notnull ''' select sum0(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_DecimalV2_notnull ''' @@ -2656,9 +2656,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), sum0(kdcmls1) from fn_test_not_nullable''' qt_sql_sum0_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_sum0_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(kdcmls1) from fn_test''' + select count(distinct id), sum0(kdcmls1) from fn_test''' qt_sql_sum0_LargeInt_gb ''' select sum0(klint) from fn_test group by kbool order by kbool''' @@ -2669,9 +2669,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_LargeInt_agg_phase_2 ''' select count(distinct id), sum0(klint) from fn_test''' qt_sql_sum0_LargeInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(klint) from fn_test group by kbool order by kbool''' qt_sql_sum0_LargeInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(klint) from fn_test''' + select count(distinct id), sum0(klint) from fn_test''' qt_sql_sum0_LargeInt_gb_notnull ''' select sum0(klint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_sum0_LargeInt_notnull ''' @@ -2681,9 +2681,9 @@ suite("nereids_agg_fn") { qt_sql_sum0_LargeInt_agg_phase_2_notnull ''' select count(distinct id), sum0(klint) from fn_test_not_nullable''' qt_sql_sum0_LargeInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), sum0(klint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), sum0(klint) from fn_test group by kbool order by kbool''' qt_sql_sum0_LargeInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), sum0(klint) from fn_test''' + select count(distinct id), sum0(klint) from fn_test''' // sum on string like explain { @@ -2708,9 +2708,9 @@ suite("nereids_agg_fn") { qt_sql_topn_Varchar_Integer_agg_phase_2 ''' select count(distinct id), topn(kvchrs1, 3) from fn_test''' qt_sql_topn_Varchar_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kvchrs1, 3) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kvchrs1, 3) from fn_test group by kbool order by kbool''' qt_sql_topn_Varchar_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kvchrs1, 3) from fn_test''' + select count(distinct id), topn(kvchrs1, 3) from fn_test''' qt_sql_topn_Varchar_Integer_gb_notnull ''' select topn(kvchrs1, 3) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_topn_Varchar_Integer_notnull ''' @@ -2720,9 +2720,9 @@ suite("nereids_agg_fn") { qt_sql_topn_Varchar_Integer_agg_phase_2_notnull ''' select count(distinct id), topn(kvchrs1, 3) from fn_test_not_nullable''' qt_sql_topn_Varchar_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kvchrs1, 3) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kvchrs1, 3) from fn_test group by kbool order by kbool''' qt_sql_topn_Varchar_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kvchrs1, 3) from fn_test''' + select count(distinct id), topn(kvchrs1, 3) from fn_test''' qt_sql_topn_String_Integer_gb ''' select topn(kstr, 3) from fn_test group by kbool order by kbool''' @@ -2733,9 +2733,9 @@ suite("nereids_agg_fn") { qt_sql_topn_String_Integer_agg_phase_2 ''' select count(distinct id), topn(kstr, 3) from fn_test''' qt_sql_topn_String_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kstr, 3) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kstr, 3) from fn_test group by kbool order by kbool''' qt_sql_topn_String_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kstr, 3) from fn_test''' + select count(distinct id), topn(kstr, 3) from fn_test''' qt_sql_topn_String_Integer_gb_notnull ''' select topn(kstr, 3) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_topn_String_Integer_notnull ''' @@ -2745,9 +2745,9 @@ suite("nereids_agg_fn") { qt_sql_topn_String_Integer_agg_phase_2_notnull ''' select count(distinct id), topn(kstr, 3) from fn_test_not_nullable''' qt_sql_topn_String_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kstr, 3) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kstr, 3) from fn_test group by kbool order by kbool''' qt_sql_topn_String_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kstr, 3) from fn_test''' + select count(distinct id), topn(kstr, 3) from fn_test''' qt_sql_topn_Varchar_Integer_Integer_gb ''' select topn(kvchrs1, 3, 100) from fn_test group by kbool order by kbool''' @@ -2758,9 +2758,9 @@ suite("nereids_agg_fn") { qt_sql_topn_Varchar_Integer_Integer_agg_phase_2 ''' select count(distinct id), topn(kvchrs1, 3, 100) from fn_test''' qt_sql_topn_Varchar_Integer_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kvchrs1, 3, 100) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kvchrs1, 3, 100) from fn_test group by kbool order by kbool''' qt_sql_topn_Varchar_Integer_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kvchrs1, 3, 100) from fn_test''' + select count(distinct id), topn(kvchrs1, 3, 100) from fn_test''' qt_sql_topn_Varchar_Integer_Integer_gb_notnull ''' select topn(kvchrs1, 3, 100) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_topn_Varchar_Integer_Integer_notnull ''' @@ -2770,9 +2770,9 @@ suite("nereids_agg_fn") { qt_sql_topn_Varchar_Integer_Integer_agg_phase_2_notnull ''' select count(distinct id), topn(kvchrs1, 3, 100) from fn_test_not_nullable''' qt_sql_topn_Varchar_Integer_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kvchrs1, 3, 100) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kvchrs1, 3, 100) from fn_test group by kbool order by kbool''' qt_sql_topn_Varchar_Integer_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kvchrs1, 3, 100) from fn_test''' + select count(distinct id), topn(kvchrs1, 3, 100) from fn_test''' qt_sql_topn_String_Integer_Integer_gb ''' select topn(kstr, 3, 100) from fn_test group by kbool order by kbool''' @@ -2783,9 +2783,9 @@ suite("nereids_agg_fn") { qt_sql_topn_String_Integer_Integer_agg_phase_2 ''' select count(distinct id), topn(kstr, 3, 100) from fn_test''' qt_sql_topn_String_Integer_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kstr, 3, 100) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kstr, 3, 100) from fn_test group by kbool order by kbool''' qt_sql_topn_String_Integer_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kstr, 3, 100) from fn_test''' + select count(distinct id), topn(kstr, 3, 100) from fn_test''' qt_sql_topn_String_Integer_Integer_gb_notnull ''' select topn(kstr, 3, 100) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_topn_String_Integer_Integer_notnull ''' @@ -2795,9 +2795,9 @@ suite("nereids_agg_fn") { qt_sql_topn_String_Integer_Integer_agg_phase_2_notnull ''' select count(distinct id), topn(kstr, 3, 100) from fn_test_not_nullable''' qt_sql_topn_String_Integer_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), topn(kstr, 3, 100) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), topn(kstr, 3, 100) from fn_test group by kbool order by kbool''' qt_sql_topn_String_Integer_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), topn(kstr, 3, 100) from fn_test''' + select count(distinct id), topn(kstr, 3, 100) from fn_test''' qt_sql_variance_TinyInt_gb ''' select variance(ktint) from fn_test group by kbool order by kbool''' @@ -2808,9 +2808,9 @@ suite("nereids_agg_fn") { qt_sql_variance_TinyInt_agg_phase_2 ''' select count(distinct id), variance(ktint) from fn_test''' qt_sql_variance_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(ktint) from fn_test group by kbool order by kbool''' qt_sql_variance_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(ktint) from fn_test''' + select count(distinct id), variance(ktint) from fn_test''' qt_sql_variance_TinyInt_gb_notnull ''' select variance(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_TinyInt_notnull ''' @@ -2820,9 +2820,9 @@ suite("nereids_agg_fn") { qt_sql_variance_TinyInt_agg_phase_2_notnull ''' select count(distinct id), variance(ktint) from fn_test_not_nullable''' qt_sql_variance_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(ktint) from fn_test group by kbool order by kbool''' qt_sql_variance_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(ktint) from fn_test''' + select count(distinct id), variance(ktint) from fn_test''' qt_sql_variance_SmallInt_gb ''' select variance(ksint) from fn_test group by kbool order by kbool''' @@ -2833,9 +2833,9 @@ suite("nereids_agg_fn") { qt_sql_variance_SmallInt_agg_phase_2 ''' select count(distinct id), variance(ksint) from fn_test''' qt_sql_variance_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(ksint) from fn_test group by kbool order by kbool''' qt_sql_variance_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(ksint) from fn_test''' + select count(distinct id), variance(ksint) from fn_test''' qt_sql_variance_SmallInt_gb_notnull ''' select variance(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_SmallInt_notnull ''' @@ -2845,9 +2845,9 @@ suite("nereids_agg_fn") { qt_sql_variance_SmallInt_agg_phase_2_notnull ''' select count(distinct id), variance(ksint) from fn_test_not_nullable''' qt_sql_variance_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(ksint) from fn_test group by kbool order by kbool''' qt_sql_variance_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(ksint) from fn_test''' + select count(distinct id), variance(ksint) from fn_test''' qt_sql_variance_Integer_gb ''' select variance(kint) from fn_test group by kbool order by kbool''' @@ -2858,9 +2858,9 @@ suite("nereids_agg_fn") { qt_sql_variance_Integer_agg_phase_2 ''' select count(distinct id), variance(kint) from fn_test''' qt_sql_variance_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kint) from fn_test group by kbool order by kbool''' qt_sql_variance_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kint) from fn_test''' + select count(distinct id), variance(kint) from fn_test''' qt_sql_variance_Integer_gb_notnull ''' select variance(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_Integer_notnull ''' @@ -2870,9 +2870,9 @@ suite("nereids_agg_fn") { qt_sql_variance_Integer_agg_phase_2_notnull ''' select count(distinct id), variance(kint) from fn_test_not_nullable''' qt_sql_variance_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kint) from fn_test group by kbool order by kbool''' qt_sql_variance_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kint) from fn_test''' + select count(distinct id), variance(kint) from fn_test''' qt_sql_variance_BigInt_gb ''' select variance(kbint) from fn_test group by kbool order by kbool''' @@ -2883,9 +2883,9 @@ suite("nereids_agg_fn") { qt_sql_variance_BigInt_agg_phase_2 ''' select count(distinct id), variance(kbint) from fn_test''' qt_sql_variance_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kbint) from fn_test group by kbool order by kbool''' qt_sql_variance_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kbint) from fn_test''' + select count(distinct id), variance(kbint) from fn_test''' qt_sql_variance_BigInt_gb_notnull ''' select variance(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_BigInt_notnull ''' @@ -2895,9 +2895,9 @@ suite("nereids_agg_fn") { qt_sql_variance_BigInt_agg_phase_2_notnull ''' select count(distinct id), variance(kbint) from fn_test_not_nullable''' qt_sql_variance_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kbint) from fn_test group by kbool order by kbool''' qt_sql_variance_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kbint) from fn_test''' + select count(distinct id), variance(kbint) from fn_test''' qt_sql_variance_Float_gb ''' select variance(kfloat) from fn_test group by kbool order by kbool''' @@ -2908,9 +2908,9 @@ suite("nereids_agg_fn") { qt_sql_variance_Float_agg_phase_2 ''' select count(distinct id), variance(kfloat) from fn_test''' qt_sql_variance_Float_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kfloat) from fn_test group by kbool order by kbool''' qt_sql_variance_Float_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kfloat) from fn_test''' + select count(distinct id), variance(kfloat) from fn_test''' qt_sql_variance_Float_gb_notnull ''' select variance(kfloat) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_Float_notnull ''' @@ -2920,9 +2920,9 @@ suite("nereids_agg_fn") { qt_sql_variance_Float_agg_phase_2_notnull ''' select count(distinct id), variance(kfloat) from fn_test_not_nullable''' qt_sql_variance_Float_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kfloat) from fn_test group by kbool order by kbool''' qt_sql_variance_Float_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kfloat) from fn_test''' + select count(distinct id), variance(kfloat) from fn_test''' qt_sql_variance_Double_gb ''' select variance(kdbl) from fn_test group by kbool order by kbool''' @@ -2933,9 +2933,9 @@ suite("nereids_agg_fn") { qt_sql_variance_Double_agg_phase_2 ''' select count(distinct id), variance(kdbl) from fn_test''' qt_sql_variance_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kdbl) from fn_test group by kbool order by kbool''' qt_sql_variance_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kdbl) from fn_test''' + select count(distinct id), variance(kdbl) from fn_test''' qt_sql_variance_Double_gb_notnull ''' select variance(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_Double_notnull ''' @@ -2945,9 +2945,9 @@ suite("nereids_agg_fn") { qt_sql_variance_Double_agg_phase_2_notnull ''' select count(distinct id), variance(kdbl) from fn_test_not_nullable''' qt_sql_variance_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kdbl) from fn_test group by kbool order by kbool''' qt_sql_variance_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kdbl) from fn_test''' + select count(distinct id), variance(kdbl) from fn_test''' qt_sql_variance_DecimalV2_gb ''' select variance(kdcmls1) from fn_test group by kbool order by kbool''' @@ -2958,9 +2958,9 @@ suite("nereids_agg_fn") { qt_sql_variance_DecimalV2_agg_phase_2 ''' select count(distinct id), variance(kdcmls1) from fn_test''' qt_sql_variance_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_variance_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kdcmls1) from fn_test''' + select count(distinct id), variance(kdcmls1) from fn_test''' qt_sql_variance_DecimalV2_gb_notnull ''' select variance(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_DecimalV2_notnull ''' @@ -2970,9 +2970,9 @@ suite("nereids_agg_fn") { qt_sql_variance_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), variance(kdcmls1) from fn_test_not_nullable''' qt_sql_variance_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_variance_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance(kdcmls1) from fn_test''' + select count(distinct id), variance(kdcmls1) from fn_test''' qt_sql_variance_samp_TinyInt_gb ''' select variance_samp(ktint) from fn_test group by kbool order by kbool''' @@ -2983,9 +2983,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_TinyInt_agg_phase_2 ''' select count(distinct id), variance_samp(ktint) from fn_test''' qt_sql_variance_samp_TinyInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(ktint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_TinyInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(ktint) from fn_test''' + select count(distinct id), variance_samp(ktint) from fn_test''' qt_sql_variance_samp_TinyInt_gb_notnull ''' select variance_samp(ktint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_samp_TinyInt_notnull ''' @@ -2995,9 +2995,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_TinyInt_agg_phase_2_notnull ''' select count(distinct id), variance_samp(ktint) from fn_test_not_nullable''' qt_sql_variance_samp_TinyInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(ktint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(ktint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_TinyInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(ktint) from fn_test''' + select count(distinct id), variance_samp(ktint) from fn_test''' qt_sql_variance_samp_SmallInt_gb ''' select variance_samp(ksint) from fn_test group by kbool order by kbool''' @@ -3008,9 +3008,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_SmallInt_agg_phase_2 ''' select count(distinct id), variance_samp(ksint) from fn_test''' qt_sql_variance_samp_SmallInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(ksint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_SmallInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(ksint) from fn_test''' + select count(distinct id), variance_samp(ksint) from fn_test''' qt_sql_variance_samp_SmallInt_gb_notnull ''' select variance_samp(ksint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_samp_SmallInt_notnull ''' @@ -3020,9 +3020,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_SmallInt_agg_phase_2_notnull ''' select count(distinct id), variance_samp(ksint) from fn_test_not_nullable''' qt_sql_variance_samp_SmallInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(ksint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(ksint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_SmallInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(ksint) from fn_test''' + select count(distinct id), variance_samp(ksint) from fn_test''' qt_sql_variance_samp_Integer_gb ''' select variance_samp(kint) from fn_test group by kbool order by kbool''' @@ -3033,9 +3033,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_Integer_agg_phase_2 ''' select count(distinct id), variance_samp(kint) from fn_test''' qt_sql_variance_samp_Integer_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_Integer_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kint) from fn_test''' + select count(distinct id), variance_samp(kint) from fn_test''' qt_sql_variance_samp_Integer_gb_notnull ''' select variance_samp(kint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_samp_Integer_notnull ''' @@ -3045,9 +3045,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_Integer_agg_phase_2_notnull ''' select count(distinct id), variance_samp(kint) from fn_test_not_nullable''' qt_sql_variance_samp_Integer_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_Integer_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kint) from fn_test''' + select count(distinct id), variance_samp(kint) from fn_test''' qt_sql_variance_samp_BigInt_gb ''' select variance_samp(kbint) from fn_test group by kbool order by kbool''' @@ -3058,9 +3058,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_BigInt_agg_phase_2 ''' select count(distinct id), variance_samp(kbint) from fn_test''' qt_sql_variance_samp_BigInt_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kbint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_BigInt_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kbint) from fn_test''' + select count(distinct id), variance_samp(kbint) from fn_test''' qt_sql_variance_samp_BigInt_gb_notnull ''' select variance_samp(kbint) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_samp_BigInt_notnull ''' @@ -3070,9 +3070,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_BigInt_agg_phase_2_notnull ''' select count(distinct id), variance_samp(kbint) from fn_test_not_nullable''' qt_sql_variance_samp_BigInt_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kbint) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kbint) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_BigInt_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kbint) from fn_test''' + select count(distinct id), variance_samp(kbint) from fn_test''' qt_sql_variance_samp_Float_gb ''' select variance_samp(kfloat) from fn_test group by kbool order by kbool''' @@ -3083,9 +3083,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_Float_agg_phase_2 ''' select count(distinct id), variance_samp(kfloat) from fn_test''' qt_sql_variance_samp_Float_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kfloat) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_Float_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kfloat) from fn_test''' + select count(distinct id), variance_samp(kfloat) from fn_test''' qt_sql_variance_samp_Float_gb_notnull ''' select variance_samp(kfloat) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_samp_Float_notnull ''' @@ -3095,9 +3095,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_Float_agg_phase_2_notnull ''' select count(distinct id), variance_samp(kfloat) from fn_test_not_nullable''' qt_sql_variance_samp_Float_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kfloat) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kfloat) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_Float_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kfloat) from fn_test''' + select count(distinct id), variance_samp(kfloat) from fn_test''' qt_sql_variance_samp_Double_gb ''' select variance_samp(kdbl) from fn_test group by kbool order by kbool''' @@ -3108,9 +3108,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_Double_agg_phase_2 ''' select count(distinct id), variance_samp(kdbl) from fn_test''' qt_sql_variance_samp_Double_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kdbl) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_Double_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kdbl) from fn_test''' + select count(distinct id), variance_samp(kdbl) from fn_test''' qt_sql_variance_samp_Double_gb_notnull ''' select variance_samp(kdbl) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_variance_samp_Double_notnull ''' @@ -3120,9 +3120,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_Double_agg_phase_2_notnull ''' select count(distinct id), variance_samp(kdbl) from fn_test_not_nullable''' qt_sql_variance_samp_Double_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kdbl) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kdbl) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_Double_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kdbl) from fn_test''' + select count(distinct id), variance_samp(kdbl) from fn_test''' // qt_sql_variance_samp_DecimalV2_gb ''' // select variance_samp(kdcmls1) from fn_test group by kbool order by kbool''' @@ -3133,9 +3133,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_DecimalV2_agg_phase_2 ''' select count(distinct id), variance_samp(kdcmls1) from fn_test''' qt_sql_variance_samp_DecimalV2_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_DecimalV2_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kdcmls1) from fn_test''' + select count(distinct id), variance_samp(kdcmls1) from fn_test''' // qt_sql_variance_samp_DecimalV2_gb_notnull ''' // select variance_samp(kdcmls1) from fn_test_not_nullable group by kbool order by kbool''' // qt_sql_variance_samp_DecimalV2_notnull ''' @@ -3145,9 +3145,9 @@ suite("nereids_agg_fn") { qt_sql_variance_samp_DecimalV2_agg_phase_2_notnull ''' select count(distinct id), variance_samp(kdcmls1) from fn_test_not_nullable''' qt_sql_variance_samp_DecimalV2_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), variance_samp(kdcmls1) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), variance_samp(kdcmls1) from fn_test group by kbool order by kbool''' qt_sql_variance_samp_DecimalV2_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), variance_samp(kdcmls1) from fn_test''' + select count(distinct id), variance_samp(kdcmls1) from fn_test''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_gb ''' select window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -3158,9 +3158,9 @@ suite("nereids_agg_fn") { qt_sql_window_funnel_BigInt_String_DateTime_Boolean_agg_phase_2 ''' select count(distinct id), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test''' + select count(distinct id), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_gb_notnull ''' select window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_notnull ''' @@ -3170,9 +3170,9 @@ suite("nereids_agg_fn") { qt_sql_window_funnel_BigInt_String_DateTime_Boolean_agg_phase_2_notnull ''' select count(distinct id), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_window_funnel_BigInt_String_DateTime_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test''' + select count(distinct id), window_funnel(3600 * 3, 'default', kdtm, kint = 1, kint = 2) from fn_test''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_gb ''' select window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' @@ -3183,9 +3183,9 @@ suite("nereids_agg_fn") { qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_2 ''' select count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_3 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_4 ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test''' + select count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_gb_notnull ''' select window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test_not_nullable group by kbool order by kbool''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_notnull ''' @@ -3195,9 +3195,9 @@ suite("nereids_agg_fn") { qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_2_notnull ''' select count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test_not_nullable''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_3_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' + select count(distinct id, kint), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test group by kbool order by kbool''' qt_sql_window_funnel_BigInt_String_DateTimeV2_Boolean_agg_phase_4_notnull ''' - select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT, TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test''' + select count(distinct id), window_funnel(3600 * 3, 'default', kdtmv2s1, kint = 1, kint = 2) from fn_test''' qt_max_null "select max(null) from fn_test;" diff --git a/regression-test/suites/nereids_p0/aggregate/aggregate.groovy b/regression-test/suites/nereids_p0/aggregate/aggregate.groovy index 0ad3af75873a04..19d56339ce1242 100644 --- a/regression-test/suites/nereids_p0/aggregate/aggregate.groovy +++ b/regression-test/suites/nereids_p0/aggregate/aggregate.groovy @@ -366,7 +366,6 @@ suite("aggregate") { sql "insert into test_four_phase_full_distribute values(1, 21, 'hello'), (2, 22, 'world')" sql " sync " order_qt_four_phase_full_distribute """select - /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT,THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI,THREE_PHASE_AGGREGATE_WITH_DISTINCT,FOUR_PHASE_AGGREGATE_WITH_DISTINCT')*/ name, count(distinct name), count(distinct age) from test_four_phase_full_distribute group by name diff --git a/regression-test/suites/nereids_rules_p0/adjust_nullable/test_adjust_nullable.groovy b/regression-test/suites/nereids_rules_p0/adjust_nullable/test_adjust_nullable.groovy index 72737cae115a48..ea889e3b039352 100644 --- a/regression-test/suites/nereids_rules_p0/adjust_nullable/test_adjust_nullable.groovy +++ b/regression-test/suites/nereids_rules_p0/adjust_nullable/test_adjust_nullable.groovy @@ -22,6 +22,7 @@ suite('test_adjust_nullable') { def tbl = 'test_adjust_nullable_t' sql "SET detail_shape_nodes='PhysicalProject'" sql "DROP TABLE IF EXISTS ${tbl} FORCE" + sql "set runtime_filter_mode=OFF" sql "CREATE TABLE ${tbl}(a int not null, b int, c int not null) distributed by hash(a) properties('replication_num' = '1')" sql "INSERT INTO ${tbl} VALUES(1, 2, 3)" diff --git a/regression-test/suites/nereids_rules_p0/agg_strategy/agg_strategy.groovy b/regression-test/suites/nereids_rules_p0/agg_strategy/agg_strategy.groovy new file mode 100644 index 00000000000000..b4564a3563cb4f --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/agg_strategy/agg_strategy.groovy @@ -0,0 +1,141 @@ +// 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("agg_strategy") { + sql "SET ignore_shape_nodes='PhysicalProject'" + sql "set enable_parallel_result_sink=false" + sql "set global enable_auto_analyze=false" + sql "set runtime_filter_mode=OFF" + + for (int i = 0; i < 2; i++) { + if (i == 0) { + // not have statistic + sql """drop stats t_gbykey_10_dstkey_10_1000_id""" + } else { + // have statistic + sql """analyze table t_gbykey_10_dstkey_10_1000_id with sync;""" + } + qt_non_agg_func "select gby_key from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1" + qt_agg_func "select count(dst_key1), gby_key from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2" + qt_agg_distinct_with_gby_key "select count(distinct dst_key1), gby_key from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2" + qt_agg_distinct_satisfy_gby_key "select count(distinct dst_key1), id from t_gbykey_10_dstkey_10_1000_id group by id order by 1,2" + qt_agg_distinct_satisfy_dst_key "select count(distinct id) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1" + qt_agg_distinct_with_gby_key_with_other_func "select count(distinct dst_key1), gby_key, sum(dst_key2), avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2,3,4" + qt_agg_distinct_satisfy_gby_key_with_other_func "select count(distinct dst_key1), id, sum(dst_key2), avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by id order by 1,2,3,4" + qt_agg_distinct_satisfy_dst_key_with_other_func "select count(distinct id), sum(dst_key2), avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2,3" + + qt_agg_distinct_without_gby_key "select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id" + qt_agg_distinct_without_gby_key_satisfy_dst_key "select count(distinct id) from t_gbykey_10_dstkey_10_1000_id" + qt_agg_distinct_without_gby_key_with_other_func "select count(distinct dst_key1),sum(dst_key1) from t_gbykey_10_dstkey_10_1000_id" + qt_agg_distinct_without_gby_key_satisfy_dst_key_with_other_func "select count(distinct id),avg(dst_key1) from t_gbykey_10_dstkey_10_1000_id" + + //shape + qt_non_agg_func "explain shape plan select gby_key from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1" + qt_agg_func "explain shape plan select count(dst_key1), gby_key from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2" + qt_agg_distinct_with_gby_key "explain shape plan select count(distinct dst_key1), gby_key from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2" + qt_agg_distinct_satisfy_gby_key "explain shape plan select count(distinct dst_key1), id from t_gbykey_10_dstkey_10_1000_id group by id order by 1,2" + qt_agg_distinct_satisfy_dst_key "explain shape plan select count(distinct id) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1" + qt_agg_distinct_with_gby_key_with_other_func "explain shape plan select count(distinct dst_key1), gby_key, sum(dst_key2), avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2,3,4" + qt_agg_distinct_satisfy_gby_key_with_other_func "explain shape plan select count(distinct dst_key1), id, sum(dst_key2), avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by id order by 1,2,3,4" + qt_agg_distinct_satisfy_dst_key_with_other_func "explain shape plan select count(distinct id), sum(dst_key2), avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2,3" + + qt_agg_distinct_without_gby_key "explain shape plan select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id" + //final multi_distinct + sum0 + qt_agg_distinct_without_gby_key_satisfy_dst_key "explain shape plan select count(distinct id) from t_gbykey_10_dstkey_10_1000_id" + qt_agg_distinct_without_gby_key_with_other_func "explain shape plan select count(distinct dst_key1),sum(dst_key1) from t_gbykey_10_dstkey_10_1000_id" + qt_agg_distinct_without_gby_key_satisfy_dst_key_with_other_func "explain shape plan select count(distinct id),avg(dst_key1) from t_gbykey_10_dstkey_10_1000_id" + } + + // count(distinct a,b) + qt_count_multi_expr "select count(distinct id, dst_key1) from t_gbykey_10_dstkey_10_1000_id group by dst_key2 order by 1;" + qt_count_multi_expr_multi_count "select count(distinct id, dst_key1), count(distinct id, dst_key2) from t_gbykey_10_dstkey_10_1000_id group by dst_key2 order by 1,2;" + + for (int i = 0; i < 2; i++) { + if (i == 0) { + // not have statistic + sql """drop stats t_gbykey_2_dstkey_10_30_id""" + } else { + // have statistic + sql """analyze table t_gbykey_2_dstkey_10_30_id with sync;""" + } + qt_non_agg_func_low_ndv "select gby_key from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1" + qt_agg_func_low_ndv "select count(dst_key1), gby_key from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2" + qt_agg_distinct_with_gby_key_low_ndv "select count(distinct dst_key1), gby_key from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2" + qt_agg_distinct_satisfy_gby_key_low_ndv "select count(distinct dst_key1), id from t_gbykey_2_dstkey_10_30_id group by id order by 1,2" + qt_agg_distinct_satisfy_dst_key_low_ndv "select count(distinct id) from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1" + qt_agg_distinct_with_gby_key_with_other_func_low_ndv "select count(distinct dst_key1), gby_key, sum(dst_key2), avg(dst_key2) from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2,3,4" + qt_agg_distinct_satisfy_gby_key_with_other_func_low_ndv "select count(distinct dst_key1), id, sum(dst_key2), avg(dst_key2) from t_gbykey_2_dstkey_10_30_id group by id order by 1,2,3,4" + qt_agg_distinct_satisfy_dst_key_with_other_func_low_ndv "select count(distinct id), sum(dst_key2), avg(dst_key2) from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2,3" + + qt_agg_distinct_without_gby_key_low_ndv "select count(distinct dst_key1) from t_gbykey_2_dstkey_10_30_id" + qt_agg_distinct_without_gby_key_satisfy_dst_key_low_ndv "select count(distinct id) from t_gbykey_2_dstkey_10_30_id" + qt_agg_distinct_without_gby_key_with_other_func_low_ndv "select count(distinct dst_key1),sum(dst_key1) from t_gbykey_2_dstkey_10_30_id" + qt_agg_distinct_without_gby_key_satisfy_dst_key_with_other_func_low_ndv "select count(distinct id),avg(dst_key1) from t_gbykey_2_dstkey_10_30_id" + + //shape + qt_non_agg_func_low_ndv "explain shape plan select gby_key from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1" + qt_agg_func_low_ndv "explain shape plan select count(dst_key1), gby_key from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2" + qt_agg_distinct_with_gby_key_low_ndv "explain shape plan select count(distinct dst_key1), gby_key from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2" + qt_agg_distinct_satisfy_gby_key_low_ndv "explain shape plan select count(distinct dst_key1), id from t_gbykey_2_dstkey_10_30_id group by id order by 1,2" + qt_agg_distinct_satisfy_dst_key_low_ndv "explain shape plan select count(distinct id) from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1" + qt_agg_distinct_with_gby_key_with_other_func_low_ndv "explain shape plan select count(distinct dst_key1), gby_key, sum(dst_key2), avg(dst_key2) from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2,3,4" + qt_agg_distinct_satisfy_gby_key_with_other_func_low_ndv "explain shape plan select count(distinct dst_key1), id, sum(dst_key2), avg(dst_key2) from t_gbykey_2_dstkey_10_30_id group by id order by 1,2,3,4" + qt_agg_distinct_satisfy_dst_key_with_other_func_low_ndv"explain shape plan select count(distinct id), sum(dst_key2), avg(dst_key2) from t_gbykey_2_dstkey_10_30_id group by gby_key order by 1,2,3" + qt_agg_distinct_without_gby_key_low_ndv "explain shape plan select count(distinct dst_key1) from t_gbykey_2_dstkey_10_30_id" + //use final multi_distinct + sum0 + qt_agg_distinct_without_gby_key_satisfy_dst_key_low_ndv "explain shape plan select count(distinct id) from t_gbykey_2_dstkey_10_30_id" + qt_agg_distinct_without_gby_key_with_other_func_low_ndv "explain shape plan select count(distinct dst_key1),sum(dst_key1) from t_gbykey_2_dstkey_10_30_id" + qt_agg_distinct_without_gby_key_satisfy_dst_key_with_other_func_low_ndv "explain shape plan select count(distinct id),avg(dst_key1) from t_gbykey_2_dstkey_10_30_id" + } + + // test count has multi children count(a,b) + qt_with_gby_split_in_rewrite "select count(distinct dst_key1,dst_key2) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1;" + qt_with_gby_split_in_cascades "select count(distinct dst_key1,dst_key2),avg(dst_key2) from t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1,2;" + qt_without_gby "select count(distinct dst_key1,dst_key2) from t_gbykey_10_dstkey_10_1000_id;" + qt_without_gby_satisfy "select count(distinct id,dst_key1) from t_gbykey_10_dstkey_10_1000_id;" + + qt_group_concat_with_order_by_without_gby_with_distinct "select group_concat(distinct dst_key1 order by id separator ',') FROM t_gbykey_10_dstkey_10_1000_id;" + qt_group_concat_with_order_by_with_gby_with_distinct "select group_concat(distinct dst_key1 order by id separator ',') FROM t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1;" + qt_group_concat_with_order_by_without_gby "select group_concat(dst_key1 order by id separator ',') FROM t_gbykey_10_dstkey_10_1000_id;" + qt_group_concat_with_order_by_with_gby "select group_concat(dst_key1 order by id separator ',') FROM t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1;" + qt_group_concat_with_multi_order_by_without_gby_with_distinct "select group_concat(distinct dst_key1 order by id,dst_key2 separator ',') FROM t_gbykey_10_dstkey_10_1000_id;" + qt_group_concat_with_multi_order_by_with_gby_with_distinct "select group_concat(distinct dst_key1 order by id,dst_key2 separator ',') FROM t_gbykey_10_dstkey_10_1000_id group by gby_key order by 1;" + + // final multi_distinct + sum0 + qt_final_multi_distinct_sum0_count "select count(distinct id), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id;" + qt_final_multi_distinct_sum0_sum "select sum(distinct id), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id;" + qt_final_multi_distinct_sum0_sum0 "select sum0(distinct id), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id;" + + qt_agg_recieve_hash_request_shape """ + explain shape plan + select l_shipdate, l_orderkey, O_ORDERDATE, + count(orders_left.O_ORDERDATE) over (partition by lineitem_left.L_SHIPDATE order by lineitem_left.L_ORDERKEY) as window_count + from lineitem_left + left join orders_left + on lineitem_left.l_orderkey = orders_left.o_orderkey + group by l_shipdate, l_orderkey, O_ORDERDATE;""" + + qt_agg_recieve_hash_request """select l_shipdate, l_orderkey, O_ORDERDATE, + count(orders_left.O_ORDERDATE) over (partition by lineitem_left.L_SHIPDATE order by lineitem_left.L_ORDERKEY) as window_count + from lineitem_left + left join orders_left + on lineitem_left.l_orderkey = orders_left.o_orderkey + group by l_shipdate, l_orderkey, O_ORDERDATE;""" + + qt_group_concat_distinct_key_is_varchar_and_distribute_key """explain shape plan + select group_concat(distinct dst_key1 ,' ') from t_gbykey_10_dstkey_10_1000_dst_key1;""" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/agg_strategy/distinct_agg_rewriter.groovy b/regression-test/suites/nereids_rules_p0/agg_strategy/distinct_agg_rewriter.groovy new file mode 100644 index 00000000000000..412600a70fa153 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/agg_strategy/distinct_agg_rewriter.groovy @@ -0,0 +1,35 @@ +// 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("distinct_agg_rewriter") { + multi_sql""" + SET ignore_shape_nodes='PhysicalProject'; + set runtime_filter_mode=OFF; + set enable_parallel_result_sink=false; + """ + multi_sql """ + analyze table t1000_2 with sync; + """ + qt_use_multi_phase1 """explain shape plan + select count(distinct b_5) from t1000_2 group by d_200;""" + qt_use_multi_phase2 """explain shape plan + select count(distinct b_5) from t1000_2 group by b_5;""" + qt_use_multi_phase3 """explain shape plan + select count(distinct d_200) from t1000_2 group by b_5;""" + qt_use_multi_distinct """explain shape plan + select count(distinct d_200) from t1000_2 group by a_1;""" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/agg_strategy/distinct_agg_strategy_selector.groovy b/regression-test/suites/nereids_rules_p0/agg_strategy/distinct_agg_strategy_selector.groovy new file mode 100644 index 00000000000000..45a99c1cca19ab --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/agg_strategy/distinct_agg_strategy_selector.groovy @@ -0,0 +1,42 @@ +// 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("distinct_agg_strategy_selector") { + multi_sql""" + SET ignore_shape_nodes='PhysicalProject'; + set runtime_filter_mode=OFF; + set enable_parallel_result_sink=false; + """ + multi_sql """ + analyze table t1000 with sync; + """ + qt_should_use_cte """ + explain shape plan + select count(distinct a_1) , count(distinct b_5),count(distinct c_10), count(distinct d_20) from t1000;""" + qt_should_use_multi_distinct """explain shape plan + select count(distinct a_1) , count(distinct b_5) from t1000;""" + qt_should_use_cte_with_group_by """ + explain shape plan + select count(distinct a_1) , count(distinct b_5) from t1000 group by d_20;""" + qt_should_use_multi_distinct_with_group_by """explain shape plan + select count(distinct d_20) , count(distinct b_5) from t1000 group by a_1;""" + sql "drop stats t1000" + qt_no_stats_should_use_cte """explain shape plan + select count(distinct a_1) , count(distinct b_5) from t1000;""" + qt_no_stats_should_use_cte_with_group_by """explain shape plan + select count(distinct d_20) , count(distinct b_5) from t1000 group by a_1;""" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/agg_strategy/load.groovy b/regression-test/suites/nereids_rules_p0/agg_strategy/load.groovy new file mode 100644 index 00000000000000..f953cec1bfb6cd --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/agg_strategy/load.groovy @@ -0,0 +1,126 @@ +// 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("agg_strategy") { + sql "set global enable_auto_analyze=false" + // ndv is high + sql "drop table if exists t_gbykey_10_dstkey_10_1000_id" + sql """create table t_gbykey_10_dstkey_10_1000_id(id int, gby_key int, dst_key1 int, dst_key2 int) duplicate key(id) distributed by hash(id) + buckets 32 properties('replication_num' = '1');""" + sql """INSERT INTO t_gbykey_10_dstkey_10_1000_id VALUES + (1, 3, 7, 42), + (2, 5, 9, 18), + (3, 2, 4, 76), + (4, 8, 1, 33), + (5, 6, 3, 91), + (6, 1, 5, 27), + (7, 4, 8, 64), + (8, 9, 2, 55), + (9, 7, 6, 13), + (10, 10, 10, 100);""" + + // ndv is low + sql "drop table if exists t_gbykey_2_dstkey_10_30_id" + sql """create table t_gbykey_2_dstkey_10_30_id(id int, gby_key int, dst_key1 int, dst_key2 int) duplicate key(id) distributed by hash(id) + buckets 32 properties('replication_num' = '1');""" + sql """ + INSERT INTO t_gbykey_2_dstkey_10_30_id (id, gby_key, dst_key1, dst_key2) VALUES + (0, 0, 0, 0),(1, 1, 1, 1),(0, 0, 0, 2),(1, 1, 1, 3),(0, 0, 0, 4),(1, 1, 1, 5),(0, 0, 0, 6), + (1, 1, 1, 7),(0, 0, 0, 8),(1, 1, 1, 9),(0, 0, 1, 0),(1, 1, 0, 1),(0, 0, 1, 2),(0, 1, 0, 3), + (1, 0, 1, 4),(0, 1, 0, 5),(1, 0, 1, 6),(0, 1, 0, 7),(1, 0, 1, 8),(0, 1, 0, 9),(1, 0, 0, 0), + (0, 1, 1, 1),(1, 0, 0, 2),(0, 1, 1, 3),(1, 0, 0, 4),(0, 1, 1, 5),(0, 0, 0, 6),(1, 1, 1, 7), + (0, 0, 0, 8),(1, 1, 1, 9);""" + + // agg father give add hash request + multi_sql """ + drop table if exists orders_left; + CREATE TABLE `orders_left` ( + `o_orderkey` bigint NULL, + `o_orderdate` date NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + insert into orders_left values(1,'2024-01-02'); + drop table if exists lineitem_left; + CREATE TABLE `lineitem_left` ( + `l_orderkey` bigint NULL, + `l_shipdate` date NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`l_orderkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + insert into lineitem_left values(1,'2024-01-02'); + """ + + multi_sql """ + drop table if exists t_gbykey_10_dstkey_10_1000_dst_key1; + CREATE TABLE `t_gbykey_10_dstkey_10_1000_dst_key1` ( + `id` int NULL, + `gby_key` int NULL, + `dst_key1` varchar(10) NULL, + `dst_key2` int NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`dst_key1`) BUCKETS 32 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + INSERT INTO t_gbykey_10_dstkey_10_1000_dst_key1 VALUES + (1, 3, 7, 42), + (2, 5, 9, 18), + (3, 2, 4, 76), + (4, 8, 1, 33), + (5, 6, 3, 91), + (6, 1, 5, 27), + (7, 4, 8, 64), + (8, 9, 2, 55), + (9, 7, 6, 13), + (10, 10, 10, 100);""" + + multi_sql """ + drop table if exists t1000_2; + create table t1000_2(a_1 int, b_5 int, c_10 int, d_200 int) distributed by hash(c_10) properties('replication_num'='1'); + insert into t1000_2 select 1, number%5 , number%10, number%200 from numbers("number"="1003"); + """ + + multi_sql """ + drop table if exists t1000; + create table t1000(a_1 int, b_5 int, c_10 int, d_20 int) distributed by hash(c_10) properties('replication_num'='1'); + insert into t1000 select 1, number%5 , number%10, number%20 from numbers("number"="1003"); + set multi_distinct_strategy=0; + """ + + multi_sql """ + drop table if exists t1025_skew5000; + create table t1025_skew5000(a_1 int, b_5 int, c_10 int, d_1025 int) distributed by hash(c_10) properties('replication_num'='1'); + insert into t1025_skew5000 select 1, number%5 , number%10, number from numbers("number"="1025"); + insert into t1025_skew5000 select 1, number%5 , number%10, 100 from numbers("number"="5000"); + """ + + multi_sql """ + drop table if exists t1025; + create table t1025(a_1 int, b_5 int, c_10 int, d_1025 int) distributed by hash(c_10) properties('replication_num'='1'); + insert into t1025 select 1, number%5 , number%10, number from numbers("number"="1025"); + """ +} diff --git a/regression-test/suites/nereids_rules_p0/agg_strategy/physical_agg_regulator.groovy b/regression-test/suites/nereids_rules_p0/agg_strategy/physical_agg_regulator.groovy new file mode 100644 index 00000000000000..8eae2507e4574c --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/agg_strategy/physical_agg_regulator.groovy @@ -0,0 +1,49 @@ +// 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("physical_agg_regulator_request_deriver") { + multi_sql""" + SET ignore_shape_nodes='PhysicalProject'; + set runtime_filter_mode=OFF; + set enable_parallel_result_sink=false; + """ + multi_sql """ + analyze table t1025_skew5000 with sample rows 7000 with sync; + --drop cached stats t1025; + """ + // 这个预期应该使用cte之后any + qt_skew """explain shape plan + select count(distinct a_1), count(distinct b_5) from t1025_skew5000 group by d_1025;""" + + // TODO 这个预期应该使用cte之后shuffle + multi_sql """ + analyze table t1025 with sample rows 4000 with sync; + --drop cached stats t1025; + """ + qt_not_skew """explain shape plan + select count(distinct a_1), count(distinct b_5) from t1025 group by d_1025;""" + + // request deriver + qt_request_deriver_parent_ndv_high """explain shape plan + select count(distinct b_5) from t1025 group by d_1025;""" + qt_request_deriver_parent_ndv_low """explain shape plan + select count(distinct d_1025) from t1025 group by b_5;""" + qt_split_multi_agg_use_three_phase """explain shape plan + select count(distinct b_5),avg(b_5) from t1025 group by d_1025;""" + qt_split_multi_agg_use_four_phase """explain shape plan + select count(distinct d_1025),avg(b_5) from t1025 group by b_5;""" +} diff --git a/regression-test/suites/nereids_rules_p0/agg_strategy/test_variables.groovy b/regression-test/suites/nereids_rules_p0/agg_strategy/test_variables.groovy new file mode 100644 index 00000000000000..43f04f778292b2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/agg_strategy/test_variables.groovy @@ -0,0 +1,129 @@ +// 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("agg_strategy_variable") { + sql "SET ignore_shape_nodes='PhysicalProject'" + sql "set enable_parallel_result_sink=false" + sql "set global enable_auto_analyze=false" + sql "set runtime_filter_mode=OFF" + + sql "drop table if exists t_gbykey_10_dstkey_10_1000_id_2" + sql """create table t_gbykey_10_dstkey_10_1000_id_2(id int, gby_key int, dst_key1 int, dst_key2 int) duplicate key(id) distributed by hash(id) + buckets 32 properties('replication_num' = '1');""" + sql """INSERT INTO t_gbykey_10_dstkey_10_1000_id_2 VALUES + (1, 3, 7, 42), + (2, 5, 9, 18), + (3, 2, 4, 76), + (4, 8, 1, 33), + (5, 6, 3, 91), + (6, 1, 5, 27), + (7, 4, 8, 64), + (8, 9, 2, 55), + (9, 7, 6, 13), + (10, 10, 10, 100);""" + + // test variable agg_phase + sql "set agg_phase=1;" + qt_agg_phase_1 """explain shape plan + select max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key""" + sql "set agg_phase=2;" + qt_agg_phase_2 """explain shape plan + select max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key""" + + sql "set agg_phase=3;" + qt_agg_phase_3 """explain shape plan + select count(distinct dst_key2), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + sql "set agg_phase=4;" + qt_agg_phase_4 """explain shape plan + select count(distinct dst_key2), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + + sql "set agg_phase=1;" + qt_distinct_phase_1_shape """explain shape plan + select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + qt_distinct_phase_1 "select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key order by 1;" + + sql "set agg_phase=2;" + qt_distinct_phase_2_shape """explain shape plan + select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + qt_distinct_phase_2 "select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key order by 1;" + + sql "set agg_phase=1;" + qt_distinct_phase_1_satisfy_gby_key_shape """explain shape plan + select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by id;""" + qt_distinct_phase_1_satisfy_gby_key"""select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by id order by 1;""" + + sql "set agg_phase=2;" + qt_distinct_phase_2_satisfy_gby_key_shape """explain shape plan + select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by id;""" + qt_distinct_phase_2_satisfy_gby_key "select count(distinct dst_key1) from t_gbykey_10_dstkey_10_1000_id_2 group by id order by 1;" + + sql "set agg_phase=1;" + qt_distinct_and_other_phase_1_shape """explain shape plan + select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + qt_distinct_and_other_phase_1 "select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key order by 1,2;" + + sql "set agg_phase=2;" + qt_distinct_and_other_phase_2_shape """explain shape plan + select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + qt_distinct_and_other_phase_2 "select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key order by 1,2;" + + sql "set agg_phase=1;" + qt_distinct_and_other_phase_1_satisfy_gby_key_shape """explain shape plan + select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by id;""" + qt_distinct_and_other_phase_1_satisfy_gby_key "select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by id order by 1,2;" + + sql "set agg_phase=2;" + qt_distinct_and_other_phase_2_satisfy_gby_key_shape """explain shape plan + select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by id;""" + qt_distinct_and_other_phase_2_satisfy_gby_key "select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2 group by id order by 1,2;" + + // test without group by key + sql "set agg_phase=3;" + qt_use_multi_phase_distinct_key_satisfy """explain shape plan + select count(distinct id), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2;""" + + sql "set agg_phase=1;" + qt_distinct_key_not_satisfy_use_final_multi """explain shape plan + select count(distinct dst_key1), max(dst_key2) from t_gbykey_10_dstkey_10_1000_id_2;""" + sql "set agg_phase=0;" + + // test variable multi_distinct_strategy + sql "set multi_distinct_strategy=1;" + qt_multi_distinct_strategy_1 """explain shape plan + select count(distinct dst_key2), count(distinct dst_key1), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + sql "set multi_distinct_strategy=2;" + qt_multi_distinct_strategy_2 """explain shape plan + select count(distinct dst_key2), count(distinct dst_key1), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + sql "set multi_distinct_strategy=0;" + + sql "set agg_phase=4;" + sql "set multi_distinct_strategy=2;" + qt_agg_phase4_and_multi_distinct_strategy2 """explain shape plan + select count(distinct dst_key2), count(distinct dst_key1), max(gby_key),min(gby_key),sum(gby_key),sum0(gby_key),count(gby_key) from t_gbykey_10_dstkey_10_1000_id_2 group by gby_key;""" + sql "set agg_phase=0;" + sql "set multi_distinct_strategy=0;" + + //test exception + test { + sql "set agg_phase=5;" + exception "agg_phase should be between 0 and 4" + } + test { + sql "set multi_distinct_strategy=-1;" + exception "multi_distinct_strategy should be between 0 and 2" + } +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/distinct_split/disitinct_split.groovy b/regression-test/suites/nereids_rules_p0/distinct_split/disitinct_split.groovy index 9efe6ce487a4f8..705afa8237ffd9 100644 --- a/regression-test/suites/nereids_rules_p0/distinct_split/disitinct_split.groovy +++ b/regression-test/suites/nereids_rules_p0/distinct_split/disitinct_split.groovy @@ -18,10 +18,11 @@ suite("distinct_split") { sql "set runtime_filter_mode = OFF" sql "set disable_join_reorder=true" + sql "set global enable_auto_analyze=false;" sql "drop table if exists test_distinct_multi" sql "create table test_distinct_multi(a int, b int, c int, d varchar(10), e date) distributed by hash(a) properties('replication_num'='1');" sql "insert into test_distinct_multi values(1,2,3,'abc','2024-01-02'),(1,2,4,'abc','2024-01-03'),(2,2,4,'abcd','2024-01-02'),(1,2,3,'abcd','2024-01-04'),(1,2,4,'eee','2024-02-02'),(2,2,4,'abc','2024-01-02');" - + sql "analyze table test_distinct_multi with sync;" // first bit 0 means distinct 1 col, 1 means distinct more than 1 col; second bit 0 means without group by, 1 means with group by; // third bit 0 means there is 1 count(distinct) in projects, 1 means more than 1 count(distinct) in projects. @@ -197,20 +198,84 @@ suite("distinct_split") { qt_has_other_func "explain shape plan select count(distinct b), count(distinct a), max(b),sum(c),min(a) from test_distinct_multi" qt_2_agg """explain shape plan select max(c1), min(c2) from (select count(distinct a,b) c1, count(distinct a,c) c2 from test_distinct_multi group by c) t""" - // should not rewrite qt_multi_count_with_gby """explain shape plan select count(distinct b), count(distinct a) from test_distinct_multi group by c""" qt_multi_sum_with_gby """explain shape plan select sum(distinct b), sum(distinct a) from test_distinct_multi group by c""" qt_sum_count_with_gby """explain shape plan select sum(distinct b), count(distinct a) from test_distinct_multi group by a""" qt_has_grouping """explain shape plan select count(distinct b), count(distinct a) from test_distinct_multi group by grouping sets((a,b),(c));""" - test { - sql """select count(distinct a,b), count(distinct a) from test_distinct_multi - group by grouping sets((a,b),(c));""" - exception "The query contains multi count distinct or sum distinct, each can't have multi columns" - } //----------------test null hash join ------------------------ sql "drop table if exists test_distinct_multi_null_hash;" sql "create table test_distinct_multi_null_hash(a int, b int, c int, d varchar(10), e date) distributed by hash(a) properties('replication_num'='1');" sql "insert into test_distinct_multi_null_hash values(1,null,null,null,'2024-12-08');" qt_null_hash "SELECT a, b, count(distinct c,e), count(distinct concat(d,e))/count(distinct e) FROM test_distinct_multi_null_hash where e = '2024-12-08' GROUP BY a, b;" + + qt_same_distinct_arg "select sum(distinct b), count(distinct b) from test_distinct_multi;" + qt_same_distinct_arg_2group "select count(distinct b), sum(distinct a), count(distinct a) from test_distinct_multi;" + qt_same_distinct_arg_shape """explain shape plan select sum(distinct b), count(distinct b) from test_distinct_multi;""" + qt_same_distinct_arg_2group_shape """explain shape plan select count(distinct b), sum(distinct a), count(distinct a) from test_distinct_multi;""" + + multi_sql """drop table if exists sales_records; + CREATE TABLE sales_records ( + region VARCHAR(20), + city VARCHAR(20), + product_category VARCHAR(20), + product_id INT, + sales_amount DECIMAL(10,2), + sales_date DATE, + customer_id INT + ) DISTRIBUTED BY HASH(region) PROPERTIES('replication_num'='1'); + + -- 插入测试数据 + INSERT INTO sales_records VALUES + ('North', 'Beijing', 'Electronics', 101, 1500.00, '2024-01-15', 1001), + ('North', 'Beijing', 'Electronics', 101, 1200.00, '2024-01-16', 1002), + ('North', 'Beijing', 'Clothing', 201, 800.00, '2024-01-15', 1003), + ('North', 'Tianjin', 'Electronics', 102, 2000.00, '2024-01-17', 1004), + ('North', 'Tianjin', 'Clothing', 202, 600.00, '2024-01-18', 1005), + ('South', 'Shanghai', 'Electronics', 103, 1800.00, '2024-01-16', 1006), + ('South', 'Shanghai', 'Electronics', 103, 1600.00, '2024-01-17', 1007), + ('South', 'Shanghai', 'Clothing', 203, 900.00, '2024-01-18', 1008), + ('South', 'Guangzhou', 'Electronics', 104, 2200.00, '2024-01-19', 1009), + ('South', 'Guangzhou', 'Clothing', 204, 750.00, '2024-01-20', 1010), + ('East', 'Nanjing', 'Electronics', 105, 1300.00, '2024-01-21', 1011), + ('East', 'Nanjing', 'Clothing', 205, 850.00, '2024-01-22', 1012);""" + sql "set multi_distinct_strategy=1;" + qt_use_multi_distinct """SELECT + region, + city, + product_category, + COUNT(DISTINCT customer_id) AS unique_customers, + COUNT(DISTINCT product_id) AS unique_products, + SUM(sales_amount) AS total_sales, + AVG(sales_amount) AS avg_sale_amount, + COUNT(*) AS total_transactions + FROM sales_records + GROUP BY GROUPING SETS ( + (region, city), + (region, product_category), + (product_category), + () + ) + order by 1,2,3,4,5,6,7,8 + ;""" + + sql "set multi_distinct_strategy=2;" + qt_use_cte_split """SELECT + region, + city, + product_category, + COUNT(DISTINCT customer_id) AS unique_customers, + COUNT(DISTINCT product_id) AS unique_products, + SUM(sales_amount) AS total_sales, + AVG(sales_amount) AS avg_sale_amount, + COUNT(*) AS total_transactions + FROM sales_records + GROUP BY GROUPING SETS ( + (region, city), + (region, product_category), + (product_category), + () + ) + order by 1,2,3,4,5,6,7,8 + ;""" } diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy index bcb989d1c956cc..742d3e1026fa98 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy @@ -21,7 +21,6 @@ suite("push_down_count_through_join") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ DROP TABLE IF EXISTS count_t; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index a50dcbd83dec4e..79a9ae76e0d757 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -22,7 +22,6 @@ suite("push_down_count_through_join_one_side") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql "set topn_opt_limit_threshold=1024" sql """ DROP TABLE IF EXISTS count_t_one_side; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy index 6e4ccc5f669a37..fb2df76cf206b7 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy @@ -21,7 +21,6 @@ suite("push_down_max_through_join") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ DROP TABLE IF EXISTS max_t; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy index 076e647cbdb51c..9ee791799ae4bb 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy @@ -21,7 +21,6 @@ suite("push_down_min_through_join") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql "set disable_join_reorder=true" sql """ diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy index 844b07807f4680..e637a58219f1c5 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy @@ -21,7 +21,6 @@ suite("push_down_sum_through_join") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ DROP TABLE IF EXISTS sum_t; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy index 3ebda779cb799f..6b04b8f6ba4e8f 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy @@ -21,7 +21,6 @@ suite("push_down_sum_through_join_one_side") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set be_number_for_test=1" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ DROP TABLE IF EXISTS sum_t_one_side; diff --git a/regression-test/suites/nereids_rules_p0/merge_aggregate/merge_aggregate.groovy b/regression-test/suites/nereids_rules_p0/merge_aggregate/merge_aggregate.groovy index a824475c57094e..4ca62279ee8273 100644 --- a/regression-test/suites/nereids_rules_p0/merge_aggregate/merge_aggregate.groovy +++ b/regression-test/suites/nereids_rules_p0/merge_aggregate/merge_aggregate.groovy @@ -19,6 +19,7 @@ suite("merge_aggregate") { sql "SET enable_fallback_to_original_planner=false" sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" sql "set enable_parallel_result_sink=false;" + sql "set global enable_auto_analyze=false;" sql """ DROP TABLE IF EXISTS mal_test1 diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy index 7551a0e42100b0..462ddbeb04d9d5 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy @@ -18,6 +18,7 @@ suite("agg_optimize_when_uniform") { String db = context.config.getDbNameByFile(context.file) sql "use ${db}" + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" sql """ set enable_agg_state=true; diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy index 8149ae894d8cc0..00d253831edd3b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy @@ -20,6 +20,7 @@ suite("aggregate_with_roll_up") { sql "use ${db}" sql "set runtime_filter_mode=OFF"; sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" sql """ drop table if exists orders diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index c3da423d2deb8d..1b936f5a609a59 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -24,6 +24,7 @@ suite("aggregate_without_roll_up") { // because mv3_0 is optimized by hyper sql "SET enable_dphyp_optimizer = false;" sql "SET max_table_count_use_cascades_join_reorder = 20;" + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" sql """ drop table if exists orders diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy index 020d27d488cc0d..c5f205638e6572 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy @@ -19,6 +19,7 @@ This suite is a one dimensional test case file. */ suite("partition_mv_rewrite_dimension_1") { + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" String db = context.config.getDbNameByFile(context.file) sql "use ${db}" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy index 1828bbcc0b9c25..2e2688d813d000 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy @@ -21,6 +21,7 @@ It mainly tests the agg function, etc */ suite("partition_mv_rewrite_dimension_2_3") { String db = context.config.getDbNameByFile(context.file) + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" sql "use ${db}" sql """ diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy index 178307f15f8afb..e722b6eeb60e17 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy @@ -19,6 +19,7 @@ This suite test self connection case */ suite("partition_mv_rewrite_dimension_self_conn") { + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" String db = context.config.getDbNameByFile(context.file) sql "use ${db}" diff --git a/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy b/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy index 09bd751ecaa536..595693d5ff602a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy @@ -16,6 +16,7 @@ // under the License. suite("materialized_view_grouping_sets") { + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" String db = context.config.getDbNameByFile(context.file) sql "use ${db}" sql "set runtime_filter_mode=OFF"; diff --git a/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/dp_hyper.groovy b/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/dp_hyper.groovy index 1a9d532f296b43..b164c051cc1233 100644 --- a/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/dp_hyper.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/dp_hyper.groovy @@ -21,6 +21,7 @@ suite("rewrite_when_dp_hyper") { sql "use ${db}" sql "set runtime_filter_mode=OFF"; sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set disable_nereids_rules = 'DISTINCT_AGGREGATE_SPLIT'" sql """ drop table if exists orders_hyper diff --git a/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/strategy.groovy b/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/strategy.groovy index 24903304b6c7f2..d85d95aad1da97 100644 --- a/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/strategy.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/strategy/strategy.groovy @@ -21,6 +21,7 @@ suite("pre_write_strategy") { sql "use ${db}" sql "set runtime_filter_mode=OFF"; sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set disable_nereids_rules = 'DISTINCT_AGGREGATE_SPLIT'" sql """ drop table if exists orders diff --git a/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy b/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy index 5c1a80fd0d6a8f..0210ea566fe099 100644 --- a/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy @@ -22,7 +22,6 @@ suite("sync_async_same_name") { sql "set runtime_filter_mode=OFF"; sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO;" - sql """ drop table if exists orders """ diff --git a/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy b/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy index 6d5524bffaa915..eff733a2fec617 100644 --- a/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy @@ -26,6 +26,7 @@ // Note: To filter out tables from sql files, use the following one-liner comamnd // sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq suite("mv_tpch_test") { + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" def tables = [customer: ["c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment,temp"], lineitem: ["l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag,l_linestatus, l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,temp"], nation : ["n_nationkey, n_name, n_regionkey, n_comment, temp"], diff --git a/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy b/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy index 8f29daee25f216..acfa3187a98e68 100644 --- a/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy @@ -137,6 +137,7 @@ suite("with_select_table_auth","p0,auth") { connect(user_name, "${pwd}", context.config.jdbcUrl) { sql "use ${db}" + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" mv_rewrite_success( """ select t1.l_partkey, t1.l_suppkey, o_orderdate, diff --git a/regression-test/suites/nereids_syntax_p0/adjust_nullable.groovy b/regression-test/suites/nereids_syntax_p0/adjust_nullable.groovy index c9f99299220ae4..d0a7ee07dd3974 100644 --- a/regression-test/suites/nereids_syntax_p0/adjust_nullable.groovy +++ b/regression-test/suites/nereids_syntax_p0/adjust_nullable.groovy @@ -18,7 +18,7 @@ suite("adjust_nullable") { sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" - + sql "set runtime_filter_mode=OFF" sql """ DROP TABLE IF EXISTS agg_subquery_table """ diff --git a/regression-test/suites/nereids_syntax_p0/agg_4_phase.groovy b/regression-test/suites/nereids_syntax_p0/agg_4_phase.groovy index e2113ee75719ce..c84ed39a7cc132 100644 --- a/regression-test/suites/nereids_syntax_p0/agg_4_phase.groovy +++ b/regression-test/suites/nereids_syntax_p0/agg_4_phase.groovy @@ -28,7 +28,7 @@ suite("agg_4_phase") { ) ENGINE=OLAP DUPLICATE KEY(id) COMMENT 'OLAP' - DISTRIBUTED BY HASH(id) BUCKETS 2 + DISTRIBUTED BY HASH(age) BUCKETS 2 PROPERTIES ( "replication_allocation" = "tag.location.default: 1", "in_memory" = "false", @@ -44,7 +44,7 @@ suite("agg_4_phase") { """ def test_sql = """ select - count(distinct id) + count(distinct id), count(id) from agg_4_phase_tbl; """ explain { @@ -59,7 +59,7 @@ suite("agg_4_phase") { sql """select GROUP_CONCAT(distinct name, " ") from agg_4_phase_tbl;""" - sql """select /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,THREE_PHASE_AGGREGATE_WITH_DISTINCT,FOUR_PHASE_AGGREGATE_WITH_DISTINCT')*/ GROUP_CONCAT(distinct name, " ") from agg_4_phase_tbl group by gender;""" + sql """select GROUP_CONCAT(distinct name, " ") from agg_4_phase_tbl group by gender;""" sql "drop table if exists agg_4_phase_tbl2" @@ -67,7 +67,7 @@ suite("agg_4_phase") { sql "insert into agg_4_phase_tbl2 values(1, -10, null), (1, -10, 'a'), (2, -4, null), (2, -4, 'b'), (3, -4, 'f');\n" qt_phase4_multi_distinct """ - select /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT,THREE_PHASE_AGGREGATE_WITH_DISTINCT,THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI')*/ + select id, group_concat(cast(field1 as varchar), ','), count(distinct field1), @@ -78,21 +78,11 @@ suite("agg_4_phase") { order by id""" qt_phase4_single_distinct """ - select /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT,THREE_PHASE_AGGREGATE_WITH_DISTINCT,THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI')*/ + select id, group_concat(cast(field1 as varchar), ','), count(distinct field1) from agg_4_phase_tbl2 group by id order by id""" - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI')*/ - id,group_concat(distinct field2 order by field1) - from agg_4_phase_tbl2 - group by id - """ - exception """lowestCostPlans with physicalProperties(GATHER) doesn't exist in root group""" - } } diff --git a/regression-test/suites/nereids_syntax_p0/aggregate_strategies.groovy b/regression-test/suites/nereids_syntax_p0/aggregate_strategies.groovy index aeb39fb275af3d..9d66de08ba70fc 100644 --- a/regression-test/suites/nereids_syntax_p0/aggregate_strategies.groovy +++ b/regression-test/suites/nereids_syntax_p0/aggregate_strategies.groovy @@ -107,8 +107,6 @@ suite("aggregate_strategies") { order_qt_count_distinct_sum_distinct_same "select max(distinct id), sum(distinct id) from $tableName" - // explain plan select /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT,TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI')*/ max(distinct id), sum(distinct id) from test_bucket1_table; - order_qt_count_distinct_sum_distinct_same "select max(distinct id), sum(distinct id) from $tableName" order_qt_count_distinct_sum_distinct_difference "select count(distinct name), sum(distinct id) from $tableName" @@ -156,37 +154,10 @@ suite("aggregate_strategies") { test { sql """select - /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ - count(distinct number) - from numbers('number' = '10000')""" - result([[10000L]]) - } - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT')*/ - count(distinct number) - from numbers('number' = '10000')""" - result([[10000L]]) - } - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ - count(distinct number) - from numbers('number' = '10000')""" - result([[10000L]]) - } - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct number) from numbers('number' = '10000')""" result([[10000L]]) } qt_sql_distinct_same_col """SELECT COUNT(DISTINCT id, id) FROM test_bucket10_table GROUP BY id """ - - qt_sql_distinct_same_col2 """SELECT COUNT(DISTINCT id, id) FROM test_bucket10_table GROUP BY id """ } diff --git a/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy b/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy index 15b6600ba1aa7c..965ec743b81ba1 100644 --- a/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy +++ b/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy @@ -69,11 +69,7 @@ suite("analyze_agg") { tt2.d, tt2.c; """ - - test { - sql "select count(distinct t2.b), variance(distinct t2.c) from t2" - exception "can't support multi distinct." - } + sql "select count(distinct t2.b), variance(distinct t2.c) from t2" // should not bind g /g in group by again, otherwise will throw exception sql "select g / g as nu, sum(c) from t2 group by nu" diff --git a/regression-test/suites/nereids_syntax_p0/distribute/colocate_agg_join.groovy b/regression-test/suites/nereids_syntax_p0/distribute/colocate_agg_join.groovy index d7d88b89cd8881..2921fd193fd5a9 100644 --- a/regression-test/suites/nereids_syntax_p0/distribute/colocate_agg_join.groovy +++ b/regression-test/suites/nereids_syntax_p0/distribute/colocate_agg_join.groovy @@ -57,7 +57,6 @@ suite("colocate_agg_join") { set enable_local_shuffle=true; set runtime_filter_mode=off; - set disable_nereids_rules='ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT,TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT,TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT,FOUR_PHASE_AGGREGATE_WITH_DISTINCT,FOUR_PHASE_AGGREGATE_WITH_DISTINCT_WITH_FULL_DISTRIBUTE'; """ for (def i in 1..10) { diff --git a/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy b/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy index 42ec0df05459a8..c0e16ed5ef96cd 100644 --- a/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy +++ b/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy @@ -26,7 +26,7 @@ suite("shuffle") { """ order_qt_4_phase_agg """ - select /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT,TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI,THREE_PHASE_AGGREGATE_WITH_DISTINCT,FOUR_PHASE_AGGREGATE_WITH_DISTINCT')*/ + select id, count(distinct value) from test_shuffle group by id diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy index 49fd431d409ea2..5d8c7b7c7884e8 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy @@ -34,15 +34,16 @@ suite ("single_slot") { properties("replication_num" = "1"); """ - sql "insert into single_slot select 1,1,1,'a';" - sql "insert into single_slot select 2,2,2,'b';" - sql "insert into single_slot select 3,-3,null,'c';" + sql "insert into single_slot select 1,2,1,'a';" + sql "insert into single_slot select 1,3,2,'b';" + sql "insert into single_slot select 2,5,null,'c';" createMV("create materialized view k1ap2spa as select abs(k1)+1,sum(abs(k2+1)) from single_slot group by abs(k1)+1;") sleep(3000) - sql "insert into single_slot select -4,-4,-4,'d';" + sql "insert into single_slot select 2,-4,-4,'d';" + sql "SET experimental_enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" diff --git a/regression-test/suites/nereids_syntax_p2/aggregate_strategies.groovy b/regression-test/suites/nereids_syntax_p2/aggregate_strategies.groovy index e2ad64f688e43d..f0da2d022c7364 100644 --- a/regression-test/suites/nereids_syntax_p2/aggregate_strategies.groovy +++ b/regression-test/suites/nereids_syntax_p2/aggregate_strategies.groovy @@ -18,31 +18,6 @@ suite("aggregate_strategies") { test { sql """select - /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ - count(distinct number) - from numbers('number' = '10000000')""" - result([[10000000L]]) - } - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT')*/ - count(distinct number) - from numbers('number' = '10000000')""" - result([[10000000L]]) - } - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_WITH_DISTINCT')*/ - count(distinct number) - from numbers('number' = '10000000')""" - result([[10000000L]]) - } - - test { - sql """select - /*+SET_VAR(disable_nereids_rules='THREE_PHASE_AGGREGATE_WITH_DISTINCT')*/ count(distinct number) from numbers('number' = '10000000')""" result([[10000000L]]) diff --git a/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy b/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy index 41ec8ce26b97e7..4655ca6b06943d 100644 --- a/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy +++ b/regression-test/suites/nereids_tpch_p0/tpch/push_topn_to_agg.groovy @@ -76,7 +76,7 @@ suite("push_topn_to_agg") { sql "select count(distinct o_clerk), sum(distinct o_shippriority) from orders group by o_orderkey limit 14; " contains("VTOP-N") contains("order by: o_orderkey") - multiContains("sortByGroupKey:true", 2) + //multiContains("sortByGroupKey:true", 2) } // use group key as sort key to enable topn-push opt diff --git a/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy b/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy index 99ef4787d63d83..b8c3639fb42841 100644 --- a/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy +++ b/regression-test/suites/rollup_p0/test_create_mv_and_mtmv.groovy @@ -21,6 +21,7 @@ suite("test_create_mv_and_mtmv") { def tableName = "test_create_mv_and_mtmt_advertiser_view_record" def mvName = "test_create_mv_and_mtmt_advertiser_uv" def mtmvName = "test_create_mv_and_mtmt_advertiser_uv_mtmv" + sql "set pre_materialized_view_rewrite_strategy = TRY_IN_RBO" sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ DROP MATERIALIZED VIEW IF EXISTS ${mtmvName} """ sql """