diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SortInfo.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SortInfo.java index d1e3129244f0f8..4cb6a63165ba8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SortInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SortInfo.java @@ -342,4 +342,9 @@ public TSortInfo toThrift() { } return sortInfo; } + + @Override + public String toString() { + return orderingExprs.toString(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 35c06e74d3cc80..dd591d4a921f59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -65,7 +65,9 @@ public static ConnectContext createMTMVContext(MTMV mtmv) { ctx.getSessionVariable().allowModifyMaterializedViewData = true; // Disable add default limit rule to avoid refresh data wrong ctx.getSessionVariable().setDisableNereidsRules( - String.join(",", ImmutableSet.of(RuleType.ADD_DEFAULT_LIMIT.name()))); + String.join(",", ImmutableSet.of( + "COMPRESSED_MATERIALIZE_AGG", "COMPRESSED_MATERIALIZE_SORT", + RuleType.ADD_DEFAULT_LIMIT.name()))); Optional workloadGroup = mtmv.getWorkloadGroup(); if (workloadGroup.isPresent()) { ctx.getSessionVariable().setWorkloadGroup(workloadGroup.get()); 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 5b276258263f37..d9e8819fe94344 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 @@ -55,6 +55,7 @@ import org.apache.doris.nereids.rules.rewrite.CountDistinctRewrite; import org.apache.doris.nereids.rules.rewrite.CountLiteralRewrite; import org.apache.doris.nereids.rules.rewrite.CreatePartitionTopNFromWindow; +import org.apache.doris.nereids.rules.rewrite.DecoupleEncodeDecode; import org.apache.doris.nereids.rules.rewrite.DeferMaterializeTopNResult; import org.apache.doris.nereids.rules.rewrite.EliminateAggCaseWhen; import org.apache.doris.nereids.rules.rewrite.EliminateAggregate; @@ -115,6 +116,7 @@ import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoinOneSide; import org.apache.doris.nereids.rules.rewrite.PushDownAggWithDistinctThroughJoinOneSide; import org.apache.doris.nereids.rules.rewrite.PushDownDistinctThroughJoin; +import org.apache.doris.nereids.rules.rewrite.PushDownEncodeSlot; import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughProject; import org.apache.doris.nereids.rules.rewrite.PushDownLimit; import org.apache.doris.nereids.rules.rewrite.PushDownLimitDistinctThroughJoin; @@ -253,6 +255,13 @@ public class Rewriter extends AbstractBatchJobExecutor { new CountLiteralRewrite(), new NormalizeSort() ), + + topDown(// must behind NormalizeAggregate/NormalizeSort + new MergeProjects(), + new PushDownEncodeSlot(), + new DecoupleEncodeDecode() + ), + topic("Window analysis", topDown( new ExtractAndNormalizeWindowExpression(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java index 2cb472ce975f31..3ac5a38c914367 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java @@ -44,11 +44,6 @@ public class LogicalProperties { protected final Supplier dataTraitSupplier; private Integer hashCode = null; - public LogicalProperties(Supplier> outputSupplier, - Supplier dataTraitSupplier) { - this(outputSupplier, dataTraitSupplier, ImmutableList::of); - } - /** * constructor of LogicalProperties. * @@ -56,8 +51,7 @@ public LogicalProperties(Supplier> outputSupplier, * throw exception for which children have UnboundRelation */ public LogicalProperties(Supplier> outputSupplier, - Supplier dataTraitSupplier, - Supplier> nonUserVisibleOutputSupplier) { + Supplier dataTraitSupplier) { this.outputSupplier = Suppliers.memoize( Objects.requireNonNull(outputSupplier, "outputSupplier can not be null") ); 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 d348889818a5dd..fffa5bccbd839c 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 @@ -108,6 +108,9 @@ public enum RuleType { // rewrite rules COMPRESSED_MATERIALIZE_AGG(RuleTypeClass.REWRITE), COMPRESSED_MATERIALIZE_SORT(RuleTypeClass.REWRITE), + COMPRESSED_MATERIALIZE_REPEAT(RuleTypeClass.REWRITE), + PUSH_DOWN_ENCODE_SLOT(RuleTypeClass.REWRITE), + DECOUPLE_DECODE_ENCODE_SLOT(RuleTypeClass.REWRITE), SIMPLIFY_ENCODE_DECODE(RuleTypeClass.REWRITE), NORMALIZE_AGGREGATE(RuleTypeClass.REWRITE), NORMALIZE_SORT(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java index 814269a63984a0..aa7ea284a3b5e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeAsSmallInt; 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.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.coercion.CharacterType; @@ -72,6 +73,11 @@ public List buildRules() { logicalSort().when(a -> ConnectContext.get() != null && ConnectContext.get().getSessionVariable().enableCompressMaterialize) .then(this::compressMaterializeSort) + // ), + // RuleType.COMPRESSED_MATERIALIZE_REPEAT.build( + // logicalRepeat().when(r -> ConnectContext.get() != null + // && ConnectContext.get().getSessionVariable().enableCompressMaterialize) + // .then(this::compressMaterializeRepeat) ) ); } @@ -101,6 +107,9 @@ private LogicalSort compressMaterializeSort(LogicalSort sort) { } private Optional getEncodeExpression(Expression expression) { + if (expression.isConstant()) { + return Optional.empty(); + } DataType type = expression.getDataType(); Expression encodeExpr = null; if (type instanceof CharacterType) { @@ -169,4 +178,52 @@ private LogicalAggregate compressedMaterializeAggregate(LogicalAggregate

getEncodeGroupingSets(LogicalRepeat repeat) { + Map encode = Maps.newHashMap(); + // the first grouping set contains all group by keys + for (Expression gb : repeat.getGroupingSets().get(0)) { + Optional encodeExpr = getEncodeExpression(gb); + encodeExpr.ifPresent(expression -> encode.put(gb, expression)); + } + return encode; + } + + private LogicalRepeat compressMaterializeRepeat(LogicalRepeat repeat) { + Map encode = getEncodeGroupingSets(repeat); + if (encode.isEmpty()) { + return repeat; + } + List> newGroupingSets = Lists.newArrayList(); + for (int i = 0; i < repeat.getGroupingSets().size(); i++) { + List grouping = Lists.newArrayList(); + for (int j = 0; j < repeat.getGroupingSets().get(i).size(); j++) { + Expression groupingExpr = repeat.getGroupingSets().get(i).get(j); + grouping.add(encode.getOrDefault(groupingExpr, groupingExpr)); + } + newGroupingSets.add(grouping); + } + List newOutputs = Lists.newArrayList(); + Map decodeMap = new HashMap<>(); + for (Expression gp : encode.keySet()) { + decodeMap.put(gp, new DecodeAsVarchar(encode.get(gp))); + } + for (NamedExpression out : repeat.getOutputExpressions()) { + Expression replaced = ExpressionUtils.replace(out, decodeMap); + if (out != replaced) { + if (out instanceof SlotReference) { + newOutputs.add(new Alias(out.getExprId(), replaced, out.getName())); + } else if (out instanceof Alias) { + newOutputs.add(((Alias) out).withChildren(replaced.children())); + } else { + // should not reach here + Preconditions.checkArgument(false, "output abnormal: " + repeat); + } + } else { + newOutputs.add(out); + } + } + repeat = repeat.withGroupSetsAndOutput(newGroupingSets, newOutputs); + return repeat; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DecoupleEncodeDecode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DecoupleEncodeDecode.java new file mode 100644 index 00000000000000..10ba8805b434bf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/DecoupleEncodeDecode.java @@ -0,0 +1,73 @@ +// 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.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.scalar.DecodeAsVarchar; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeString; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * in project: + * decode_as_varchar(encode_as_xxx(v)) => v + */ +public class DecoupleEncodeDecode extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalProject().then(this::rewrite) + .toRule(RuleType.DECOUPLE_DECODE_ENCODE_SLOT); + } + + private LogicalProject rewrite(LogicalProject project) { + List newProjections = Lists.newArrayList(); + boolean hasNewProjections = false; + for (NamedExpression e : project.getProjects()) { + boolean changed = false; + if (e instanceof Alias) { + Alias alias = (Alias) e; + Expression body = alias.child(); + if (body instanceof DecodeAsVarchar && body.child(0) instanceof EncodeString) { + Expression encodeBody = body.child(0).child(0); + newProjections.add((NamedExpression) alias.withChildren(encodeBody)); + changed = true; + } else if (body instanceof EncodeString && body.child(0) instanceof DecodeAsVarchar) { + Expression decodeBody = body.child(0).child(0); + newProjections.add((NamedExpression) alias.withChildren(decodeBody)); + changed = true; + } + } + if (!changed) { + newProjections.add(e); + hasNewProjections = true; + } + } + if (hasNewProjections) { + project = project.withProjects(newProjections); + } + return project; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java index b3df9b92c56a84..8c21b12a6d8fa1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpJoinFromUnionAll.java @@ -586,10 +586,17 @@ boolean comparePlan(Plan plan1, Plan plan2) { isEqual = false; } for (int i = 0; isEqual && i < plan2.getOutput().size(); i++) { - NamedExpression expr = ((LogicalProject) plan1).getProjects().get(i); - NamedExpression replacedExpr = (NamedExpression) - expr.rewriteUp(e -> plan1ToPlan2.getOrDefault(e, e)); - if (!replacedExpr.equals(((LogicalProject) plan2).getProjects().get(i))) { + Expression expr1 = ((LogicalProject) plan1).getProjects().get(i); + Expression expr2 = ((LogicalProject) plan2).getProjects().get(i); + if (expr1 instanceof Alias) { + if (!(expr2 instanceof Alias)) { + return false; + } + expr1 = expr1.child(0); + expr2 = expr2.child(0); + } + Expression replacedExpr = expr1.rewriteUp(e -> plan1ToPlan2.getOrDefault(e, e)); + if (!replacedExpr.equals(expr2)) { isEqual = false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownEncodeSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownEncodeSlot.java new file mode 100644 index 00000000000000..01835b5cb3070b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownEncodeSlot.java @@ -0,0 +1,655 @@ +// 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.common.Pair; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DecodeAsVarchar; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeString; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalLeaf; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.types.coercion.CharacterType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +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; + +/** + * push down encode_as_int(slot) down + * example: + * group by x + * -->project(encode_as_int(A) as x) + * -->Any(A) + * -->project(A) + * --> scan + * => + * group by x + * -->project(x) + * -->Any(x) + * --> project(encode_as_int(A) as x) + * -->scan + * Note: + * do not push down encode if encode.child() is not slot, + * example + * group by encode_as_int(A + B) + * --> any(A, B) + */ +public class PushDownEncodeSlot extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalProject() + .whenNot(project -> project.child() instanceof LogicalRepeat) + .whenNot(project -> (project.child() instanceof LogicalLeaf)) + .then(this::pushDownEncodeSlot) + .toRule(RuleType.PUSH_DOWN_ENCODE_SLOT); + } + + private List collectEncodeAliases(LogicalProject project) { + List encodeAliases = new ArrayList<>(); + Set windowSlots = new HashSet<>(); + for (NamedExpression e : project.getProjects()) { + if (e instanceof Alias && e.child(0) instanceof WindowExpression) { + windowSlots.addAll(e.getInputSlots()); + } + } + for (NamedExpression e : project.getProjects()) { + if (e instanceof Alias && e.child(0) instanceof EncodeString + && e.child(0).child(0) instanceof SlotReference + && !windowSlots.contains(e.child(0).child(0))) { + encodeAliases.add((Alias) e); + } + } + return encodeAliases; + } + + /** + * case 1 + * project(encode(A) as B) + * --> any(A) + * => + * project(B) + * -->any(A): push "encode(A) as B" + * + * case 2 + * project(A, encode(A) as B) + * -->any(A) + * => + * project(decode(B) as A, B) + * -->any(A): push "encode(A) as B" + * + * case 3 + * project(A as C, encode(A) as B) + * -->any(A) + * => + * project(decode(B) as C, B) + * -->any(A): push "encode(A) as B" + */ + private LogicalProject rewriteRootProject(LogicalProject project, + List pushedEncodeAlias) { + if (pushedEncodeAlias.isEmpty()) { + return project; + } + Map encodeBodyToEncodeAlias = new HashMap<>(); + for (Alias alias : pushedEncodeAlias) { + Expression encodeBody = alias.child().child(0); + encodeBodyToEncodeAlias.put(encodeBody, alias); + } + List projections = Lists.newArrayListWithCapacity(project.getProjects().size()); + for (NamedExpression e : project.getProjects()) { + if (pushedEncodeAlias.contains(e)) { + // case 1 + projections.add(e.toSlot()); + } else if (encodeBodyToEncodeAlias.containsKey(e)) { + // case 2 + ExprId id = e.getExprId(); + DecodeAsVarchar decode = new DecodeAsVarchar(encodeBodyToEncodeAlias.get(e).toSlot()); + Alias alias = new Alias(id, decode, decode.toSql()); + projections.add(alias); + } else if (e instanceof Alias && encodeBodyToEncodeAlias.containsKey(e.child(0))) { + // case 3 + Alias alias = (Alias) e; + DecodeAsVarchar decode = new DecodeAsVarchar(encodeBodyToEncodeAlias.get(e.child(0)).toSlot()); + Alias newAlias = (Alias) alias.withChildren(decode); + projections.add(newAlias); + } else { + projections.add(e); + } + } + return project.withProjects(projections); + + } + + private LogicalProject pushDownEncodeSlot(LogicalProject project) { + List encodeAliases = collectEncodeAliases(project); + if (encodeAliases.isEmpty()) { + return project; + } + + PushDownContext ctx = new PushDownContext(project, encodeAliases); + ctx.prepare(); + if (ctx.notPushed.size() == encodeAliases.size()) { + return project; + } + Plan child = project.child(); + PushDownContext childContext = new PushDownContext(child, ctx.toBePushedToChild.get(child)); + Plan newChild = child.accept(EncodeSlotPushDownVisitor.INSTANCE, childContext); + List pushed = ctx.toBePused; + if (child != newChild) { + if (newChild instanceof LogicalProject) { + pushed.removeAll(childContext.notPushed); + newChild = ((LogicalProject) newChild).child(); + } + project = (LogicalProject) project.withChildren(newChild); + project = rewriteRootProject(project, pushed); + } + return project; + } + + /** + * push down encode slot context + */ + public static class PushDownContext { + public Plan plan; + + public List encodeAliases; + // encode_as_int(slot1) as slot2 + // replaceMap: + // slot1 -> slot2 + Map replaceMap = new HashMap<>(); + // child plan -> aliases in encodeAliases which can be pushed down to child plan + Map> toBePushedToChild = new HashMap<>(); + List toBePused = new ArrayList<>(); + // the aliases that cannot be pushed down to any child plan + // for example: + // encode(A+B) as x, where plan is a join, and A, B comes from join's left and right child respectively + List notPushed = new ArrayList<>(); + + public PushDownContext(Plan plan, List encodeAliases) { + this.plan = plan; + this.encodeAliases = encodeAliases; + } + + // init replaceMap/toBePushed/notPushed + private void prepare() { + List> childrenPassThroughSlots = + plan.children().stream().map(n -> getPassThroughSlots(n)).collect(Collectors.toList()); + for (int i = 0; i < plan.children().size(); i++) { + Plan child = plan.children().get(i); + if (child instanceof LogicalJoin) { + LogicalJoin join = (LogicalJoin) child; + BiMap compareSlots = EncodeSlotPushDownVisitor + .getEncodeCandidateSlotsAndShouldNotPushSlotsFromJoinCondition(join).first; + childrenPassThroughSlots.get(i).addAll(compareSlots.keySet()); + } + } + for (Alias alias : encodeAliases) { + EncodeString encode = (EncodeString) alias.child(); + Expression strExpr = encode.child(); + boolean pushed = false; + Preconditions.checkArgument(strExpr instanceof SlotReference, + "expect encode_as_xxx(slot), but " + alias); + + for (int i = 0; i < childrenPassThroughSlots.size(); i++) { + if (childrenPassThroughSlots.get(i).contains(strExpr)) { + toBePushedToChild.putIfAbsent(plan.child(i), new ArrayList<>()); + toBePushedToChild.get(plan.child(i)).add(alias); + toBePused.add(alias); + replaceMap.put(alias.child().child(0), (SlotReference) alias.toSlot()); + pushed = true; + break; + } + } + if (!pushed) { + notPushed.add(alias); + } + } + } + + /** + * expandEncodeAliasForJoin + */ + public void expandEncodeAliasForJoin(BiMap equalSlots) { + List expanded = new ArrayList<>(); + for (Alias alias : encodeAliases) { + if (alias.child().child(0) instanceof SlotReference) { + SlotReference slot = (SlotReference) alias.child().child(0); + SlotReference otherHand = equalSlots.get(slot); + if (otherHand != null) { + EncodeString encodeOtherHand = (EncodeString) alias.child().withChildren(otherHand); + Alias encodeOtherHandAlias = new Alias(encodeOtherHand, encodeOtherHand.toSql()); + if (!encodeAliases.contains(encodeOtherHandAlias)) { + expanded.add(encodeOtherHandAlias); + } + } + } + } + encodeAliases.addAll(expanded); + } + + // the child of alias is a slot reference. for example: slotA as B + // + private boolean isSlotAlias(Expression expr) { + return expr instanceof Alias && expr.child(0) instanceof SlotReference; + } + + private Set getPassThroughSlots(Plan plan) { + Set outputSlots = Sets.newHashSet(plan.getOutputSet()); + Set keySlots = Sets.newHashSet(); + for (Expression e : plan.getExpressions()) { + if (!(e instanceof SlotReference) && !isSlotAlias(e)) { + keySlots.addAll(e.getInputSlots()); + } + } + outputSlots.removeAll(keySlots); + return outputSlots; + } + } + + /** + * push down encode slot + */ + public static class EncodeSlotPushDownVisitor extends PlanVisitor { + public static EncodeSlotPushDownVisitor INSTANCE = new EncodeSlotPushDownVisitor(); + + /** + * visitChildren + */ + public Plan visitChildren(Plan plan, PushDownContext ctx) { + ImmutableList.Builder newChildren = ImmutableList.builderWithExpectedSize(plan.arity()); + boolean hasNewChildren = false; + for (Plan child : plan.children()) { + Plan newChild; + if (ctx.toBePushedToChild.containsKey(child)) { + newChild = child.accept(this, new PushDownContext(child, ctx.toBePushedToChild.get(child))); + if (!hasNewChildren && newChild != child) { + hasNewChildren = true; + } + } else { + newChild = child; + } + newChildren.add(newChild); + } + if (hasNewChildren) { + plan = plan.withChildren(newChildren.build()); + } + return plan; + } + + private Plan projectNotPushedAlias(Plan plan, List notPushedAlias) { + if (!notPushedAlias.isEmpty()) { + // project encode expressions if they are not pushed down + // project(encode) + // +--> plan + List projections = + notPushedAlias.stream().map(e -> (NamedExpression) e).collect(Collectors.toList()); + projections.addAll(plan.getOutput()); + plan = new LogicalProject<>(projections, plan); + } + return plan; + } + + @Override + public Plan visit(Plan plan, PushDownContext ctx) { + ctx.prepare(); + plan = visitChildren(plan, ctx); + plan = projectNotPushedAlias(plan, ctx.notPushed); + return plan; + } + + @Override + public Plan visitLogicalRepeat(LogicalRepeat repeat, PushDownContext ctx) { + Plan plan = projectNotPushedAlias(repeat, ctx.encodeAliases); + return plan; + } + + private Optional findEncodeAliasByEncodeSlot(SlotReference slot, List aliases) { + for (Alias alias : aliases) { + if (alias.child().child(0).equals(slot)) { + return Optional.of(alias); + } + } + return Optional.empty(); + } + + @Override + public LogicalProject visitLogicalProject( + LogicalProject project, PushDownContext ctx) { + /* + * case 1 + * push down "encode(v1) as v2 + * project(v1, ...) + * +--->any(v1) + * => + * project(v2, ...) + * +--->any(v1) + * and push down "encode(v1) as v2" to any(v1) + * + * case 2 + * push down "encode(v1) as v2 + * project(k as v1, ...) + * +--->any(k) + * => + * project(v2, ...) + * +--->any(k) + * and push down "encode(k) as v2" to any(v1) + * + * case 3 + * push down "encode(v44) as v307" + * project(decode(v305) as v44) + * +-->agg(v305, groupBy[v305]) + * +--->project(encode(v44) as v305) + * => + * project(v305 as v307) + * +-->agg + * + * case 4 + * push down "encode(v1) as v2 + * project(a + b as v1, ...) + * +--->any(a, b) + * => + * project(encode(a+b) as v2, ...) + * +-->any(a, b) + + * + */ + List projections = Lists.newArrayListWithCapacity(project.getProjects().size()); + List toBePushed = Lists.newArrayList(); + List notPushed = Lists.newArrayList(ctx.encodeAliases); + + for (NamedExpression e : project.getProjects()) { + boolean changed = false; + + if (e instanceof SlotReference) { + Optional encodeAliasOpt = findEncodeAliasByEncodeSlot((SlotReference) e, ctx.encodeAliases); + if (encodeAliasOpt.isPresent()) { + // case 1 + projections.add(encodeAliasOpt.get().toSlot()); + toBePushed.add(encodeAliasOpt.get()); + notPushed.remove(encodeAliasOpt.get()); + changed = true; + } + } else { + // e is Alias + Expression aliasExpr = e.child(0); + if (aliasExpr instanceof SlotReference) { + //case 2 + Optional encodeAliasOpt = findEncodeAliasByEncodeSlot((SlotReference) e.toSlot(), + ctx.encodeAliases); + if (encodeAliasOpt.isPresent()) { + projections.add(encodeAliasOpt.get().toSlot()); + Alias encodeAlias = encodeAliasOpt.get(); + EncodeString encode = (EncodeString) encodeAlias.child(); + SlotReference baseSlot = (SlotReference) aliasExpr; + Alias encodeAliasForChild = (Alias) encodeAlias.withChildren(encode.withChildren(baseSlot)); + toBePushed.add(encodeAliasForChild); + notPushed.remove(encodeAlias); + changed = true; + } + } else { + Optional encodeAliasOpt = findEncodeAliasByEncodeSlot((SlotReference) e.toSlot(), + ctx.encodeAliases); + if (encodeAliasOpt.isPresent()) { + Alias encodeAlias = encodeAliasOpt.get(); + if (aliasExpr instanceof DecodeAsVarchar) { + // case 3 + // push down "encode(v44) as v307" + // project(decode(v305) as v44) + // +-->agg(v305, groupBy[v305]) + // +--->project(encode(v44) as v305) + Expression decodeBody = aliasExpr.child(0); + Alias aliasForProject = (Alias) encodeAlias.withChildren(decodeBody); + projections.add(aliasForProject); + notPushed.remove(encodeAlias); + changed = true; + } else { + // case 4 + EncodeString encode = (EncodeString) encodeAlias.child(); + Alias encodeAliasForProject = (Alias) encodeAlias + .withChildren(encode.withChildren(aliasExpr)); + projections.add(encodeAliasForProject); + notPushed.remove(encodeAlias); + changed = true; + } + } + } + } + if (!changed) { + projections.add(e); + } + } + projections.addAll(notPushed); + + project = project.withProjects(projections); + if (!toBePushed.isEmpty()) { + PushDownContext childContext = new PushDownContext(project.child(), toBePushed); + Plan newChild = project.child().accept(this, childContext); + if (project.child() != newChild) { + project = (LogicalProject) project.withChildren(newChild); + } + } + return project; + } + + private static boolean canBothSidesEncode(ComparisonPredicate compare) { + return compare.left().getDataType() instanceof CharacterType + && ((CharacterType) compare.left().getDataType()).getLen() < 15 + && ((CharacterType) compare.right().getDataType()).getLen() < 15 + && compare.left() instanceof SlotReference && compare.right() instanceof SlotReference; + } + + /** + * getEncodeCandidateSlotsFromJoinCondition + * + */ + public static Pair, Set> + getEncodeCandidateSlotsAndShouldNotPushSlotsFromJoinCondition(LogicalJoin join) { + // T1 join T2 on v1=v2 => v1/v2 can be encoded + // T1 join T2 on v1=v2 and fun(v1) => v1/v2 can not be encoded + BiMap compareSlots = HashBiMap.create(); + List conditions = new ArrayList<>(); + conditions.addAll(join.getHashJoinConjuncts()); + conditions.addAll(join.getOtherJoinConjuncts()); + Set shouldNotPushSlots = Sets.newHashSet(); + for (Expression e : conditions) { + boolean canPush = false; + if (e instanceof ComparisonPredicate) { + ComparisonPredicate compare = (ComparisonPredicate) e; + if (canBothSidesEncode(compare)) { + compareSlots.put((SlotReference) compare.left(), (SlotReference) compare.right()); + canPush = true; + } + } + if (!canPush) { + shouldNotPushSlots.addAll(e.getInputSlots()); + } + } + for (Slot notPushSlot : shouldNotPushSlots) { + if (compareSlots.isEmpty()) { + break; + } + compareSlots.remove((SlotReference) notPushSlot); + compareSlots.inverse().remove((SlotReference) notPushSlot); + } + return Pair.of(compareSlots, shouldNotPushSlots); + } + + @Override + public Plan visitLogicalJoin(LogicalJoin join, PushDownContext ctx) { + List pushLeft = new ArrayList<>(); + List pushRight = new ArrayList<>(); + Pair, Set> pair = + getEncodeCandidateSlotsAndShouldNotPushSlotsFromJoinCondition(join); + BiMap encodeCandidateSlots = pair.first; + Set shouldNotPushSlots = pair.second; + Set leftOutputSlots = join.left().getOutputSet(); + Map replaceMap = new HashMap<>(); + List notPushed = new ArrayList<>(); + for (Alias encodeAlias : ctx.encodeAliases) { + SlotReference encodeSlot = (SlotReference) encodeAlias.child().child(0); + if (encodeCandidateSlots.containsKey(encodeSlot)) { + SlotReference otherHand = encodeCandidateSlots.get(encodeSlot); + Alias otherHandAlias = new Alias(encodeAlias.child().withChildren(otherHand)); + if (leftOutputSlots.contains(encodeSlot)) { + pushLeft.add(encodeAlias); + pushRight.add(otherHandAlias); + } else { + pushRight.add(encodeAlias); + pushLeft.add(otherHandAlias); + } + replaceMap.put(encodeSlot, (SlotReference) encodeAlias.toSlot()); + replaceMap.put(otherHand, (SlotReference) otherHandAlias.toSlot()); + } else if (!shouldNotPushSlots.contains(encodeSlot)) { + if (leftOutputSlots.contains(encodeSlot)) { + pushLeft.add(encodeAlias); + } else { + pushRight.add(encodeAlias); + } + } else { + notPushed.add(encodeAlias); + } + replaceMap.put(encodeSlot, (SlotReference) encodeAlias.toSlot()); + } + List newConjuncts = Lists.newArrayListWithCapacity(join.getOtherJoinConjuncts().size()); + boolean changed = false; + for (Expression e : join.getOtherJoinConjuncts()) { + if (e instanceof ComparisonPredicate) { + ComparisonPredicate compare = (ComparisonPredicate) e; + if (canBothSidesEncode(compare)) { + SlotReference newLeft = replaceMap.get(compare.left()); + SlotReference newRight = replaceMap.get(compare.right()); + if (newLeft != null && newRight != null) { + compare = (ComparisonPredicate) compare.withChildren(newLeft, newRight); + changed = true; + } + Preconditions.checkArgument((newLeft == null) == (newRight == null), + "PushDownEncodeSlot replaceMap is not valid, " + compare); + } + newConjuncts.add(compare); + } else { + newConjuncts.add(e); + } + } + if (changed) { + join = join.withJoinConjuncts(join.getHashJoinConjuncts(), newConjuncts, join.getJoinReorderContext()); + } + Plan newLeft; + if (pushLeft.isEmpty()) { + newLeft = join.left(); + } else { + newLeft = join.left().accept(this, new PushDownContext(join.left(), pushLeft)); + } + Plan newRight; + if (pushRight.isEmpty()) { + newRight = join.right(); + } else { + newRight = join.right().accept(this, new PushDownContext(join.right(), pushRight)); + } + Plan result = join.withChildren(newLeft, newRight); + if (!notPushed.isEmpty()) { + List projections = new ArrayList<>(); + projections.addAll(notPushed); + projections.addAll(join.getOutput()); + result = new LogicalProject(projections, join); + } + return result; + } + + @Override + public Plan visitLogicalSetOperation(LogicalSetOperation op, PushDownContext ctx) { + // push down "encode(v) as x" through + // union(output[v], regular([v1],[v2])) + // -->child1(v1) + // -->child2(v2) + // rewrite union to: union(output[x], regular([x1], [x2])) + // and then push "encode(v1) as x1" to child(v1) + // push "encode(v2) as x2" to child(v2) + + List newOutput = Lists.newArrayListWithCapacity(op.getOutput().size()); + List> newRegularOutputs = Lists.newArrayListWithCapacity(op.getOutput().size()); + for (int cid = 0; cid < op.children().size(); cid++) { + newRegularOutputs.add(Lists.newArrayList(op.getRegularChildOutput(cid))); + } + + for (int oid = 0; oid < op.getOutput().size(); oid++) { + NamedExpression e = op.getOutput().get(oid); + boolean changed = false; + for (Alias alias : ctx.encodeAliases) { + if (alias.child().child(0).equals(e)) { + newOutput.add(alias.toSlot()); + changed = true; + EncodeString encode = (EncodeString) alias.child(); + ctx.toBePused.add(alias); + for (int cid = 0; cid < op.children().size(); cid++) { + Plan child = op.child(cid); + ctx.toBePushedToChild.putIfAbsent(child, new ArrayList<>()); + Alias aliasForChild = new Alias( + encode.withChildren(op.getRegularChildrenOutputs().get(cid).get(oid))); + ctx.toBePushedToChild.get(child).add(aliasForChild); + newRegularOutputs.get(cid).set(oid, (SlotReference) aliasForChild.toSlot()); + } + break; + } + } + if (!changed) { + newOutput.add(e); + } + } + op = op.withNewOutputs(newOutput); + + //rewrite children + List newChildren = Lists.newArrayListWithCapacity(op.children().size()); + for (Plan child : op.children()) { + if (!ctx.toBePushedToChild.get(child).isEmpty()) { + PushDownContext childCtx = new PushDownContext(child, ctx.toBePushedToChild.get(child)); + Plan newChild = child.accept(this, childCtx); + newChildren.add(newChild); + } else { + newChildren.add(child); + } + } + op = op.withChildrenAndTheirOutputs(newChildren, newRegularOutputs); + return op; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java index 38b687ba8387a0..3963c1b651ba28 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java @@ -23,7 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.DecodeAsVarchar; -import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeStrToInteger; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeString; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; @@ -149,7 +149,7 @@ private static Set eliminateDecodeAndEncode(Set expressi } private static Expression eliminateDecodeAndEncode(Expression expression) { - if (expression instanceof DecodeAsVarchar && expression.child(0) instanceof EncodeStrToInteger) { + if (expression instanceof DecodeAsVarchar && expression.child(0) instanceof EncodeString) { return expression.child(0).child(0); } boolean hasNewChild = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyEncodeDecode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyEncodeDecode.java index b6513fc7580707..fd15fae0d07a53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyEncodeDecode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyEncodeDecode.java @@ -23,7 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.functions.scalar.DecodeAsVarchar; -import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeStrToInteger; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeString; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -48,7 +48,7 @@ public List buildRules() { boolean changed = false; for (NamedExpression namedExpression : project.getProjects()) { if (namedExpression instanceof Alias - && namedExpression.child(0) instanceof EncodeStrToInteger + && namedExpression.child(0) instanceof EncodeString && namedExpression.child(0).child(0) instanceof DecodeAsVarchar) { Alias alias = (Alias) namedExpression; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java index 5bea07fff00326..d3d40c7178c623 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java @@ -199,7 +199,7 @@ public String toSql() { @Override public String toString() { StringBuilder sb = new StringBuilder(); - sb.append(function).append(" WindowSpec("); + sb.append("WindowExpression(").append(function).append(" spec("); if (!partitionKeys.isEmpty()) { sb.append("PARTITION BY ").append(partitionKeys.stream() .map(Expression::toString) @@ -211,7 +211,7 @@ public String toString() { .collect(Collectors.joining(", ", "", " "))); } windowFrame.ifPresent(wf -> sb.append(wf.toSql())); - return sb.toString().trim() + ")"; + return sb.toString().trim() + "))"; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java index 7d798ecf3e8cab..d3f267efa36d33 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java @@ -33,8 +33,8 @@ /** * ScalarFunction 'EncodeAsBigInt'. */ -public class EncodeAsBigInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { +public class EncodeAsBigInt extends EncodeString + implements ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java index 5c6382d6ea144d..661f05fe3be82d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java @@ -33,8 +33,8 @@ /** * ScalarFunction 'EncodeAsInt'. */ -public class EncodeAsInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { +public class EncodeAsInt extends EncodeString + implements ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(IntegerType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java index bb30a9a8e8aef5..ee8d723d2b5143 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java @@ -33,8 +33,8 @@ /** * ScalarFunction 'EncodeAsLargeInt'. */ -public class EncodeAsLargeInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { +public class EncodeAsLargeInt extends EncodeString + implements ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(LargeIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java index 355a740197c33e..d0e6a1fa23b121 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java @@ -33,8 +33,8 @@ /** * ScalarFunction 'CompressAsSmallInt'. */ -public class EncodeAsSmallInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { +public class EncodeAsSmallInt extends EncodeString + implements ExplicitlyCastableSignature, PropagateNullable { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(SmallIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeStrToInteger.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeString.java similarity index 71% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeStrToInteger.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeString.java index 87a9c43687d6a3..778d76c3462508 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeStrToInteger.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeString.java @@ -17,8 +17,17 @@ package org.apache.doris.nereids.trees.expressions.functions.scalar; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; + /** * Encode_as_XXXInt */ -public interface EncodeStrToInteger { +public abstract class EncodeString extends ScalarFunction implements UnaryExpression { + /** + * constructor with 1 argument. + */ + public EncodeString(String name, Expression arg0) { + super(name, arg0); + } } 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 11baf2137ab5d8..f243937e33d084 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 @@ -207,7 +207,8 @@ public String toString() { "outputExpr", outputExpressions, "partitionExpr", partitionExpressions, "topnFilter", topnPushInfo != null, - "topnPushDown", getMutableState(MutableState.KEY_PUSH_TOPN_TO_AGG).isPresent() + "topnPushInfo", getMutableState(MutableState.KEY_PUSH_TOPN_TO_AGG).isPresent() + ? (TopnPushInfo) getMutableState(MutableState.KEY_PUSH_TOPN_TO_AGG).get() : "false" ); } @@ -329,6 +330,15 @@ public TopnPushInfo(List orderkeys, long limit) { this.orderkeys = ImmutableList.copyOf(orderkeys); this.limit = limit; } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder("["); + builder.append("orderkeys=").append(orderkeys); + builder.append(", limit=").append(limit); + builder.append("]"); + return builder.toString(); + } } public TopnPushInfo getTopnPushInfo() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 25637d1b816656..813a3edb7d2c31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -420,7 +420,7 @@ public static Optional extractSlotOrCastOnSlot(Expression expr) { /** * Generate replaceMap Slot -> Expression from NamedExpression[Expression as name] */ - public static Map generateReplaceMap(List namedExpressions) { + public static Map generateReplaceMap(List namedExpressions) { Map replaceMap = Maps.newLinkedHashMapWithExpectedSize(namedExpressions.size()); for (NamedExpression namedExpression : namedExpressions) { if (namedExpression instanceof Alias) { @@ -484,7 +484,7 @@ public static Set replace(Set exprs, /** * Replace expression node in the expression tree by `replaceMap` in top-down manner. */ - public static List replaceNamedExpressions(List namedExpressions, + public static List replaceNamedExpressions(List namedExpressions, Map replaceMap) { Builder replaceExprs = ImmutableList.builderWithExpectedSize(namedExpressions.size()); for (NamedExpression namedExpression : namedExpressions) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/PlanUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/PlanUtils.java index 0076c232340d29..28f7cda427ed7f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/PlanUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/PlanUtils.java @@ -121,8 +121,8 @@ public static List adjustNullableForRepeat( /** * merge childProjects with parentProjects */ - public static List mergeProjections(List childProjects, - List parentProjects) { + public static List mergeProjections(List childProjects, + List parentProjects) { Map replaceMap = ExpressionUtils.generateReplaceMap(childProjects); return ExpressionUtils.replaceNamedExpressions(parentProjects, replaceMap); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java index 446f49c3782902..049b18ff431d4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/AggregationNode.java @@ -412,9 +412,9 @@ public String getNodeExplainString(String detailPrefix, TExplainLevel detailLeve if (!conjuncts.isEmpty()) { output.append(detailPrefix).append("having: ").append(getExplainString(conjuncts)).append("\n"); } - output.append(detailPrefix).append("sortByGroupKey:").append(sortByGroupKey != null).append("\n"); output.append(detailPrefix).append(String.format( "cardinality=%,d", cardinality)).append("\n"); + output.append(detailPrefix).append("sortInfo:").append(sortByGroupKey).append("\n"); return output.toString(); } 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 126ed1135e9dc0..d576508dab98cd 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 @@ -2206,13 +2206,13 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { public boolean enableSortSpill = false; @VariableMgr.VarAttr( - name = "ENABLE_COMPRESS_MATERIALIZE", + name = "enable_compress_materialize", description = {"控制是否启用compress materialize。", "enable compress-materialize. "}, needForward = true, fuzzy = false, varType = VariableAnnotation.EXPERIMENTAL ) - public boolean enableCompressMaterialize = false; + public boolean enableCompressMaterialize = true; @VariableMgr.VarAttr( name = ENABLE_AGG_SPILL, diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java index a91c0dd47126fc..6462b1da5714a4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java @@ -101,6 +101,7 @@ protected void runBeforeAll() throws Exception { SSBUtils.createTables(this); createView("CREATE VIEW V1 AS SELECT * FROM part"); createView("CREATE VIEW V2 AS SELECT * FROM part"); + connectContext.getSessionVariable().enableCompressMaterialize = false; } @Override diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index 45e1190412d0a4..95e6cf5fe790a5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -249,6 +249,7 @@ protected void runBeforeAll() throws Exception { + ");\n"); // Should not make scan to empty relation when the table used by materialized view has no data connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION,ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"); + connectContext.getSessionVariable().enableCompressMaterialize = false; } // Test when join both side are all partition table and partition column name is same diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out index f12195a12a34f7..5decaba01a8518 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -15,7 +15,8 @@ PhysicalResultSink ------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out index 0d4af14e34d2cc..eb614998e10a2c 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) Hint log: Used: diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out index 46aab2d9f4aa45..ed578931b3952f 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] Hint log: Used: leading(catalog_sales date_dim { customer customer_address } ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out index f2827768aa02af..060dd42f4a2f56 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out index e06cd4e9b57753..a7fd2d0f92a3a8 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out @@ -1,44 +1,46 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------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 broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) -------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] Hint log: Used: leading(catalog_sales cd1 { cd2 { customer customer_address } } date_dim item ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out index df767a4b96e95a..65e5b4c4d94039 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------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 -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((store.s_state = 'TN')) +--------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((store.s_state = 'TN')) -----------------------------------PhysicalOlapScan[store] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] Hint log: Used: leading(store_sales customer_demographics store date_dim item ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out index f1da603468bba0..8bfe881068a78e 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out @@ -18,26 +18,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'AR')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'AR')) +----------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) Hint log: Used: leading(web_returns date_dim customer_address ) leading(ctr1 { customer customer_address } ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out index c5dc60c2da2bce..3dff7083483717 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] 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 -------------PhysicalProject ---------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) -----------------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] ---------------------------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_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------PhysicalProject -------------------------------------filter((store.s_county = 'Williamson County')) ---------------------------------------PhysicalOlapScan[store] -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -------------------------------PhysicalOlapScan[household_demographics] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] 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 +--------------PhysicalProject +----------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +------------------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] +----------------------------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_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------------------PhysicalProject +--------------------------------------filter((store.s_county = 'Williamson County')) +----------------------------------------PhysicalOlapScan[store] +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +--------------------------------PhysicalOlapScan[household_demographics] Hint log: Used: leading(store_sales store date_dim household_demographics ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out index b8219c68f1fe8b..61186cb491907e 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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_item_sk->[cr_item_sk];RF3 cs_order_number->[cr_order_number] -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------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=((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 RF4 -------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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_item_sk->[cr_item_sk];RF3 cs_order_number->[cr_order_number] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 +------------------------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=((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 RF4 +--------------------------------PhysicalProject +----------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] Hint log: Used: leading(catalog_returns { catalog_sales item date_dim } warehouse ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out index dcc2e202e23752..99296a8b2b52ab 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out index 8b807baf5409e4..2042698b5baf51 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out index 9575ee04f0fe62..b676bbcaffdd49 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out @@ -1,31 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] Hint log: Used: leading(store_sales { store_returns d2 } broadcast d1 store ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out index 43c8732d7f8553..fc50c412111f88 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out index 924459179feee9..5114322d79c144 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[ws_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF6 w_warehouse_sk->[cs_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out index 31101f12eab21a..bc678a459c46a6 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out index f01a29de46298f..e681101eafdb7d 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out @@ -13,34 +13,41 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] Hint log: Used: leading(store_sales d1 store ) leading(store_sales date_dim store ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out index 473b9fded85715..00a991c21a3757 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out @@ -1,40 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject ------------------------------filter(cs_bill_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out index fcbe4a8ad57c34..142ba55e2f5798 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out @@ -18,26 +18,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ctr_customer_sk->[c_customer_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'TN')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ctr_customer_sk->[c_customer_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'TN')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) Hint log: Used: leading(catalog_returns date_dim customer_address ) leading(customer customer_address ctr1 ) diff --git a/regression-test/data/nereids_hint_tpch_p0/shape/q12.out b/regression-test/data/nereids_hint_tpch_p0/shape/q12.out index a8710941069079..c04d5e0d01e0dc 100644 --- a/regression-test/data/nereids_hint_tpch_p0/shape/q12.out +++ b/regression-test/data/nereids_hint_tpch_p0/shape/q12.out @@ -1,19 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] Hint log: Used: leading(orders lineitem ) diff --git a/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out b/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out index eee04795628144..8922b2155aec2b 100644 --- a/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out +++ b/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out @@ -53,3 +53,21 @@ a 1 中 8 bb 3 +-- !explain_sort_agg -- +cost = 19.563333333333333 +PhysicalResultSink[294] ( outputExprs=[v1#1] ) ++--PhysicalProject[289]@5 ( stats=1, projects=[v1#1] ) + +--PhysicalQuickSort[284]@4 ( stats=1, orderKeys=[encode_as_bigint(v1)#4 asc null first], phase=MERGE_SORT ) + +--PhysicalDistribute[279]@7 ( stats=1, distributionSpec=DistributionSpecGather ) + +--PhysicalQuickSort[274]@7 ( stats=1, orderKeys=[encode_as_bigint(v1)#4 asc null first], phase=LOCAL_SORT ) + +--PhysicalProject[269]@3 ( stats=1, projects=[decode_as_varchar(encode_as_bigint(v1)#3) AS `v1`#1, encode_as_bigint(decode_as_varchar(encode_as_bigint(v1)#3)) AS `encode_as_bigint(decode_as_varchar(encode_as_bigint(v1)))`#4], multi_proj=l0([encode_as_bigint(v1)#3, decode_as_varchar(encode_as_bigint(v1)#3) AS `decode_as_varchar(encode_as_bigint(v1))`#5])l1([decode_as_varchar(encode_as_bigint(v1))#5 AS `v1`#1, encode_as_bigint(decode_as_varchar(encode_as_bigint(v1))#5) AS `encode_as_bigint(decode_as_varchar(encode_as_bigint(v1)))`#4]) ) + +--PhysicalHashAggregate[264]@2 ( stats=1, aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[encode_as_bigint(v1)#3], outputExpr=[encode_as_bigint(v1)#3], partitionExpr=Optional[[encode_as_bigint(v1)#3]], topnFilter=false, topnPushDown=false ) + +--PhysicalDistribute[259]@8 ( stats=1, distributionSpec=DistributionSpecHash ( orderedShuffledColumns=[3], shuffleType=EXECUTION_BUCKETED, tableId=-1, selectedIndexId=-1, partitionIds=[], equivalenceExprIds=[[3]], exprIdToEquivalenceSet={3=0} ) ) + +--PhysicalHashAggregate[254]@8 ( stats=1, aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=true, groupByExpr=[encode_as_bigint(v1)#3], outputExpr=[encode_as_bigint(v1)#3], partitionExpr=Optional[[encode_as_bigint(v1)#3]], topnFilter=false, topnPushDown=false ) + +--PhysicalProject[249]@1 ( stats=1, projects=[encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#3] ) + +--PhysicalOlapScan[t1]@0 ( stats=1 ) + +-- !exec_sort_agg -- +a +b + diff --git a/regression-test/data/nereids_p0/compress_materialize/pushdown_encode.out b/regression-test/data/nereids_p0/compress_materialize/pushdown_encode.out new file mode 100644 index 00000000000000..da63c94a5abc17 --- /dev/null +++ b/regression-test/data/nereids_p0/compress_materialize/pushdown_encode.out @@ -0,0 +1,36 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !exec_sort_agg -- +a +b + +-- !exec_sort_filter -- +a +b + +-- !exec_agg_join -- +b + +-- !agg_join_2 -- +b 2 + +-- !nlj -- +a 9 +b 7 + +-- !union -- +1 +2 +2 +3 +4 + +-- !intersect -- +2 + +-- !except -- +1 + +-- !agg_sort -- +a +b + 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 10820e9ee48414..ecef42943dcedf 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 @@ -80,13 +80,11 @@ PhysicalResultSink -- !complex_join_condition1 -- PhysicalResultSink ---PhysicalUnion -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() -------PhysicalOlapScan[table_a] +--hashJoin[INNER_JOIN] hashCondition=((expr_(cast(id as BIGINT) - 1) = expr_cast(id as BIGINT))) otherCondition=() +----PhysicalUnion ------PhysicalOlapScan[table_b] -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() -------PhysicalOlapScan[table_a] ------PhysicalOlapScan[table_c] +----PhysicalOlapScan[table_a] -- !complex_join_condition2 -- PhysicalResultSink diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out index c1f86cac10185d..2c637ff79af530 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_category = 'MFGR#12')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#12')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'AMERICA')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'AMERICA')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out index 5b7b82f23355a2..a645aa975868a8 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((supplier.s_region = 'ASIA')) +----------------------------------PhysicalOlapScan[supplier] --------------------------PhysicalProject -----------------------------filter((supplier.s_region = 'ASIA')) -------------------------------PhysicalOlapScan[supplier] +----------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) +------------------------------PhysicalOlapScan[part] ----------------------PhysicalProject -------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) ---------------------------PhysicalOlapScan[part] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out index 0523fe55e5bbc9..35b07d5c9c95a9 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_brand = 'MFGR#2239')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_brand = 'MFGR#2239')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'EUROPE')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'EUROPE')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out index 7d0b454caac190..2be3aa25885ede 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter((supplier.s_nation = 'UNITED STATES')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter((customer.c_nation = 'UNITED STATES')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[lo_custkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter((customer.c_nation = 'UNITED STATES')) +--------------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------filter((supplier.s_nation = 'UNITED STATES')) +----------------------------PhysicalOlapScan[supplier] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out index 628f3df9a5831a..e66f2bb1a8e6c9 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out index f725ccdbc2c1f7..77a89293cd0d1c 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_yearmonth = 'Dec1997')) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_yearmonth = 'Dec1997')) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out index efc1e0061ed88d..669575fcff515b 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out @@ -1,32 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((supplier.s_region = 'AMERICA')) +--------------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_region = 'AMERICA')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------filter(d_year IN (1997, 1998)) +----------------------------------PhysicalOlapScan[dates] --------------------------PhysicalProject -----------------------------filter(d_year IN (1997, 1998)) -------------------------------PhysicalOlapScan[dates] +----------------------------filter((customer.c_region = 'AMERICA')) +------------------------------PhysicalOlapScan[customer] ----------------------PhysicalProject -------------------------filter((customer.c_region = 'AMERICA')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) -----------------------PhysicalOlapScan[part] +------------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out index 5bd9ad4782cb3d..fcbbd46edfae65 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out @@ -1,31 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF3 lo_custkey->[c_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF2 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF0 d_datekey->[lo_orderdate] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter(d_year IN (1997, 1998)) +--------------------------------------PhysicalOlapScan[dates] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_nation = 'UNITED STATES')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalOlapScan[customer] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#14')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_nation = 'UNITED STATES')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter(d_year IN (1997, 1998)) ---------------------------PhysicalOlapScan[dates] +------------------------filter((part.p_category = 'MFGR#14')) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out index dcc2e202e23752..99296a8b2b52ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out index 43c8732d7f8553..fc50c412111f88 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out index 78fd7c847c29ed..3745f45707a9b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out index 78fd7c847c29ed..3745f45707a9b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out index 61f29b11211346..6e6c71f54a16d5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out index 06c1b08293ef85..d8c6f24f30628c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out index 12fa11701b619f..275f764af4545a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out @@ -1,44 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF7 sr_customer_sk->[cs_bill_customer_sk];RF8 sr_item_sk->[cs_item_sk] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 RF7 RF8 +----------------------------PhysicalProject +------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (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_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF9 ------------------------------------PhysicalProject --------------------------------------filter((d1.d_quarter_name = '2001Q1')) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 +--------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 ------------------------------------PhysicalProject --------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out index ea401d9c36dc08..1b60930683fe9b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------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 broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) -------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query24.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query24.out index b1e5bf29828394..ac318f13888da5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query24.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query24.out @@ -7,28 +7,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------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:RF5 sr_ticket_number->[ss_ticket_number];RF6 sr_item_sk->[i_item_sk,ss_item_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF4 ca_address_sk->[c_current_addr_sk];RF5 ca_zip->[s_zip] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_zip = customer_address.ca_zip) and (store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ca_zip->[s_zip];RF3 c_customer_sk->[ss_customer_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_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 -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF3 RF4 RF5 RF6 +------------------------------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:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF6 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 ----------------------------PhysicalProject ------------------------------filter((store.s_market_id = 5)) ---------------------------------PhysicalOlapScan[store] apply RFs: RF2 +--------------------------------PhysicalOlapScan[store] apply RFs: RF5 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalOlapScan[customer] apply RFs: RF4 --------------------PhysicalProject -----------------------PhysicalOlapScan[item] apply RFs: RF6 +----------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject -------------------PhysicalOlapScan[store_returns] +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out index 47ceeb712c2a8c..5d29068f38988c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------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 -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((store.s_state = 'TN')) +--------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((store.s_state = 'TN')) -----------------------------------PhysicalOlapScan[store] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out index 8bea4f656a6132..35515fe0691a06 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'AR')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'AR')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out index c75d4fc3e18155..186cd53926f24a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] 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)) ---------------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:RF2 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((store.s_county = 'Williamson County')) -------------------------------------PhysicalOlapScan[store] -----------------------------PhysicalProject -------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] 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)) +----------------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:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((store.s_county = 'Williamson County')) +--------------------------------------PhysicalOlapScan[store] +------------------------------PhysicalProject +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out index 2f4be8c2912555..148ee99a2a16bc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out index 041e5711184598..c8c519df5c2f66 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 -----------------------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=((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 RF4 -------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 +------------------------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=((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 RF4 +--------------------------------PhysicalProject +----------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out index dcc2e202e23752..99296a8b2b52ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out index 8b807baf5409e4..2042698b5baf51 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out index f5c3f38463d42c..a0d1b0fcc71efa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out index 43c8732d7f8553..fc50c412111f88 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out index ebc2519e119218..627795684c1c9f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out @@ -11,11 +11,9 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ----------------PhysicalProject ------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 c_customer_sk->[ss_customer_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF17 p_promo_sk->[ss_promo_sk] +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF17 ca_address_sk->[ss_addr_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ss_addr_sk->[ca_address_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF16 +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF16 p_promo_sk->[ss_promo_sk] ----------------------------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:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] --------------------------------PhysicalProject @@ -34,7 +32,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------------------PhysicalProject --------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF7 RF8 RF9 RF11 RF12 RF13 RF17 RF18 +------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF7 RF8 RF9 RF11 RF12 RF13 RF16 RF17 RF18 ----------------------------------------------------------PhysicalProject ------------------------------------------------------------filter(d_year IN (1999, 2000)) --------------------------------------------------------------PhysicalOlapScan[date_dim] @@ -60,8 +58,10 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[promotion] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[promotion] +--------------------------PhysicalOlapScan[customer_address] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF4 c_current_addr_sk->[ca_address_sk] ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out index 924459179feee9..5114322d79c144 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[ws_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF6 w_warehouse_sk->[cs_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out index 31101f12eab21a..bc678a459c46a6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out index ec1bdd0e99afb6..81f39a4b46df1d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out index 473b9fded85715..00a991c21a3757 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out @@ -1,40 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject ------------------------------filter(cs_bill_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out index 97ae3085a37ff7..c09a4d2af44dd2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'TN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'TN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out index 5fb40519b131d5..855f62276766db 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -33,7 +33,8 @@ PhysicalResultSink ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalProject ----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out index d817c6f0053791..0fb037b0609dce 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------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:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------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:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out index 5825559155b8e7..1914adfa112b30 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out index c10cc616923d3c..8d000d6a01fe54 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out index dcda7d5d7eb3ee..2164dcd1cd27f3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out index 886eca75570635..a8be055a16bb89 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out index e956468874dbb5..c9d0e3d2e35648 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out index 15e7650bae15c3..bf993275fd7dbf 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out index c14ce7550a3069..6599fd90e6358c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out index 1465471817388b..9695f4e8e09a52 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_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=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] -----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_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=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[warehouse] +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out index 7270fe9092a53b..c41f2e4cd17fda 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out index 0db3fa841189b2..83259727b845b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out index 8bdf05df39ed98..247e35fea344e2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] -----------------------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out index fe17bd5d3c7235..2fe5d1ab00d5fa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[item] ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out index 596cb44922b54a..409a62a46e5c1f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out index e0bbeea823735d..90baf285121c7c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out index 9c9d7b7638d6a5..41c315a3884d8a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out index 2f21640b079929..b2c618967c7c09 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out @@ -1,33 +1,38 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ----------------------------PhysicalProject @@ -35,6 +40,6 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out index 08951b2ec22771..98f023f77c1beb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out index 4fdff8b37961c5..e9a49dc675c313 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -33,7 +33,8 @@ PhysicalResultSink ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalProject ----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out index e73d45b0732736..30d320756649eb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------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:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------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:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------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:RF14 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------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:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out index 81b0bae51498c1..7c98ba4724c8b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF1 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out index 52da90d84ff3a8..1a5abfa9ed9bc1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out index 22f67b07a4698c..68d3302b171fe0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out index 3eec2f7437212b..0aac3cae957c5f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------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 -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out index 748165ced2fb2a..53e93243b3467c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out index a19fe778a57647..e29fbf60a9a210 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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] -------------------------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 RF3 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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] +--------------------------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 RF3 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out index b414e22cb15150..0e31ac24a25ea0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out index aae0d788557045..36e05b9a00a315 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_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=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] -----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_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=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[warehouse] +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out index 68d1ef7855a7fd..dd13e12220dafe 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out index 0db3fa841189b2..83259727b845b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out index e7941ce875f02f..7617da1d0fa5f9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 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:RF3 s_store_sk->[ss_store_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] -----------------------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 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:RF3 s_store_sk->[ss_store_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out index 81aba1038a542a..fc50c412111f88 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF5 ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out index 7ef36371976ad6..2848ab93542957 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out index af6d7e8c85a5f6..6be757345e61ff 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out index 866c026a90dd67..4614292ea19fd1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out index 8f739a1d12b35a..4230cbe94d15d8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out @@ -1,40 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject ------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out index c06a48a1f5f922..e40e8f5035d91b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out index 4dfc2de4cf3fe5..5b1148d1d5d003 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out index 48ac240d961d98..17d3a28495675e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out index f5813cde38de7d..7f7c97a1087234 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out index 5342955a97aae2..b60d155992fdf7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out @@ -1,24 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF7 sr_customer_sk->[cs_bill_customer_sk];RF8 sr_item_sk->[cs_item_sk] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 RF7 RF8 +----------------------------PhysicalProject +------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() ----------------------------PhysicalProject @@ -39,6 +39,6 @@ PhysicalResultSink ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out index b1490c33c43896..915e0e08b3d456 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------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] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query24.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query24.out index 2c6c329803d85f..d4ee839c266bcb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query24.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query24.out @@ -7,28 +7,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------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=() +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip)) otherCondition=(( not (c_birth_country = upper(ca_country)))) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_zip = customer_address.ca_zip) and (store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ca_zip->[s_zip];RF3 c_customer_sk->[ss_customer_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------------------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 -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF3 +------------------------------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:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] ----------------------------PhysicalProject ------------------------------filter((store.s_market_id = 8)) ---------------------------------PhysicalOlapScan[store] apply RFs: RF2 +--------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalOlapScan[customer] --------------------PhysicalProject -----------------------PhysicalOlapScan[item] +----------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject -------------------PhysicalOlapScan[store_returns] +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out index 2dc8f171dee0d8..ade9c692f7ba35 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_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 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out index 9c11294b4ec73f..2b4f4cce8d9a4d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------filter((date_dim.d_year = 2002)) ------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out index 79c46ccb77bbb8..a7e60904f4ce5d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +--------------------------------------PhysicalOlapScan[household_demographics] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out index dea7b62c38003d..27e7bf5154ddca 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out index ade38048fb9732..0cb32067bebcba 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------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 -----------------------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=((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_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------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 +------------------------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=((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_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out index baafdf90cdef7a..7c90ea535de5df 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out index 7d0f6b1ce22d0f..90cf1cf013b3d7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out index 2f0a1b10cbff1b..a4c861e47385c9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------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 = d1.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out index ddb58c3887ed6f..4dbf4cdd0ad1fc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out index 6017253a3383c5..4c9e2b1f2eb798 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out index a68ff0c1138094..17033c2b549530 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out index ae5b26647980e7..3f4904e2692c08 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_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 <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out index 668c3625c56841..ef765627c390f7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out @@ -1,39 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ss_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 +------------------------PhysicalUnion +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------filter(ss_hdemo_sk IS NULL) +------------------------------------PhysicalOlapScan[store_sales] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------filter(ws_bill_addr_sk IS NULL) +------------------------------------PhysicalOlapScan[web_sales] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 cs_item_sk->[i_item_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------------PhysicalProject ------------------------------filter(cs_warehouse_sk IS NULL) --------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out index 809170997c3eed..2c5aebe5b8b460 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out index 4dfc2de4cf3fe5..5b1148d1d5d003 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out index 5aad6142d951f9..50b81c67589ae3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index f5813cde38de7d..7f7c97a1087234 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out index 7cc4a196c206c3..275f764af4545a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out @@ -1,24 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF7 sr_customer_sk->[cs_bill_customer_sk];RF8 sr_item_sk->[cs_item_sk] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 RF7 RF8 +----------------------------PhysicalProject +------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] ----------------------------PhysicalProject @@ -26,7 +26,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF9 ------------------------------------PhysicalProject --------------------------------------filter((d1.d_quarter_name = '2001Q1')) ----------------------------------------PhysicalOlapScan[date_dim] @@ -39,6 +39,6 @@ PhysicalResultSink ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out index b1490c33c43896..915e0e08b3d456 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------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] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out index 14dbf92e98e5fd..e30925e0a37aa9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out @@ -7,28 +7,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------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:RF5 sr_ticket_number->[ss_ticket_number];RF6 sr_item_sk->[i_item_sk,ss_item_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF4 ca_address_sk->[c_current_addr_sk];RF5 ca_zip->[s_zip] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_zip = customer_address.ca_zip) and (store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ca_zip->[s_zip];RF3 c_customer_sk->[ss_customer_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_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 -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF3 RF4 RF5 RF6 +------------------------------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:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF6 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 ----------------------------PhysicalProject ------------------------------filter((store.s_market_id = 8)) ---------------------------------PhysicalOlapScan[store] apply RFs: RF2 +--------------------------------PhysicalOlapScan[store] apply RFs: RF5 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalOlapScan[customer] apply RFs: RF4 --------------------PhysicalProject -----------------------PhysicalOlapScan[item] apply RFs: RF6 +----------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject -------------------PhysicalOlapScan[store_returns] +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out index 3ddc47823586a0..371285b5371c00 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN shuffle] 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_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_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 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out index b60e62f5e7bade..2b4f4cce8d9a4d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------filter((date_dim.d_year = 2002)) ------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out index 79c46ccb77bbb8..a7e60904f4ce5d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +--------------------------------------PhysicalOlapScan[household_demographics] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out index 9728d7d30707cd..8b2cd5d4640da2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out index 5ff27658e2ed3f..5d0867a399f520 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 -----------------------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=((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 RF4 -------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 +------------------------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=((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 RF4 +--------------------------------PhysicalProject +----------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out index baafdf90cdef7a..7c90ea535de5df 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out index 7d0f6b1ce22d0f..90cf1cf013b3d7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out index f5c3f38463d42c..a0d1b0fcc71efa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out index 43c8732d7f8553..aa6e2c2f2a5be0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out index a4ff984e4cdc43..f5c511ea739d4f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[ws_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF6 w_warehouse_sk->[cs_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out index a68ff0c1138094..17033c2b549530 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out index b1074dc6ffeed5..cd5b9f29b52f86 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out index 668c3625c56841..ef765627c390f7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out @@ -1,39 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ss_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 +------------------------PhysicalUnion +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------filter(ss_hdemo_sk IS NULL) +------------------------------------PhysicalOlapScan[store_sales] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------filter(ws_bill_addr_sk IS NULL) +------------------------------------PhysicalOlapScan[web_sales] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 cs_item_sk->[i_item_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------------PhysicalProject ------------------------------filter(cs_warehouse_sk IS NULL) --------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out index 27fc8430ff3e83..4069c82aded854 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out index 43aef07e0be2c0..28f642daf84c2e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out @@ -1,21 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] @@ -27,16 +27,17 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------filter(ca_county IN ('Bonneville County', 'Boone County', 'Brown County', 'Fillmore County', 'McPherson County')) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------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_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(ca_county IN ('Bonneville County', 'Boone County', 'Brown County', 'Fillmore County', 'McPherson County')) +----------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[customer_demographics] -------------------------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_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] ----------------------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 diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out index 9d5c47615cb77c..98b2ab8ce7db72 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------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:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------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:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out index 828af0129d7187..1080611b9b0903 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF1 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 1998)) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 1998)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out index e0b281146ad099..22eac9a0189c00 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out index 083105bf02dd54..31a5c9fa632852 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[cs_bill_cdemo_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[c_current_cdemo_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[c_current_cdemo_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF3 RF4 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF3 RF4 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter(c_birth_month IN (1, 4, 5, 7, 8, 9)) +--------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 ------------------------------------------PhysicalProject ---------------------------------------------filter(c_birth_month IN (1, 4, 5, 7, 8, 9)) -----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 +--------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------filter(ca_state IN ('AL', 'AR', 'GA', 'MS', 'NC', 'TX', 'WV')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------------------PhysicalProject -------------------------------------filter(ca_state IN ('AL', 'AR', 'GA', 'MS', 'NC', 'TX', 'WV')) ---------------------------------------PhysicalOlapScan[customer_address] +------------------------------------filter((cd1.cd_education_status = 'Unknown') and (cd1.cd_gender = 'M')) +--------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------PhysicalProject ---------------------------------filter((cd1.cd_education_status = 'Unknown') and (cd1.cd_gender = 'M')) -----------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 2002)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 2002)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out index f203bce45719dc..8128f2519de7e4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 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] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------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 RF3 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +------------------------------------filter((customer_demographics.cd_education_status = '2 yr Degree') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'U')) +--------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------PhysicalProject ---------------------------------filter((customer_demographics.cd_education_status = '2 yr Degree') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'U')) -----------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------filter((date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'FL', 'IN', 'NY', 'OH', 'SC')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'FL', 'IN', 'NY', 'OH', 'SC')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out index cf017069f995b3..4277be3da86219 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2000)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF2 ctr_state->[ctr_state] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------PhysicalProject -----------------filter((customer_address.ca_state = 'GA')) -------------------PhysicalOlapScan[customer_address] +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF2 ctr_state->[ctr_state] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------PhysicalProject +------------------filter((customer_address.ca_state = 'GA')) +--------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out index 654477132bf75e..1c1f58f0afc099 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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] -------------------------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 RF3 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Arthur County', 'Halifax County', 'Lunenburg County', 'Oglethorpe County', 'Perry County', 'Salem County', 'Sumner County', 'Terrell County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('>10000', 'Unknown')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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] +--------------------------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 RF3 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Arthur County', 'Halifax County', 'Lunenburg County', 'Oglethorpe County', 'Perry County', 'Salem County', 'Sumner County', 'Terrell County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('>10000', 'Unknown')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out index beedc2a19350ab..6a911c6ae43053 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out @@ -1,21 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject ------------------filter(OR[ifnull($c$1, FALSE),ifnull($c$2, FALSE)]) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] ----------------------------------PhysicalProject ------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] @@ -27,16 +27,16 @@ PhysicalResultSink ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] -------------------------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_qoy < 4) and (date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] +----------------------------------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_qoy < 4) and (date_dim.d_year = 2001)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_demographics] ----------------------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 diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out index 414f6e3e0bd2ed..8edcefaac5a058 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_OUTER_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] -----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2000-04-17') and (date_dim.d_date >= '2000-02-17')) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_OUTER_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[warehouse] +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------filter((date_dim.d_date <= '2000-04-17') and (date_dim.d_date >= '2000-02-17')) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out index 9f0c50ae0a648a..5dd385b2224009 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 1998)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out index 04db2001d08cf7..7a451dcaeecd39 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out index d2eda4d35debce..5ee8f79f48ece7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 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:RF3 s_store_sk->[ss_store_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] -----------------------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 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:RF3 s_store_sk->[ss_store_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 9) and (d2.d_year = 1998)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 9) and (d2.d_year = 1998)) -----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out index 0c390df2e8e5f8..4f8f9a3dbc3e9e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF5 ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out index e9d1404570f6ed..76d5bbe15122e8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) -----------------------------------PhysicalOlapScan[ship_mode] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out index 115564d2edf45e..d9c21f7a26316b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out index 3038640513b88d..24ea74e4aebc65 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1229) and (d1.d_month_seq >= 1218)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1229) and (date_dim.d_month_seq >= 1218)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1229) and (d1.d_month_seq >= 1218)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1229) and (date_dim.d_month_seq >= 1218)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out index 47d1baebaf8c15..26aaafbacdec21 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out @@ -1,40 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ws_ship_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------filter(ws_ship_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject ------------------------------filter(cs_ship_mode_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out index da27f4206dda1a..93d10836ae12e6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 1998)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF3 ctr_state->[ctr_state] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF4 -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalProject -----------------filter((customer_address.ca_state = 'TX')) -------------------PhysicalOlapScan[customer_address] +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF3 ctr_state->[ctr_state] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF4 +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------filter((customer_address.ca_state = 'TX')) +--------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q1.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out index 7b04caaf3e087a..95c7b4b338279c 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------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] +--------------------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 +------------------------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/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out index 176b64caf7833f..498641ff0de018 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q1.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out index f04b0bc766338b..3158a8c782c15f 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------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/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out index 207538326c7118..99242aaac78d55 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------PhysicalOlapScan[customer] +------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[orders] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out index f04b0bc766338b..3158a8c782c15f 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------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/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index 207538326c7118..99242aaac78d55 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------PhysicalOlapScan[customer] +------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[orders] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q1.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out index 7b04caaf3e087a..95c7b4b338279c 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------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] +--------------------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 +------------------------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/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out index 176b64caf7833f..498641ff0de018 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out b/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out index a8710941069079..c04d5e0d01e0dc 100644 --- a/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out +++ b/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out @@ -1,19 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] Hint log: Used: leading(orders lineitem ) diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out index c1f86cac10185d..2c637ff79af530 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_category = 'MFGR#12')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#12')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'AMERICA')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'AMERICA')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out index 5b7b82f23355a2..a645aa975868a8 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((supplier.s_region = 'ASIA')) +----------------------------------PhysicalOlapScan[supplier] --------------------------PhysicalProject -----------------------------filter((supplier.s_region = 'ASIA')) -------------------------------PhysicalOlapScan[supplier] +----------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) +------------------------------PhysicalOlapScan[part] ----------------------PhysicalProject -------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) ---------------------------PhysicalOlapScan[part] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out index 0523fe55e5bbc9..35b07d5c9c95a9 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_brand = 'MFGR#2239')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_brand = 'MFGR#2239')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'EUROPE')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'EUROPE')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out index 7d0b454caac190..2be3aa25885ede 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter((supplier.s_nation = 'UNITED STATES')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter((customer.c_nation = 'UNITED STATES')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[lo_custkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter((customer.c_nation = 'UNITED STATES')) +--------------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------filter((supplier.s_nation = 'UNITED STATES')) +----------------------------PhysicalOlapScan[supplier] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out index 628f3df9a5831a..e66f2bb1a8e6c9 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out index f725ccdbc2c1f7..77a89293cd0d1c 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_yearmonth = 'Dec1997')) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_yearmonth = 'Dec1997')) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out index efc1e0061ed88d..669575fcff515b 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out @@ -1,32 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((supplier.s_region = 'AMERICA')) +--------------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_region = 'AMERICA')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------filter(d_year IN (1997, 1998)) +----------------------------------PhysicalOlapScan[dates] --------------------------PhysicalProject -----------------------------filter(d_year IN (1997, 1998)) -------------------------------PhysicalOlapScan[dates] +----------------------------filter((customer.c_region = 'AMERICA')) +------------------------------PhysicalOlapScan[customer] ----------------------PhysicalProject -------------------------filter((customer.c_region = 'AMERICA')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) -----------------------PhysicalOlapScan[part] +------------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out index 5bd9ad4782cb3d..fcbbd46edfae65 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out @@ -1,31 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF3 lo_custkey->[c_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF2 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF0 d_datekey->[lo_orderdate] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter(d_year IN (1997, 1998)) +--------------------------------------PhysicalOlapScan[dates] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_nation = 'UNITED STATES')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalOlapScan[customer] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#14')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_nation = 'UNITED STATES')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter(d_year IN (1997, 1998)) ---------------------------PhysicalOlapScan[dates] +------------------------filter((part.p_category = 'MFGR#14')) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out index 5fb40519b131d5..855f62276766db 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -33,7 +33,8 @@ PhysicalResultSink ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalProject ----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out index 10192bf86cb782..b44b7db331182f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------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:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------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:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out index 5825559155b8e7..1914adfa112b30 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out index c10cc616923d3c..8d000d6a01fe54 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out index dcda7d5d7eb3ee..2164dcd1cd27f3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out index 886eca75570635..a8be055a16bb89 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out index 6f1b848ed00034..7afe19d0cbde7a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out index 15e7650bae15c3..bf993275fd7dbf 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out index 6def6ef536b340..d7b691ea5e2f4a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out index 1465471817388b..9695f4e8e09a52 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_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=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] -----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_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=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[warehouse] +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out index 7270fe9092a53b..c41f2e4cd17fda 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out index 0db3fa841189b2..83259727b845b6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out index 8bdf05df39ed98..247e35fea344e2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] -----------------------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out index 55144c3f93417e..c9ab430dee54d4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[item] ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out index 596cb44922b54a..409a62a46e5c1f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out index e0bbeea823735d..90baf285121c7c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out index 9c9d7b7638d6a5..41c315a3884d8a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out index 2f21640b079929..b2c618967c7c09 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out @@ -1,33 +1,38 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ----------------------------PhysicalProject @@ -35,6 +40,6 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out index 8e795ffc2e0eda..8db7858d58f48f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out index 4fdff8b37961c5..e9a49dc675c313 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -33,7 +33,8 @@ PhysicalResultSink ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalProject ----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out index 966f8701126465..1c57f1e9910105 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------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:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------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:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk,i_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[cs_item_sk] +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk,i_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] apply RFs: RF14 ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[cs_item_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF14 +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------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:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[i_item_sk,ws_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[ws_item_sk] +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[i_item_sk,ws_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] apply RFs: RF17 ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[ws_item_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF17 +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out index 81b0bae51498c1..7c98ba4724c8b8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF1 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalProject +----------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out index 52da90d84ff3a8..1a5abfa9ed9bc1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out index 22f67b07a4698c..68d3302b171fe0 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out index 3eec2f7437212b..0aac3cae957c5f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------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 -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out index 6671347af5cc6c..89847451ae9a67 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out index a19fe778a57647..e29fbf60a9a210 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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] -------------------------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 RF3 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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] +--------------------------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 RF3 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out index 83807f4b912bfe..e56d202787a622 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out index aae0d788557045..36e05b9a00a315 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] ---------------------------PhysicalProject -----------------------------hashJoin[LEFT_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=() -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] -----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_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=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[warehouse] +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out index 68d1ef7855a7fd..dd13e12220dafe 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out index 0db3fa841189b2..83259727b845b6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out index e7941ce875f02f..7617da1d0fa5f9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 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:RF3 s_store_sk->[ss_store_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] -----------------------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 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:RF3 s_store_sk->[ss_store_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out index 5f53b66408a500..ca5b3a32a70757 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF5 ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out index 7ef36371976ad6..2848ab93542957 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------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_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out index af6d7e8c85a5f6..6be757345e61ff 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out index 866c026a90dd67..4614292ea19fd1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out index 8f739a1d12b35a..4230cbe94d15d8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out @@ -1,40 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject ------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out index 8006799a3cc5fd..0286c6e8566c94 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out index 4dfc2de4cf3fe5..5b1148d1d5d003 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out index 2a29746e37ef07..60890e71af5712 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out index c070b7d34c00d4..67bc70aa67d9d6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) -------------------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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) +--------------------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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out index 5342955a97aae2..b60d155992fdf7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out @@ -1,24 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF7 sr_customer_sk->[cs_bill_customer_sk];RF8 sr_item_sk->[cs_item_sk] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 RF7 RF8 +----------------------------PhysicalProject +------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() ----------------------------PhysicalProject @@ -39,6 +39,6 @@ PhysicalResultSink ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out index 57183675eb5fc2..0929ca1a0ece88 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------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] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query24.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query24.out index 3d2e80e2381a00..d4ee839c266bcb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query24.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query24.out @@ -7,28 +7,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------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=() +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip)) otherCondition=(( not (c_birth_country = upper(ca_country)))) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_zip = customer_address.ca_zip) and (store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------------------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 -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +------------------------------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:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] ----------------------------PhysicalProject ------------------------------filter((store.s_market_id = 8)) --------------------------------PhysicalOlapScan[store] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(( not (c_birth_country = upper(ca_country)))) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[customer] --------------------PhysicalProject -----------------------PhysicalOlapScan[item] +----------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject -------------------PhysicalOlapScan[store_returns] +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out index 9b311ff91423bf..322e3efe5d76d3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_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 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out index 99f54520a25e12..9f2571fe3f97a0 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out index 79c46ccb77bbb8..a7e60904f4ce5d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +--------------------------------------PhysicalOlapScan[household_demographics] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out index 1e865046f6cf27..0831f0b661f181 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out index ade38048fb9732..0cb32067bebcba 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------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 -----------------------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=((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_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------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 +------------------------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=((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_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out index 40b25ae51ad929..fa7855af55f54e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out index 7d0f6b1ce22d0f..90cf1cf013b3d7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out index 2f0a1b10cbff1b..a4c861e47385c9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------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 = d1.d_date_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out index ddb58c3887ed6f..c13085d640d320 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out index 6017253a3383c5..4c9e2b1f2eb798 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out index a68ff0c1138094..17033c2b549530 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out index ae5b26647980e7..3f4904e2692c08 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_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 <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out index 668c3625c56841..ef765627c390f7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out @@ -1,39 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ss_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 +------------------------PhysicalUnion +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------filter(ss_hdemo_sk IS NULL) +------------------------------------PhysicalOlapScan[store_sales] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------filter(ws_bill_addr_sk IS NULL) +------------------------------------PhysicalOlapScan[web_sales] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 cs_item_sk->[i_item_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------------PhysicalProject ------------------------------filter(cs_warehouse_sk IS NULL) --------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out index fdc3edc4efb31f..0dafcf2058004a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out index 4dfc2de4cf3fe5..5b1148d1d5d003 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out index 196a98b5a2f51d..386242af1221bd 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk,i_item_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk] +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk,i_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] apply RFs: RF15 -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------------PhysicalOlapScan[item] apply RFs: RF15 +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[i_item_sk,ws_item_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ws_item_sk] +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[i_item_sk,ws_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ws_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] apply RFs: RF18 -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------PhysicalOlapScan[item] apply RFs: RF18 +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out index b93c82158d296a..45d2788e6df647 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out index 7cc4a196c206c3..275f764af4545a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out @@ -1,24 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF7 sr_customer_sk->[cs_bill_customer_sk];RF8 sr_item_sk->[cs_item_sk] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 RF7 RF8 +----------------------------PhysicalProject +------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] ----------------------------PhysicalProject @@ -26,7 +26,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF9 ------------------------------------PhysicalProject --------------------------------------filter((d1.d_quarter_name = '2001Q1')) ----------------------------------------PhysicalOlapScan[date_dim] @@ -39,6 +39,6 @@ PhysicalResultSink ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out index 57183675eb5fc2..0929ca1a0ece88 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------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] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query24.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query24.out index 0dc2c851744de2..e30925e0a37aa9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query24.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query24.out @@ -7,28 +7,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------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:RF5 sr_ticket_number->[ss_ticket_number];RF6 sr_item_sk->[i_item_sk,ss_item_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF4 ca_address_sk->[c_current_addr_sk];RF5 ca_zip->[s_zip] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_zip = customer_address.ca_zip) and (store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ca_zip->[s_zip];RF3 c_customer_sk->[ss_customer_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_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 -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF3 RF4 RF5 RF6 +------------------------------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:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF6 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 ----------------------------PhysicalProject ------------------------------filter((store.s_market_id = 8)) ---------------------------------PhysicalOlapScan[store] apply RFs: RF2 +--------------------------------PhysicalOlapScan[store] apply RFs: RF5 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[customer] apply RFs: RF4 --------------------PhysicalProject -----------------------PhysicalOlapScan[item] apply RFs: RF6 +----------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject -------------------PhysicalOlapScan[store_returns] +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out index c6137e774ae1b9..126222e256bdbb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN shuffleBucket] 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_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_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 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out index 1fd2b5a1688c12..5db7e0e9a2854b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out index 79c46ccb77bbb8..a7e60904f4ce5d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -------------------------------------PhysicalOlapScan[household_demographics] -----------------------------PhysicalProject -------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------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] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------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_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF0 hd_demo_sk->[ss_hdemo_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +--------------------------------------PhysicalOlapScan[household_demographics] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (1998, 1999, 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out index dc926eb4b522f0..cb9bf3d4816674 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out index 5ff27658e2ed3f..5d0867a399f520 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 -----------------------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=((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 RF4 -------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 +------------------------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=((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 RF4 +--------------------------------PhysicalProject +----------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out index 6bbc52ecb1a343..8870198c6f6525 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out index 7d0f6b1ce22d0f..90cf1cf013b3d7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out index f5c3f38463d42c..a0d1b0fcc71efa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out index 43c8732d7f8553..12a800f3c82d37 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out index a4ff984e4cdc43..f5c511ea739d4f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[ws_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF6 w_warehouse_sk->[cs_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out index a68ff0c1138094..17033c2b549530 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out index b1074dc6ffeed5..cd5b9f29b52f86 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out index 668c3625c56841..ef765627c390f7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out @@ -1,39 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 ss_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF2 +------------------------PhysicalUnion +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------filter(ss_hdemo_sk IS NULL) +------------------------------------PhysicalOlapScan[store_sales] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] ----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------filter(ws_bill_addr_sk IS NULL) +------------------------------------PhysicalOlapScan[web_sales] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 cs_sold_date_sk->[d_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +--------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 cs_item_sk->[i_item_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalOlapScan[item] apply RFs: RF3 ----------------------------PhysicalProject ------------------------------filter(cs_warehouse_sk IS NULL) --------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out index e6aef6266d392b..1ccd679f36f293 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out index 72acf1dc6b71ff..478fe2274b02fb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out index 43c8732d7f8553..ca5b3a32a70757 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out index 78fd7c847c29ed..3745f45707a9b6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out index 78fd7c847c29ed..3745f45707a9b6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] @@ -36,7 +36,8 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject ----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out index 61f29b11211346..6e6c71f54a16d5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out index fe0fd80c2f8b05..e42a77ec70fbe5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),ca_state IN ('CA', 'GA', 'WA'),(catalog_sales.cs_sales_price > 500.00)]) build RFs:RF2 c_customer_sk->[cs_bill_customer_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 +--------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------PhysicalProject +--------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out index 12fa11701b619f..275f764af4545a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out @@ -1,44 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF7 sr_customer_sk->[cs_bill_customer_sk];RF8 sr_item_sk->[cs_item_sk] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF6 RF7 RF8 +----------------------------PhysicalProject +------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (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_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF9 ------------------------------------PhysicalProject --------------------------------------filter((d1.d_quarter_name = '2001Q1')) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 +--------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 ------------------------------------PhysicalProject --------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] +------------------------------PhysicalOlapScan[item] +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out index ea401d9c36dc08..1b60930683fe9b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------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:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------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 broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------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] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) -------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query24.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query24.out index d93b73ade16ecb..ac318f13888da5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query24.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query24.out @@ -7,28 +7,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------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:RF5 sr_ticket_number->[ss_ticket_number];RF6 sr_item_sk->[i_item_sk,ss_item_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[ss_item_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF4 ca_address_sk->[c_current_addr_sk];RF5 ca_zip->[s_zip] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_zip = customer_address.ca_zip) and (store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ca_zip->[s_zip];RF3 c_customer_sk->[ss_customer_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_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 -------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF3 RF4 RF5 RF6 +------------------------------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:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF6 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 ----------------------------PhysicalProject ------------------------------filter((store.s_market_id = 5)) ---------------------------------PhysicalOlapScan[store] apply RFs: RF2 +--------------------------------PhysicalOlapScan[store] apply RFs: RF5 ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(( not (c_birth_country = upper(ca_country)))) build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[customer] apply RFs: RF4 --------------------PhysicalProject -----------------------PhysicalOlapScan[item] apply RFs: RF6 +----------------------PhysicalOlapScan[customer_address] ----------------PhysicalProject -------------------PhysicalOlapScan[store_returns] +------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out index 0fa387fb0d6bb9..7230028e9e726e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 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] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------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 -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((store.s_state = 'TN')) +--------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((store.s_state = 'TN')) -----------------------------------PhysicalOlapScan[store] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out index c894fcceff19a5..d88a3aa78d8a74 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'AR')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'AR')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out index c75d4fc3e18155..186cd53926f24a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] 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)) ---------------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:RF2 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((store.s_county = 'Williamson County')) -------------------------------------PhysicalOlapScan[store] -----------------------------PhysicalProject -------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] 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)) +----------------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:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((store.s_county = 'Williamson County')) +--------------------------------------PhysicalOlapScan[store] +------------------------------PhysicalProject +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalProject +----------------------------filter((date_dim.d_dom <= 28) and (date_dim.d_dom >= 1) and OR[(date_dim.d_dom <= 3),(date_dim.d_dom >= 25)] and d_year IN (2000, 2001, 2002)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out index 9012700621a358..a442be56643b23 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out index 041e5711184598..c8c519df5c2f66 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 -----------------------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=((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 RF4 -------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_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 +------------------------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=((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 RF4 +--------------------------------PhysicalProject +----------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +--------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out index 72acf1dc6b71ff..478fe2274b02fb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] -------------------------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 RF2 RF3 -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(OR[substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274'),$c$1]) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--------------------------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 RF2 RF3 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +--------------------------PhysicalProject ----------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out index 8b807baf5409e4..2042698b5baf51 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(OR[(return_rank <= 10),(currency_rank <= 10)]) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out index f5c3f38463d42c..a0d1b0fcc71efa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out @@ -1,29 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out index 43c8732d7f8553..ca5b3a32a70757 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out index 924459179feee9..5114322d79c144 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------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_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[ws_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF1 t_time_sk->[ws_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF6 w_warehouse_sk->[cs_warehouse_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF5 t_time_sk->[cs_sold_time_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +--------------------------------------------PhysicalOlapScan[time_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out index 31101f12eab21a..bc678a459c46a6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] ----------hashAgg[GLOBAL] ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out index ec1bdd0e99afb6..81f39a4b46df1d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out index 473b9fded85715..00a991c21a3757 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out @@ -1,40 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalUnion ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[cs_item_sk] ----------------------------PhysicalProject ------------------------------filter(cs_bill_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out index b3b6627dd0716d..3eddfb269717ea 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'TN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'TN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q1.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q1.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out index 7b04caaf3e087a..95c7b4b338279c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------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] +--------------------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 +------------------------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/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out index 63d82280b35b16..c3d504ca32c73c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q1.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q1.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out index f04b0bc766338b..3158a8c782c15f 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------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/new_shapes_p0/tpch_sf1000/rf_prune/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out index 63d82280b35b16..c3d504ca32c73c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q1.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q1.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out index f04b0bc766338b..3158a8c782c15f 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------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/new_shapes_p0/tpch_sf1000/shape/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out index 63d82280b35b16..c3d504ca32c73c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q1.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q1.out index 22f0777694a7ce..e126a122453f79 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q1.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q1.out @@ -1,13 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= '1998-09-02')) -------------------PhysicalOlapScan[lineitem] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((lineitem.l_shipdate <= '1998-09-02')) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out index 8df830dd428e58..99ea02d4359a94 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out index 7b04caaf3e087a..95c7b4b338279c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----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=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------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] +--------------------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 +------------------------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/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out index 63d82280b35b16..c3d504ca32c73c 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/suites/mv_p0/test_substr/test_substr.groovy b/regression-test/suites/mv_p0/test_substr/test_substr.groovy index 7d6ae82634fd8c..0bbd8b64c61bc7 100644 --- a/regression-test/suites/mv_p0/test_substr/test_substr.groovy +++ b/regression-test/suites/mv_p0/test_substr/test_substr.groovy @@ -18,8 +18,10 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite ("test_substr") { - sql """set enable_nereids_planner=true""" - sql """SET enable_fallback_to_original_planner=false""" + sql "set enable_nereids_planner=true;" + sql "SET enable_fallback_to_original_planner=false;" + sql "set enable_compress_materialize=false;" + sql """ drop table if exists dwd;""" sql """ diff --git a/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy b/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy index 8489de2aa2a1fd..3599220209c503 100644 --- a/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy +++ b/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy @@ -193,6 +193,5 @@ suite("compress_materialize") { qt_sort "select * from compressSort order by k desc, v"; qt_sort "select * from compressSort order by k desc nulls last"; qt_sort "select * from compressSort order by k desc nulls last, v limit 3"; - } diff --git a/regression-test/suites/nereids_p0/compress_materialize/pushdown_encode.groovy b/regression-test/suites/nereids_p0/compress_materialize/pushdown_encode.groovy new file mode 100644 index 00000000000000..265799339877b2 --- /dev/null +++ b/regression-test/suites/nereids_p0/compress_materialize/pushdown_encode.groovy @@ -0,0 +1,292 @@ +// 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("pushdown_encode") { +// push down encode slot + sql """ + drop table if exists t1; + CREATE TABLE t1 ( + `k1` int NOT NULL, + `v1` char(5) NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + insert into t1 values (1, "a"), (2, "b"); + + drop table if exists t2; + CREATE TABLE t2 ( + `k2` int NOT NULL, + `v2` char(5) NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k2`) + DISTRIBUTED BY HASH(`k2`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + insert into t2 values (3, "c"), (4, "d"), (2, "b"); + + set disable_join_reorder=true; + """ + + explain{ + sql """ + physical plan + select v1 + from (select sum(k1) as k, v1 from t1 group by v1) t + order by v1; + """ + contains("orderKeys=[encode_as_bigint(v1)#4 asc null first]") + contains("projects=[decode_as_varchar(encode_as_bigint(v1)#3) AS `v1`#1, encode_as_bigint(v1)#3 AS `encode_as_bigint(v1)`#4]") + contains("groupByExpr=[encode_as_bigint(v1)#3]") + contains("projects=[encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#3]") + } + + order_qt_exec_sort_agg """ + select v1 + from (select sum(k1) as k, v1 from t1 group by v1) t + order by v1; + """ + + explain{ + sql """ + physical plan + select v1 + from t1 + where k1 > 0 + order by v1; + """ + contains("orderKeys=[encode_as_bigint(v1)#2 asc null first]") + contains("projects=[decode_as_varchar(encode_as_bigint(v1#1)) AS `decode_as_varchar(encode_as_bigint(v1))`#1, encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#2]") + } + + order_qt_exec_sort_filter """ + select v1 + from t1 + where k1 > 0 + order by v1; + """ + + explain{ + sql """ + physical plan + select v1 + from t1 join t2 on v1=v2 + group by v1; + """ + contains("groupByExpr=[encode_as_bigint(v1)#4]") + contains("projects=[encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#4]") + } + + order_qt_exec_agg_join""" + select v1 + from t1 join t2 on v1=v2 + group by v1; + """ + + explain { + sql""" + physical plan + select v1 + from t1 join t2 on v1=v2 + group by v1; + """ + contains("groupByExpr=[encode_as_bigint(v1)#4]") + contains("projects=[encode_as_bigint(v1)#4]") + contains("hashCondition=[(encode_as_bigint(v1)#4 = encode_as_bigint(v2)#5)]") + contains("projects=[encode_as_bigint(v2#3) AS `encode_as_bigint(v2)`#5]") + contains("projects=[encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#4]") + } + + explain { + // because of "length(v1)>0", encode not pushed down through join + sql """ + physical plan + select v1 + from t1 left join t2 on v1=v2 and length(v1)>0 + group by v1; + """ + contains("hashCondition=[(v1#1 = v2#3)], otherCondition=[(length(v1#1) > 0)]") + } + + order_qt_agg_join_2 """ + select v1, sum(k2) + from t1 join t2 on v1=v2 + group by v1;""" + + explain { + sql """physical plan + select v1, sum(k2) + from t1 join t2 on v1=v2 + group by v1;""" + contains("projects=[decode_as_varchar(encode_as_bigint(v1)#5) AS `v1`#1, sum(k2)#4]") + contains("groupByExpr=[encode_as_bigint(v1)#5]") + contains("projects=[encode_as_bigint(v1)#5, k2#2]") + contains("hashCondition=[(encode_as_bigint(v1)#5 = encode_as_bigint(v2)#6)]") + contains("projects=[encode_as_bigint(v2#3) AS `encode_as_bigint(v2)`#6, k2#2]") + contains("projects=[encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#5]") + } + + explain { + sql """ + physical plan + select v1, sum(k2) + from t1 right outer join t2 on v1 < v2 + group by v1; + """ + contains("groupByExpr=[encode_as_bigint(v1)#5]") + contains("projects=[encode_as_bigint(v1)#5, k2#2]") + contains("otherCondition=[(encode_as_bigint(v1)#5 < encode_as_bigint(v2)#6)]") + contains("projects=[encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#5]") + contains("projects=[encode_as_bigint(v2#3) AS `encode_as_bigint(v2)`#6, k2#2]") + } + + + explain { + sql """ + physical plan + select v1, sum(k2) + from + (select t1.k1, t1.v1 from t1 join t2 on v1 < concat(v2,'a')) t3 + join t2 on t3.k1=t2.k2 + group by v1; + """ + contains("otherCondition=[(v1#1 < concat(v2, 'a')#8)]") + contains("projects=[k1#0, encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#7]") + + // +--PhysicalHashJoin[730]@7 ( stats=3, type=INNER_JOIN, hashCondition=[(k1#0 = k2#4)], otherCondition=[], markCondition=[], runtimeFilters=[RF0[k2#4->[k1#0](ndv/size = 3/4) , RF1[k2#4->[k1#0](ndv/size = 3/4) ] ) + // |--PhysicalProject[714]@4 ( stats=3, projects=[k1#0, encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#7] ) + // | +--PhysicalNestedLoopJoin[709]@3 ( stats=3, type=INNER_JOIN, hashCondition=[], otherCondition=[(v1#1 < concat(v2, 'a')#8)], markCondition=[] ) + // | |--PhysicalProject[698]@2 ( stats=3, projects=[concat(v2#3, 'a') AS `concat(v2, 'a')`#8] ) + // | | +--PhysicalOlapScan[t2]@1 ( stats=3 ) + // | +--PhysicalDistribute[704]@0 ( stats=2, distributionSpec=DistributionSpecReplicated ) + // | +--PhysicalOlapScan[t1]@0 ( stats=2, RFs= RF0 RF1 ) + // +--PhysicalDistribute[725]@6 ( stats=3, distributionSpec=DistributionSpecReplicated ) + // +--PhysicalProject[720]@6 ( stats=3, projects=[k2#4] ) + // +--PhysicalOlapScan[t2]@5 ( stats=3 ) + } + + + order_qt_nlj """ + select v1, sum(k2) + from t1 right outer join t2 on v1 < v2 and v2>"abc" + group by v1; + """ + + // not push through join, because v2>"abc" + sql """ + select v1, sum(k2) + from t1 right outer join t2 on v1 = v2 and (k1!=k2 or v2>"abc") + group by v1; + """ + explain { + sql """ + shape plan + select v1, sum(k2) + from t1 right outer join t2 on v1 = v2 and (k1!=k2 or v2>"abc") + group by v1; + """ + contains "hashCondition=((t1.v1 = t2.v2))" + } + + explain { + sql """ + physical plan + select k + from ( + (select k1 as k, v1 as v from t1) + union all + (select k2 as k, v2 as v from t2) + ) T + order by v; + """ + contains("orderKeys=[encode_as_bigint(v)#10 asc null first]") + contains("outputs=[k#8, encode_as_bigint(v)#10], regularChildrenOutputs=[[k#4, encode_as_bigint(v)#11], [k#6, encode_as_bigint(v)#12]]") + contains("projects=[k1#0 AS `k`#4, encode_as_bigint(v1#1) AS `encode_as_bigint(v)`#11]") + contains("projects=[k2#2 AS `k`#6, encode_as_bigint(v2#3) AS `encode_as_bigint(v)`#12]") + } + + order_qt_union """ + select k + from ( + (select k1 as k, v1 as v from t1) + union all + (select k2 as k, v2 as v from t2) + ) T + order by v; + """ + order_qt_intersect """ + select k + from ( + (select k1 as k, v1 as v from t1) + intersect + (select k2 as k, v2 as v from t2) + ) T + order by v; + """ + + order_qt_except """ + select k + from ( + (select k1 as k, v1 as v from t1) + except + (select k2 as k, v2 as v from t2) + ) T + order by v; + """ + + order_qt_agg_sort """ + select v1 + from (select v1 from t1 where k1 > 0 order by v1 limit 10) t group by v1 + """ + + explain{ + sql """ + physical plan + select v1 + from (select v1 from t1 where k1 > 0 order by v1 limit 10) t group by v1 + """ + contains("projects=[decode_as_varchar(encode_as_bigint(v1#1)) AS `decode_as_varchar(encode_as_bigint(v1))`#1, encode_as_bigint(v1#1) AS `encode_as_bigint(v1)`#3]") + } + + // if encodeBody is used in windowExpression, do not push encode down + + sql """ + SELECT v1, k1, k2 + , sum(k2) OVER (PARTITION BY v1 ORDER BY k1) AS w_sum + FROM t1 + JOIN t2 ON k1 = k2 - 2 + ORDER BY k1, v1, w_sum; + """ + + explain { + sql """ + physical plan + SELECT v1, k1, k2 + , sum(k2) OVER (PARTITION BY v1 ORDER BY k1) AS w_sum + FROM t1 + JOIN t2 ON k1 = k2 - 2 + ORDER BY k1, v1, w_sum; + """ + contains("orderKeys=[k1#0 asc null first, encode_as_bigint(v1)#5 asc null first, w_sum#4 asc null first]") + } + +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_gby_key.groovy b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_gby_key.groovy index 9139fea3c9db4a..342bb7484d259f 100644 --- a/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_gby_key.groovy +++ b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_gby_key.groovy @@ -64,7 +64,7 @@ suite("eliminate_gby_key") { select t2_c1 from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18]") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19]") } explain { @@ -84,7 +84,7 @@ suite("eliminate_gby_key") { select t2_c2 from temp; """) - contains("groupByExpr=[t2_c2#19, c1#13, c3#18], outputExpr=[t2_c2#19, c1#13, c3#18]") + contains("groupByExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19], outputExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19]") } explain { @@ -104,7 +104,7 @@ suite("eliminate_gby_key") { select c3 from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18]") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19]") } explain { @@ -124,7 +124,7 @@ suite("eliminate_gby_key") { select cnt from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19") } explain { @@ -144,7 +144,7 @@ suite("eliminate_gby_key") { select t2_c2, t2_c1 from temp; """) - contains("groupByExpr=[t2_c2#19, c1#13, c3#18], outputExpr=[t2_c2#19, c1#13, c3#18]") + contains("groupByExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19], outputExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19]") } explain { @@ -164,7 +164,7 @@ suite("eliminate_gby_key") { select c3, t2_c1 from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18]") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19]") } explain { @@ -184,7 +184,7 @@ suite("eliminate_gby_key") { select c3, t2_c2 from temp; """) - contains("groupByExpr=[t2_c2#19, c1#13, c3#18], outputExpr=[t2_c2#19, c1#13, c3#18]") + contains("groupByExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19], outputExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19]") } explain { @@ -204,7 +204,7 @@ suite("eliminate_gby_key") { select t2_c1, cnt from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19,") } explain { @@ -224,7 +224,7 @@ suite("eliminate_gby_key") { select c3, cnt from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19") } explain { @@ -244,7 +244,7 @@ suite("eliminate_gby_key") { select t2_c1, c3, cnt from temp; """) - contains("groupByExpr=[c1#13, c3#18], outputExpr=[c1#13, c3#18") + contains("groupByExpr=[c1#14, c3#19], outputExpr=[c1#14, c3#19") } explain { @@ -264,7 +264,7 @@ suite("eliminate_gby_key") { select t2_c2, c3, t2_c1 from temp; """) - contains("groupByExpr=[t2_c2#19, c1#13, c3#18], outputExpr=[t2_c2#19, c1#13, c3#18]") + contains("groupByExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19], outputExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19]") } explain { @@ -284,6 +284,6 @@ suite("eliminate_gby_key") { select t2_c2, c3, t2_c1, cnt from temp; """) - contains("groupByExpr=[t2_c2#19, c1#13, c3#18], outputExpr=[t2_c2#19, c1#13, c3#18,") + contains("groupByExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19], outputExpr=[encode_as_int(substring(c2, 1, 3))#20, c1#14, c3#19,") } } 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 5b8aa6e00d2308..33ffb02a47a792 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 @@ -19,6 +19,7 @@ suite("aggregate_with_roll_up") { String db = context.config.getDbNameByFile(context.file) sql "use ${db}" sql "set runtime_filter_mode=OFF"; + sql "set enable_compress_materialize=false;" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql """ diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy index 44815c8a1427f0..dcb966161e33c7 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy @@ -21,6 +21,7 @@ suite("any_value_roll_up") { sql "use ${db}" sql "set runtime_filter_mode=OFF"; sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set enable_compress_materialize=false;" sql """ drop table if exists orders_2 diff --git a/regression-test/suites/query_p0/limit/test_group_by_limit.groovy b/regression-test/suites/query_p0/limit/test_group_by_limit.groovy index d6f49f9d7a397f..076f3e203f83fd 100644 --- a/regression-test/suites/query_p0/limit/test_group_by_limit.groovy +++ b/regression-test/suites/query_p0/limit/test_group_by_limit.groovy @@ -17,105 +17,122 @@ suite("test_group_by_limit", "query") { -sql 'set enable_agg_spill=false' + sql 'set enable_agg_spill=false' -sql 'set enable_force_spill=false' + sql 'set enable_force_spill=false' -sql 'set topn_opt_limit_threshold=10' + sql 'set topn_opt_limit_threshold=10' -sql "set experimental_ENABLE_COMPRESS_MATERIALIZE=true;" + // different types + qt_select1 """ select sum(orderkey), count(partkey), shipmode from tpch_tiny_lineitem group by shipmode limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode from tpch_tiny_lineitem group by shipmode limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode)]") + } -// different types -qt_select1 """ select sum(orderkey), count(partkey), shipmode from tpch_tiny_lineitem group by shipmode limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode from tpch_tiny_lineitem group by shipmode limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + qt_select2 """ select sum(orderkey), count(partkey), linenumber from tpch_tiny_lineitem group by linenumber limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), linenumber from tpch_tiny_lineitem group by linenumber limit 3; " + contains("VTOP-N") + contains("sortInfo:[linenumber]") + } -qt_select2 """ select sum(orderkey), count(partkey), linenumber from tpch_tiny_lineitem group by linenumber limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), linenumber from tpch_tiny_lineitem group by linenumber limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + qt_select3 """ select sum(orderkey), count(partkey), tax from tpch_tiny_lineitem group by tax limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), tax from tpch_tiny_lineitem group by tax limit 3; " + contains("VTOP-N") + contains("sortInfo:[tax]") + } -qt_select3 """ select sum(orderkey), count(partkey), tax from tpch_tiny_lineitem group by tax limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), tax from tpch_tiny_lineitem group by tax limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + qt_select4 """ select sum(orderkey), count(partkey), commitdate from tpch_tiny_lineitem group by commitdate limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), commitdate from tpch_tiny_lineitem group by commitdate limit 3; " + contains("VTOP-N") + contains("sortInfo:[commitdate]") + } -qt_select4 """ select sum(orderkey), count(partkey), commitdate from tpch_tiny_lineitem group by commitdate limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), commitdate from tpch_tiny_lineitem group by commitdate limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + // group by functions + qt_select5 """ select sum(orderkey), count(partkey), cast(commitdate as datetime) from tpch_tiny_lineitem group by cast(commitdate as datetime) limit 3; """ + explain { + sql " select sum(orderkey), count(partkey), cast(commitdate as datetime) from tpch_tiny_lineitem group by cast(commitdate as datetime) limit 3; " + contains("VTOP-N") + contains("sortInfo:[cast(commitdate as DATETIMEV2(0))]") + } -// group by functions -qt_select5 """ select sum(orderkey), count(partkey), cast(commitdate as datetime) from tpch_tiny_lineitem group by cast(commitdate as datetime) limit 3; """ -explain { - sql " select sum(orderkey), count(partkey), cast(commitdate as datetime) from tpch_tiny_lineitem group by cast(commitdate as datetime) limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + qt_select6 """ select sum(orderkey), count(partkey), month(commitdate) from tpch_tiny_lineitem group by month(commitdate) limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), month(commitdate) from tpch_tiny_lineitem group by month(commitdate) limit 3; " + contains("VTOP-N") + contains("sortInfo:[month(commitdate)]") + } -qt_select6 """ select sum(orderkey), count(partkey), month(commitdate) from tpch_tiny_lineitem group by month(commitdate) limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), month(commitdate) from tpch_tiny_lineitem group by month(commitdate) limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + // mutli column + qt_select7 """ select sum(orderkey), count(partkey), shipmode, linenumber from tpch_tiny_lineitem group by shipmode, linenumber limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber from tpch_tiny_lineitem group by shipmode, linenumber limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber]") + } + qt_select8 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber, tax]") + } + qt_select9 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax , commitdate from tpch_tiny_lineitem group by shipmode, linenumber, tax, commitdate limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax , commitdate from tpch_tiny_lineitem group by shipmode, linenumber, tax, commitdate limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber, tax, commitdate]") + } -// mutli column -qt_select7 """ select sum(orderkey), count(partkey), shipmode, linenumber from tpch_tiny_lineitem group by shipmode, linenumber limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber from tpch_tiny_lineitem group by shipmode, linenumber limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} -qt_select8 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} -qt_select9 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax , commitdate from tpch_tiny_lineitem group by shipmode, linenumber, tax, commitdate limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax , commitdate from tpch_tiny_lineitem group by shipmode, linenumber, tax, commitdate limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + // group by + order by -// group by + order by + // group by columns eq order by columns + qt_select10 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode, linenumber, tax limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode, linenumber, tax limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber, tax]") + } + // group by columns contains order by columns + qt_select11 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber, tax]") + } + // desc order by column + qt_select12 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber, tax]") + } + qt_select13 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax desc limit 3; """ + explain{ + sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax desc limit 3; " + contains("VTOP-N") + contains("sortInfo:[encode_as_largeint(shipmode), linenumber, tax]") + } -// group by columns eq order by columns -qt_select10 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode, linenumber, tax limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode, linenumber, tax limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} -// group by columns contains order by columns -qt_select11 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} -// desc order by column -qt_select12 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} -qt_select13 """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax desc limit 3; """ -explain{ - sql " select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by shipmode, linenumber, tax order by shipmode desc, linenumber, tax desc limit 3; " - contains("VTOP-N") - contains("sortByGroupKey:true") -} + explain { + sql """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by rollup(shipmode, linenumber, tax) order by shipmode desc, linenumber, tax desc limit 3; """ + contains("sortInfo:null") + } + + explain { + sql """ select sum(orderkey), count(partkey), shipmode, linenumber , tax from tpch_tiny_lineitem group by rollup(shipmode, linenumber, tax) limit 3; """ + notContains("VTOP-N") + contains("sortInfo:null") + } + + explain { + sql """ select sum(orderkey), count(partkey) from tpch_tiny_lineitem limit 3; """ + notContains("VTOP-N") + contains("sortInfo:null") + } + + }