diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java index 9bf097f994ccf6..484ab16b509427 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.analyzer; +import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; @@ -143,6 +144,6 @@ public LogicalProperties computeLogicalProperties() { @Override public List computeOutput() { - return child().getOutput(); + throw new UnboundException("output"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java new file mode 100644 index 00000000000000..9743d0fe8f5eb4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java @@ -0,0 +1,89 @@ +// 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.analyzer; + +import org.apache.doris.nereids.exceptions.UnboundException; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Optional; + +/** + * unbound result sink + */ +public class UnboundResultSink extends LogicalSink implements Unbound, Sink { + + public UnboundResultSink(CHILD_TYPE child) { + super(PlanType.LOGICAL_UNBOUND_RESULT_SINK, child); + } + + public UnboundResultSink(Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_UNBOUND_RESULT_SINK, groupExpression, logicalProperties, child); + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); + return new UnboundResultSink<>(groupExpression, Optional.of(getLogicalProperties()), children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitUnboundResultSink(this, context); + } + + @Override + public List getExpressions() { + throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new UnboundResultSink<>(groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); + return new UnboundResultSink<>(groupExpression, logicalProperties, children.get(0)); + + } + + @Override + public List computeOutput() { + throw new UnboundException("output"); + } + + @Override + public String toString() { + return Utils.toSqlString("UnboundResultSink[" + id.asInt() + "]"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 40d08cddefc9bd..8fa4fdaa9e0010 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -112,6 +112,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalQuickSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSchemaScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; @@ -318,6 +319,12 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d * sink Node, in lexicographical order * ******************************************************************************************** */ + @Override + public PlanFragment visitPhysicalResultSink(PhysicalResultSink physicalResultSink, + PlanTranslatorContext context) { + return physicalResultSink.child().accept(this, context); + } + @Override public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, PlanTranslatorContext context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 696ac16f6f36bc..1f578dd554bb31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -120,6 +120,7 @@ import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.analyzer.UnboundStar; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -311,7 +312,12 @@ public LogicalPlan visitSingleStatement(SingleStatementContext ctx) { @Override public LogicalPlan visitStatementDefault(StatementDefaultContext ctx) { LogicalPlan plan = plan(ctx.query()); - return withExplain(withOutFile(plan, ctx.outFileClause()), ctx.explain()); + if (ctx.outFileClause() != null) { + plan = withOutFile(plan, ctx.outFileClause()); + } else { + plan = new UnboundResultSink<>(plan); + } + return withExplain(plan, ctx.explain()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java index 11375064f474fc..9b5603fe86f9a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java @@ -34,6 +34,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -102,6 +103,12 @@ public Void visitPhysicalOlapTableSink(PhysicalOlapTableSink ola return null; } + @Override + public Void visitPhysicalResultSink(PhysicalResultSink physicalResultSink, PlanContext context) { + addRequestPropertyToChildren(PhysicalProperties.GATHER); + return null; + } + /* ******************************************************************************************** * Other Node, in lexicographical order * ******************************************************************************************** */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index acdf9f790a5103..f3e9bf5c55b8d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -62,6 +62,7 @@ import org.apache.doris.nereids.rules.implementation.LogicalPartitionTopNToPhysicalPartitionTopN; import org.apache.doris.nereids.rules.implementation.LogicalProjectToPhysicalProject; import org.apache.doris.nereids.rules.implementation.LogicalRepeatToPhysicalRepeat; +import org.apache.doris.nereids.rules.implementation.LogicalResultSinkToPhysicalResultSink; import org.apache.doris.nereids.rules.implementation.LogicalSchemaScanToPhysicalSchemaScan; import org.apache.doris.nereids.rules.implementation.LogicalSortToPhysicalQuickSort; import org.apache.doris.nereids.rules.implementation.LogicalTVFRelationToPhysicalTVFRelation; @@ -161,6 +162,7 @@ public class RuleSet { .add(new LogicalGenerateToPhysicalGenerate()) .add(new LogicalOlapTableSinkToPhysicalOlapTableSink()) .add(new LogicalFileSinkToPhysicalFileSink()) + .add(new LogicalResultSinkToPhysicalResultSink()) .build(); public static final List ZIG_ZAG_TREE_JOIN_REORDER = planRuleFactories() 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 422c566a9238f3..2d8f913e88a191 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 @@ -29,6 +29,7 @@ public enum RuleType { // binding rules // **** make sure BINDING_UNBOUND_LOGICAL_PLAN is the lowest priority in the rewrite rules. **** + BINDING_RESULT_SINK(RuleTypeClass.REWRITE), BINDING_NON_LEAF_LOGICAL_PLAN(RuleTypeClass.REWRITE), BINDING_ONE_ROW_RELATION_SLOT(RuleTypeClass.REWRITE), BINDING_RELATION(RuleTypeClass.REWRITE), @@ -299,6 +300,7 @@ public enum RuleType { LOGICAL_JDBC_SCAN_TO_PHYSICAL_JDBC_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ES_SCAN_TO_PHYSICAL_ES_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_OLAP_TABLE_SINK_TO_PHYSICAL_OLAP_TABLE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_RESULT_SINK_TO_PHYSICAL_RESULT_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_FILE_SINK_TO_PHYSICAL_FILE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ASSERT_NUM_ROWS_TO_PHYSICAL_ASSERT_NUM_ROWS(RuleTypeClass.IMPLEMENTATION), STORAGE_LAYER_AGGREGATE_WITHOUT_PROJECT(RuleTypeClass.IMPLEMENTATION), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 9cb49f1c763040..f10bab4e10975a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -65,6 +65,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; 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.LogicalResultSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; @@ -555,6 +556,14 @@ protected boolean condition(Rule rule, Plan plan) { checkSameNameSlot(subQueryAlias.child(0).getOutput(), subQueryAlias.getAlias()); return subQueryAlias; }) + ), + RuleType.BINDING_RESULT_SINK.build( + unboundResultSink().then(sink -> { + List outputExprs = sink.child().getOutput().stream() + .map(NamedExpression.class::cast) + .collect(ImmutableList.toImmutableList()); + return new LogicalResultSink<>(outputExprs, sink.child()); + }) ) ).stream().map(ruleCondition).collect(ImmutableList.toImmutableList()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 00fc9b4c2c86f2..68e81ab292a5a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -30,6 +30,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.pattern.MatchingContext; @@ -238,6 +239,10 @@ private Plan parseAndAnalyzeHiveView(TableIf table, CascadesContext cascadesCont private Plan parseAndAnalyzeView(String viewSql, CascadesContext parentContext) { LogicalPlan parsedViewPlan = new NereidsParser().parseSingle(viewSql); + // TODO: use a good to do this, such as eliminate UnboundResultSink + if (parsedViewPlan instanceof UnboundResultSink) { + parsedViewPlan = (LogicalPlan) ((UnboundResultSink) parsedViewPlan).child(); + } CascadesContext viewContext = CascadesContext.initContext( parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); viewContext.newAnalyzer().analyze(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java new file mode 100644 index 00000000000000..47bbad63b38620 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.implementation; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; + +import java.util.Optional; + +/** + * implement result sink. + */ +public class LogicalResultSinkToPhysicalResultSink extends OneImplementationRuleFactory { + @Override + public Rule build() { + return logicalResultSink().thenApply(ctx -> { + LogicalResultSink sink = ctx.root; + return new PhysicalResultSink<>( + sink.getOutputExprs(), + Optional.empty(), + sink.getLogicalProperties(), + sink.child()); + }).toRule(RuleType.LOGICAL_RESULT_SINK_TO_PHYSICAL_RESULT_SINK_RULE); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java index 93cbf2f7940489..7887836475c821 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java @@ -63,8 +63,7 @@ public Plan visitLogicalCTEAnchor(LogicalCTEAnchor logicalSink, StatementContext context) { - Plan child = logicalSink.child().accept(this, context); - return logicalSink.withChildren(child); + return super.visit(logicalSink, context); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java index 356333e4f69907..c6461be27186fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; -import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; @@ -72,11 +71,4 @@ public LogicalCTEProducer visitLogicalCTEProducer(LogicalCTEProducer logicalSink, List> producers) { - return logicalSink.withChildren(rewriteRoot(logicalSink.child(), producers)); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 2c3a68b093c74e..da98e0e4d14f4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -21,94 +21,104 @@ * Types for all Plan in Nereids. */ public enum PlanType { + // special + GROUP_PLAN, UNKNOWN, - // logical plan - LOGICAL_OLAP_TABLE_SINK, - LOGICAL_CTE, - LOGICAL_WINDOW, - LOGICAL_SUBQUERY_ALIAS, - LOGICAL_UNBOUND_ONE_ROW_RELATION, + // logical plans + // logical relations + LOGICAL_BOUND_RELATION, + LOGICAL_CTE_CONSUMER, + LOGICAL_FILE_SCAN, LOGICAL_EMPTY_RELATION, + LOGICAL_ES_SCAN, + LOGICAL_JDBC_SCAN, + LOGICAL_OLAP_SCAN, LOGICAL_ONE_ROW_RELATION, + LOGICAL_SCHEMA_SCAN, + LOGICAL_TVF_RELATION, + LOGICAL_UNBOUND_ONE_ROW_RELATION, LOGICAL_UNBOUND_RELATION, LOGICAL_UNBOUND_TVF_RELATION, - LOGICAL_BOUND_RELATION, + + // logical sinks + LOGICAL_FILE_SINK, + LOGICAL_OLAP_TABLE_SINK, + LOGICAL_RESULT_SINK, LOGICAL_UNBOUND_OLAP_TABLE_SINK, - LOGICAL_TVF_RELATION, - LOGICAL_PROJECT, + LOGICAL_UNBOUND_RESULT_SINK, + + // logical others + LOGICAL_AGGREGATE, + LOGICAL_APPLY, + LOGICAL_ASSERT_NUM_ROWS, + LOGICAL_CHECK_POLICY, + LOGICAL_CTE, + LOGICAL_CTE_ANCHOR, + LOGICAL_CTE_PRODUCER, + LOGICAL_EXCEPT, LOGICAL_FILTER, LOGICAL_GENERATE, + LOGICAL_HAVING, + LOGICAL_INTERSECT, LOGICAL_JOIN, - LOGICAL_AGGREGATE, + LOGICAL_LIMIT, + LOGICAL_MULTI_JOIN, + LOGICAL_PARTITION_TOP_N, + LOGICAL_PROJECT, LOGICAL_REPEAT, + LOGICAL_SELECT_HINT, + LOGICAL_SUBQUERY_ALIAS, LOGICAL_SORT, LOGICAL_TOP_N, - LOGICAL_PARTITION_TOP_N, - LOGICAL_LIMIT, - LOGICAL_OLAP_SCAN, - LOGICAL_SCHEMA_SCAN, - LOGICAL_FILE_SCAN, - LOGICAL_JDBC_SCAN, - LOGICAL_ES_SCAN, - LOGICAL_APPLY, - LOGICAL_SELECT_HINT, - LOGICAL_ASSERT_NUM_ROWS, - LOGICAL_HAVING, - LOGICAL_MULTI_JOIN, - LOGICAL_CHECK_POLICY, LOGICAL_UNION, - LOGICAL_EXCEPT, - LOGICAL_INTERSECT, LOGICAL_USING_JOIN, - LOGICAL_CTE_RELATION, - LOGICAL_CTE_ANCHOR, - LOGICAL_CTE_PRODUCER, - LOGICAL_CTE_CONSUMER, - LOGICAL_FILE_SINK, - - GROUP_PLAN, + LOGICAL_WINDOW, - // physical plan - PHYSICAL_OLAP_TABLE_SINK, - PHYSICAL_CTE_PRODUCE, - PHYSICAL_CTE_CONSUME, - PHYSICAL_CTE_ANCHOR, - PHYSICAL_WINDOW, + // physical plans + // logical relations + PHYSICAL_CTE_CONSUMER, PHYSICAL_EMPTY_RELATION, - PHYSICAL_ONE_ROW_RELATION, - PHYSICAL_OLAP_SCAN, + PHYSICAL_ES_SCAN, PHYSICAL_FILE_SCAN, PHYSICAL_JDBC_SCAN, - PHYSICAL_ES_SCAN, - PHYSICAL_TVF_RELATION, + PHYSICAL_ONE_ROW_RELATION, + PHYSICAL_OLAP_SCAN, PHYSICAL_SCHEMA_SCAN, - PHYSICAL_PROJECT, + PHYSICAL_TVF_RELATION, + + // logical sinks + PHYSICAL_FILE_SINK, + PHYSICAL_OLAP_TABLE_SINK, + PHYSICAL_RESULT_SINK, + + // logical others + PHYSICAL_HASH_AGGREGATE, + PHYSICAL_ASSERT_NUM_ROWS, + PHYSICAL_CTE_PRODUCER, + PHYSICAL_CTE_ANCHOR, + PHYSICAL_DISTRIBUTE, + PHYSICAL_EXCEPT, PHYSICAL_FILTER, PHYSICAL_GENERATE, - PHYSICAL_BROADCAST_HASH_JOIN, - PHYSICAL_AGGREGATE, + PHYSICAL_INTERSECT, + PHYSICAL_HASH_JOIN, + PHYSICAL_NESTED_LOOP_JOIN, + PHYSICAL_LIMIT, + PHYSICAL_PARTITION_TOP_N, + PHYSICAL_PROJECT, PHYSICAL_REPEAT, + PHYSICAL_LOCAL_QUICK_SORT, PHYSICAL_QUICK_SORT, PHYSICAL_TOP_N, - PHYSICAL_PARTITION_TOP_N, - PHYSICAL_LOCAL_QUICK_SORT, - PHYSICAL_LIMIT, - PHYSICAL_HASH_JOIN, - PHYSICAL_NESTED_LOOP_JOIN, - PHYSICAL_EXCHANGE, - PHYSICAL_DISTRIBUTION, - PHYSICAL_ASSERT_NUM_ROWS, PHYSICAL_UNION, - PHYSICAL_EXCEPT, - PHYSICAL_INTERSECT, - PHYSICAL_FILE_SINK, + PHYSICAL_WINDOW, - COMMAND, - EXPLAIN_COMMAND, + // commands CREATE_POLICY_COMMAND, - INSERT_INTO_TABLE_COMMAND, - UPDATE_COMMAND, DELETE_COMMAND, - SELECT_INTO_OUTFILE_COMMAND + EXPLAIN_COMMAND, + INSERT_INTO_TABLE_COMMAND, + SELECT_INTO_OUTFILE_COMMAND, + UPDATE_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java index 86f58ce4c1d271..c96d71e5daa0b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java @@ -52,7 +52,7 @@ public class LogicalCTEConsumer extends LogicalRelation { */ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, Map consumerToProducerOutputMap, Map producerToConsumerOutputMap) { - super(relationId, PlanType.LOGICAL_CTE_RELATION, Optional.empty(), Optional.empty()); + super(relationId, PlanType.LOGICAL_CTE_CONSUMER, Optional.empty(), Optional.empty()); this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); this.name = Objects.requireNonNull(name, "name should not null"); this.consumerToProducerOutputMap = Objects.requireNonNull(consumerToProducerOutputMap, @@ -65,7 +65,7 @@ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, * Logical CTE consumer. */ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, LogicalPlan producerPlan) { - super(relationId, PlanType.LOGICAL_CTE_RELATION, Optional.empty(), Optional.empty()); + super(relationId, PlanType.LOGICAL_CTE_CONSUMER, Optional.empty(), Optional.empty()); this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); this.name = Objects.requireNonNull(name, "name should not null"); this.consumerToProducerOutputMap = new LinkedHashMap<>(); @@ -79,7 +79,7 @@ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, Logic public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, Map consumerToProducerOutputMap, Map producerToConsumerOutputMap, Optional groupExpression, Optional logicalProperties) { - super(relationId, PlanType.LOGICAL_CTE_RELATION, groupExpression, logicalProperties); + super(relationId, PlanType.LOGICAL_CTE_CONSUMER, groupExpression, logicalProperties); this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); this.name = Objects.requireNonNull(name, "name should not null"); this.consumerToProducerOutputMap = Objects.requireNonNull(consumerToProducerOutputMap, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java new file mode 100644 index 00000000000000..c21422e858a8c2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java @@ -0,0 +1,122 @@ +// 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.trees.plans.logical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +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.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * result sink + */ +public class LogicalResultSink extends LogicalSink implements Sink { + + private final List outputExprs; + + public LogicalResultSink(List outputExprs, CHILD_TYPE child) { + super(PlanType.LOGICAL_RESULT_SINK, child); + this.outputExprs = outputExprs; + } + + public LogicalResultSink(List outputExprs, + Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_RESULT_SINK, groupExpression, logicalProperties, child); + this.outputExprs = outputExprs; + } + + public List getOutputExprs() { + return outputExprs; + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "LogicalResultSink's children size must be 1, but real is %s", children.size()); + return new LogicalResultSink<>(outputExprs, children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalResultSink(this, context); + } + + @Override + public List getExpressions() { + return outputExprs; + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new LogicalResultSink<>(outputExprs, groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); + return new LogicalResultSink<>(outputExprs, groupExpression, logicalProperties, children.get(0)); + } + + @Override + public List computeOutput() { + return outputExprs.stream() + .map(NamedExpression::toSlot) + .collect(ImmutableList.toImmutableList()); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalResultSink[" + id.asInt() + "]", + "outputExprs", outputExprs); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + LogicalResultSink that = (LogicalResultSink) o; + return Objects.equals(outputExprs, that.outputExprs); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), outputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java index 2c0d9c6925a172..de9d8879851f85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java @@ -76,7 +76,7 @@ public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, Map consumerToProducerSlotMap, Map producerToConsumerSlotMap, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(relationId, PlanType.PHYSICAL_CTE_CONSUME, groupExpression, + super(relationId, PlanType.PHYSICAL_CTE_CONSUMER, groupExpression, logicalProperties, physicalProperties, statistics); this.cteId = cteId; this.consumerToProducerSlotMap = ImmutableMap.copyOf(Objects.requireNonNull( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java index 787649519d9809..2f57716179db59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java @@ -54,7 +54,8 @@ public PhysicalCTEProducer(CTEId cteId, Optional groupExpressio public PhysicalCTEProducer(CTEId cteId, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { - super(PlanType.PHYSICAL_CTE_PRODUCE, groupExpression, logicalProperties, physicalProperties, statistics, child); + super(PlanType.PHYSICAL_CTE_PRODUCER, groupExpression, + logicalProperties, physicalProperties, statistics, child); this.cteId = cteId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java index ce6a0411b260b1..7c97cafa7fbb14 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java @@ -58,14 +58,14 @@ public PhysicalDistribute(DistributionSpec spec, LogicalProperties logicalProper public PhysicalDistribute(DistributionSpec spec, Optional groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) { - super(PlanType.PHYSICAL_DISTRIBUTION, groupExpression, logicalProperties, child); + super(PlanType.PHYSICAL_DISTRIBUTE, groupExpression, logicalProperties, child); this.distributionSpec = spec; } public PhysicalDistribute(DistributionSpec spec, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { - super(PlanType.PHYSICAL_DISTRIBUTION, groupExpression, logicalProperties, physicalProperties, statistics, + super(PlanType.PHYSICAL_DISTRIBUTE, groupExpression, logicalProperties, physicalProperties, statistics, child); this.distributionSpec = spec; } 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 9a408bb9ab85bb..5217fe16fe04be 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 @@ -96,7 +96,7 @@ public PhysicalHashAggregate(List groupByExpressions, List> partitionExpressions, AggregateParam aggregateParam, boolean maybeUsingStream, Optional groupExpression, LogicalProperties logicalProperties, RequireProperties requireProperties, CHILD_TYPE child) { - super(PlanType.PHYSICAL_AGGREGATE, groupExpression, logicalProperties, child); + super(PlanType.PHYSICAL_HASH_AGGREGATE, groupExpression, logicalProperties, child); this.groupByExpressions = ImmutableList.copyOf( Objects.requireNonNull(groupByExpressions, "groupByExpressions cannot be null")); this.outputExpressions = ImmutableList.copyOf( @@ -122,7 +122,7 @@ public PhysicalHashAggregate(List groupByExpressions, List groupExpression, LogicalProperties logicalProperties, RequireProperties requireProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { - super(PlanType.PHYSICAL_AGGREGATE, groupExpression, logicalProperties, physicalProperties, statistics, + super(PlanType.PHYSICAL_HASH_AGGREGATE, groupExpression, logicalProperties, physicalProperties, statistics, child); this.groupByExpressions = ImmutableList.copyOf( Objects.requireNonNull(groupByExpressions, "groupByExpressions cannot be null")); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java new file mode 100644 index 00000000000000..f3cbeae9c88ab8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java @@ -0,0 +1,125 @@ +// 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.trees.plans.physical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.Statistics; + +import com.google.common.base.Preconditions; +import org.jetbrains.annotations.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * result sink + */ +public class PhysicalResultSink extends PhysicalSink implements Sink { + + private final List outputExprs; + + public PhysicalResultSink(List outputExprs, LogicalProperties logicalProperties, + CHILD_TYPE child) { + super(PlanType.PHYSICAL_RESULT_SINK, logicalProperties, child); + this.outputExprs = outputExprs; + } + + public PhysicalResultSink(List outputExprs, Optional groupExpression, + LogicalProperties logicalProperties, CHILD_TYPE child) { + super(PlanType.PHYSICAL_RESULT_SINK, groupExpression, logicalProperties, child); + this.outputExprs = outputExprs; + } + + public PhysicalResultSink(List outputExprs, Optional groupExpression, + LogicalProperties logicalProperties, @Nullable PhysicalProperties physicalProperties, + Statistics statistics, CHILD_TYPE child) { + super(PlanType.PHYSICAL_RESULT_SINK, groupExpression, logicalProperties, physicalProperties, statistics, child); + this.outputExprs = outputExprs; + } + + @Override + public PhysicalResultSink withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "PhysicalResultSink's children size must be 1, but real is %s", children.size()); + return new PhysicalResultSink<>(outputExprs, groupExpression, getLogicalProperties(), children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalResultSink(this, context); + } + + @Override + public List getExpressions() { + return outputExprs; + } + + @Override + public PhysicalResultSink withGroupExpression(Optional groupExpression) { + return new PhysicalResultSink<>(outputExprs, groupExpression, getLogicalProperties(), child()); + } + + @Override + public PhysicalResultSink withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + return new PhysicalResultSink<>(outputExprs, groupExpression, logicalProperties.get(), child()); + } + + @Override + public PhysicalResultSink withPhysicalPropertiesAndStats( + PhysicalProperties physicalProperties, Statistics statistics) { + return new PhysicalResultSink<>(outputExprs, groupExpression, + getLogicalProperties(), physicalProperties, statistics, child()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + PhysicalResultSink that = (PhysicalResultSink) o; + return Objects.equals(outputExprs, that.outputExprs); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), outputExprs); + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalResultSink[" + id.asInt() + "]", + "outputExprs", outputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java index 629deaacf7d62b..63f371aecba4b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java @@ -18,12 +18,15 @@ package org.apache.doris.nereids.trees.plans.visitor; import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; +import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; /** @@ -47,6 +50,10 @@ default R visitUnboundOlapTableSink(UnboundOlapTableSink unbound return visitLogicalSink(unboundOlapTableSink, context); } + default R visitUnboundResultSink(UnboundResultSink unboundResultSink, C context) { + return visitLogicalSink(unboundResultSink, context); + } + // ******************************* // logical // ******************************* @@ -59,6 +66,10 @@ default R visitLogicalOlapTableSink(LogicalOlapTableSink olapTab return visitLogicalSink(olapTableSink, context); } + default R visitLogicalResultSink(LogicalResultSink logicalResultSink, C context) { + return visitLogicalSink(logicalResultSink, context); + } + // ******************************* // physical // ******************************* @@ -70,4 +81,8 @@ default R visitPhysicalFileSink(PhysicalFileSink fileSink, C con default R visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, C context) { return visitPhysicalSink(olapTableSink, context); } + + default R visitPhysicalResultSink(PhysicalResultSink physicalResultSink, C context) { + return visitPhysicalSink(physicalResultSink, context); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java index 631a89b7762402..ac43c5698ad8ed 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java @@ -114,22 +114,23 @@ public void testHintWithReorderCrossJoin() throws Exception { PlanChecker.from(connectContext).checkExplain(sql, planner -> { Plan plan = planner.getOptimizedPlan(); MatchingUtils.assertMatches(plan, - physicalDistribute( - physicalProject( - physicalHashJoin( - physicalHashJoin(physicalDistribute().when(dis -> { - DistributionSpec spec = dis.getDistributionSpec(); - Assertions.assertTrue(spec instanceof DistributionSpecHash); - DistributionSpecHash hashSpec = (DistributionSpecHash) spec; - Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, - hashSpec.getShuffleType()); - return true; - }), physicalDistribute()), - physicalDistribute() - ).when(join -> join.getHint() == JoinHint.SHUFFLE_RIGHT) + physicalResultSink( + physicalDistribute( + physicalProject( + physicalHashJoin( + physicalHashJoin(physicalDistribute().when(dis -> { + DistributionSpec spec = dis.getDistributionSpec(); + Assertions.assertTrue(spec instanceof DistributionSpecHash); + DistributionSpecHash hashSpec = (DistributionSpecHash) spec; + Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, + hashSpec.getShuffleType()); + return true; + }), physicalDistribute()), + physicalDistribute() + ).when(join -> join.getHint() == JoinHint.SHUFFLE_RIGHT) + ) ) - ) - ); + )); }); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java index 75636ef5982de9..38bbb4c45ebaa6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java @@ -23,24 +23,24 @@ public class LimitClauseTest extends ParserTestBase { @Test public void testLimit() { parsePlan("SELECT b FROM test order by a limit 3 offset 100") - .matchesFromRoot( + .matches( logicalLimit( logicalSort() ).when(limit -> limit.getLimit() == 3 && limit.getOffset() == 100) ); parsePlan("SELECT b FROM test order by a limit 100, 3") - .matchesFromRoot( + .matches( logicalLimit( logicalSort() ).when(limit -> limit.getLimit() == 3 && limit.getOffset() == 100) ); parsePlan("SELECT b FROM test limit 3") - .matchesFromRoot(logicalLimit().when(limit -> limit.getLimit() == 3 && limit.getOffset() == 0)); + .matches(logicalLimit().when(limit -> limit.getLimit() == 3 && limit.getOffset() == 0)); parsePlan("SELECT b FROM test order by a limit 3") - .matchesFromRoot( + .matches( logicalLimit( logicalSort() ).when(limit -> limit.getLimit() == 3 && limit.getOffset() == 0) @@ -49,13 +49,13 @@ public void testLimit() { @Test public void testNoLimit() { - parsePlan("select a from tbl order by x").matchesFromRoot(logicalSort()); + parsePlan("select a from tbl order by x").matches(logicalSort()); } @Test public void testNoQueryOrganization() { parsePlan("select a from tbl") - .matchesFromRoot( + .matches( logicalProject( logicalCheckPolicy( unboundRelation() diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index 63da2752fb118a..fd4d1f2e1287f1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -80,7 +80,7 @@ public void testErrorListener() { @Test public void testPostProcessor() { parsePlan("select `AD``D` from t1 where a = 1") - .matchesFromRoot( + .matches( logicalProject().when(p -> "AD`D".equals(p.getProjects().get(0).getName())) ); } @@ -90,17 +90,17 @@ public void testParseCTE() { NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan; String cteSql1 = "with t1 as (select s_suppkey from supplier where s_suppkey < 10) select * from t1"; - logicalPlan = nereidsParser.parseSingle(cteSql1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql1).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 1); String cteSql2 = "with t1 as (select s_suppkey from supplier), t2 as (select s_suppkey from t1) select * from t2"; - logicalPlan = nereidsParser.parseSingle(cteSql2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql2).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 2); String cteSql3 = "with t1 (key, name) as (select s_suppkey, s_name from supplier) select * from t1"; - logicalPlan = nereidsParser.parseSingle(cteSql3); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql3).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 1); Optional> columnAliases = ((LogicalCTE) logicalPlan).getAliasQueries().get(0).getColumnAliases(); @@ -112,12 +112,12 @@ public void testParseWindowFunctions() { NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan; String windowSql1 = "select k1, rank() over(partition by k1 order by k1) as ranking from t1"; - logicalPlan = nereidsParser.parseSingle(windowSql1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(windowSql1).child(0); Assertions.assertEquals(PlanType.LOGICAL_PROJECT, logicalPlan.getType()); Assertions.assertEquals(((LogicalProject) logicalPlan).getProjects().size(), 2); String windowSql2 = "select k1, sum(k2), rank() over(partition by k1 order by k1) as ranking from t1 group by k1"; - logicalPlan = nereidsParser.parseSingle(windowSql2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(windowSql2).child(0); Assertions.assertEquals(PlanType.LOGICAL_AGGREGATE, logicalPlan.getType()); Assertions.assertEquals(((LogicalAggregate) logicalPlan).getOutputExpressions().size(), 3); @@ -135,7 +135,7 @@ public void testExplainNormal() { ExplainCommand explainCommand = (ExplainCommand) logicalPlan; ExplainLevel explainLevel = explainCommand.getLevel(); Assertions.assertEquals(ExplainLevel.NORMAL, explainLevel); - logicalPlan = explainCommand.getLogicalPlan(); + logicalPlan = (LogicalPlan) explainCommand.getLogicalPlan().child(0); LogicalProject logicalProject = (LogicalProject) logicalPlan; Assertions.assertEquals("AD`D", logicalProject.getProjects().get(0).getName()); } @@ -168,7 +168,7 @@ public void testParseSQL() { Assertions.assertEquals(2, statementBases.size()); Assertions.assertTrue(statementBases.get(0) instanceof LogicalPlanAdapter); Assertions.assertTrue(statementBases.get(1) instanceof LogicalPlanAdapter); - LogicalPlan logicalPlan0 = ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan(); + LogicalPlan logicalPlan0 = (LogicalPlan) ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan().child(0); LogicalPlan logicalPlan1 = ((LogicalPlanAdapter) statementBases.get(1)).getLogicalPlan(); Assertions.assertTrue(logicalPlan0 instanceof LogicalProject); Assertions.assertTrue(logicalPlan1 instanceof ExplainCommand); @@ -181,57 +181,57 @@ public void testParseJoin() { LogicalJoin logicalJoin; String innerJoin1 = "SELECT t1.a FROM t1 INNER JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(innerJoin1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(innerJoin1).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.INNER_JOIN, logicalJoin.getJoinType()); String innerJoin2 = "SELECT t1.a FROM t1 JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(innerJoin2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(innerJoin2).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.INNER_JOIN, logicalJoin.getJoinType()); String leftJoin1 = "SELECT t1.a FROM t1 LEFT JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftJoin1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftJoin1).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, logicalJoin.getJoinType()); String leftJoin2 = "SELECT t1.a FROM t1 LEFT OUTER JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftJoin2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftJoin2).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, logicalJoin.getJoinType()); String rightJoin1 = "SELECT t1.a FROM t1 RIGHT JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(rightJoin1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(rightJoin1).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_OUTER_JOIN, logicalJoin.getJoinType()); String rightJoin2 = "SELECT t1.a FROM t1 RIGHT OUTER JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(rightJoin2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(rightJoin2).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_OUTER_JOIN, logicalJoin.getJoinType()); String leftSemiJoin = "SELECT t1.a FROM t1 LEFT SEMI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftSemiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftSemiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_SEMI_JOIN, logicalJoin.getJoinType()); String rightSemiJoin = "SELECT t2.a FROM t1 RIGHT SEMI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(rightSemiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(rightSemiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_SEMI_JOIN, logicalJoin.getJoinType()); String leftAntiJoin = "SELECT t1.a FROM t1 LEFT ANTI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftAntiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftAntiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_ANTI_JOIN, logicalJoin.getJoinType()); String righAntiJoin = "SELECT t2.a FROM t1 RIGHT ANTI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(righAntiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(righAntiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_ANTI_JOIN, logicalJoin.getJoinType()); String crossJoin = "SELECT t1.a FROM t1 CROSS JOIN t2;"; - logicalPlan = nereidsParser.parseSingle(crossJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(crossJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.CROSS_JOIN, logicalJoin.getJoinType()); } @@ -252,7 +252,7 @@ void parseJoinEmptyConditionError() { public void testParseDecimal() { String f1 = "SELECT col1 * 0.267081789095306 FROM t"; NereidsParser nereidsParser = new NereidsParser(); - LogicalPlan logicalPlan = nereidsParser.parseSingle(f1); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(f1).child(0); long doubleCount = logicalPlan .getExpressions() .stream() @@ -334,7 +334,7 @@ public void testJoinHint() { public void testParseCast() { String sql = "SELECT CAST(1 AS DECIMAL(20, 6)) FROM t"; NereidsParser nereidsParser = new NereidsParser(); - LogicalPlan logicalPlan = nereidsParser.parseSingle(sql); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(sql).child(0); Cast cast = (Cast) logicalPlan.getExpressions().get(0).child(0); if (Config.enable_decimal_conversion) { DecimalV3Type decimalV3Type = (DecimalV3Type) cast.getDataType(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/TopNRuntimeFilterTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/TopNRuntimeFilterTest.java index 6cbbcbf07110c9..a3449909deb9c2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/TopNRuntimeFilterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/TopNRuntimeFilterTest.java @@ -40,8 +40,8 @@ public void testUseTopNRf() { .implement(); PhysicalPlan plan = checker.getPhysicalPlan(); new PlanPostProcessors(checker.getCascadesContext()).process(plan); - Assertions.assertTrue(plan.children().get(0) instanceof PhysicalTopN); - PhysicalTopN localTopN = (PhysicalTopN) plan.children().get(0); + Assertions.assertTrue(plan.children().get(0).child(0) instanceof PhysicalTopN); + PhysicalTopN localTopN = (PhysicalTopN) plan.children().get(0).child(0); Assertions.assertTrue(localTopN.getMutableState(PhysicalTopN.TOPN_RUNTIME_FILTER).isPresent()); } 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 7e95409118c256..522f198e3ff774 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 @@ -153,7 +153,7 @@ public void testCTEInHavingAndSubquery() { public void testCTEWithAlias() { PlanChecker.from(connectContext) .analyze(cteConsumerJoin) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer(), logicalCTEAnchor( @@ -173,7 +173,7 @@ public void testCTEWithAlias() { public void testCTEWithAnExistedTableOrViewName() { PlanChecker.from(connectContext) .analyze(cteReferToAnotherOne) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer(), logicalCTEAnchor( @@ -191,7 +191,7 @@ public void testCTEWithAnExistedTableOrViewName() { public void testDifferenceRelationId() { PlanChecker.from(connectContext) .analyze(cteWithDiffRelationId) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer(), logicalProject( @@ -212,7 +212,7 @@ public void testDifferenceRelationId() { public void testCteInTheMiddle() { PlanChecker.from(connectContext) .analyze(cteInTheMiddle) - .matchesFromRoot( + .matches( logicalProject( logicalSubQueryAlias( logicalCTEAnchor( @@ -231,7 +231,7 @@ public void testCteInTheMiddle() { public void testCteNested() { PlanChecker.from(connectContext) .analyze(cteNested) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer( logicalSubQueryAlias( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java index e73713bc7dfb08..26496a4dfeaea3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java @@ -105,7 +105,7 @@ public void testCaseSubQuery() { PlanChecker.from(connectContext) .analyze(testSql.get(0)) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) - .matchesFromRoot( + .matches( logicalProject( logicalProject( logicalProject( @@ -129,7 +129,7 @@ public void testCaseMixed() { PlanChecker.from(connectContext) .analyze(testSql.get(1)) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalProject( @@ -165,7 +165,7 @@ public void testCaseJoinSameTable() { PlanChecker.from(connectContext) .analyze(testSql.get(5)) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalOlapScan(), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java index c05ddc1bd065a2..b3387012f2ec0c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java @@ -374,37 +374,39 @@ public void testSql10AfterAnalyze() { PlanChecker.from(connectContext) .analyze(sql10) .matchesFromRoot( - logicalProject( - logicalFilter( - logicalProject( - logicalApply( - any(), - logicalAggregate( - logicalSubQueryAlias( - logicalProject( - logicalFilter() - ).when(p -> p.getProjects().equals(ImmutableList.of( - new Alias(new ExprId(7), new SlotReference(new ExprId(5), "v1", BigIntType.INSTANCE, - true, - ImmutableList.of("default_cluster:test", "t7")), "aa") + logicalResultSink( + logicalProject( + logicalFilter( + logicalProject( + logicalApply( + any(), + logicalAggregate( + logicalSubQueryAlias( + logicalProject( + logicalFilter() + ).when(p -> p.getProjects().equals(ImmutableList.of( + new Alias(new ExprId(7), new SlotReference(new ExprId(5), "v1", BigIntType.INSTANCE, + true, + ImmutableList.of("default_cluster:test", "t7")), "aa") + ))) + ) + .when(a -> a.getAlias().equals("t2")) + .when(a -> a.getOutput().equals(ImmutableList.of( + new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, + true, ImmutableList.of("t2")) ))) - ) - .when(a -> a.getAlias().equals("t2")) - .when(a -> a.getOutput().equals(ImmutableList.of( - new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, - true, ImmutableList.of("t2")) + ).when(agg -> agg.getOutputExpressions().equals(ImmutableList.of( + new Alias(new ExprId(8), + (new Max(new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, + true, + ImmutableList.of("t2")))).withAlwaysNullable(true), "max(aa)") ))) - ).when(agg -> agg.getOutputExpressions().equals(ImmutableList.of( - new Alias(new ExprId(8), - (new Max(new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, - true, - ImmutableList.of("t2")))).withAlwaysNullable(true), "max(aa)") - ))) - .when(agg -> agg.getGroupByExpressions().equals(ImmutableList.of())) + .when(agg -> agg.getGroupByExpressions().equals(ImmutableList.of())) + ) + .when(apply -> apply.getCorrelationSlot().equals(ImmutableList.of( + new SlotReference(new ExprId(1), "k2", BigIntType.INSTANCE, true, + ImmutableList.of("default_cluster:test", "t6"))))) ) - .when(apply -> apply.getCorrelationSlot().equals(ImmutableList.of( - new SlotReference(new ExprId(1), "k2", BigIntType.INSTANCE, true, - ImmutableList.of("default_cluster:test", "t6"))))) ) ) ) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java index 3a99e91dd84e29..af012ce3fb9f95 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java @@ -115,7 +115,7 @@ public boolean hasDeleteSign() { .parse("select * from " + tableName + " as et join db1.t on et.id = t.a") .customAnalyzer(Optional.of(customTableResolver)) // analyze internal relation .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan().when(r -> r.getTable() == externalOlapTable), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java index 72fe56dcc8f86c..84e7cf4ff72c3b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java @@ -56,11 +56,11 @@ public void testCountDistinctBitmap() { ConnectContext connectContext = MemoTestUtils.createConnectContext(); PlanChecker.from(connectContext) .analyze("select count(distinct id) from (select to_bitmap(1) id) tbl") - .matchesFromRoot(logicalAggregate().when(agg -> + .matches(logicalAggregate().when(agg -> agg.getOutputExpressions().get(0).child(0) instanceof Count )) .rewrite() - .matchesFromRoot(logicalAggregate().when(agg -> + .matches(logicalAggregate().when(agg -> agg.getOutputExpressions().get(0).child(0) instanceof BitmapUnionCount )); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java index 7cbafd8d5f4994..03cc549bc2c1fa 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java @@ -85,7 +85,7 @@ public void testHavingGroupBySlot() { ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") ); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalFilter( logicalAggregate( logicalOlapScan() @@ -99,7 +99,7 @@ public void testHavingGroupBySlot() { Alias value = new Alias(new ExprId(3), a1, "value"); PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -110,7 +110,7 @@ public void testHavingGroupBySlot() { sql = "SELECT a1 as value FROM t1 GROUP BY a1 HAVING value > 0"; PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) - .matchesFromRoot( + .matches( logicalFilter( logicalAggregate( logicalOlapScan() @@ -129,7 +129,7 @@ public void testHavingGroupBySlot() { Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -152,7 +152,7 @@ public void testHavingAggregateFunction() { ); Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -164,7 +164,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 HAVING SUM(a2) > 0"; sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -183,7 +183,7 @@ public void testHavingAggregateFunction() { ); Alias value = new Alias(new ExprId(3), new Sum(a2), "value"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -193,7 +193,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 HAVING value > 0"; PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalFilter( logicalAggregate( logicalOlapScan() @@ -216,7 +216,7 @@ public void testHavingAggregateFunction() { ); Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -228,7 +228,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 HAVING SUM(a1 + a2) > 0"; Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -240,7 +240,7 @@ public void testHavingAggregateFunction() { Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new TinyIntLiteral((byte) 3))), "sum(((a1 + a2) + 3))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -252,7 +252,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1 FROM t1 GROUP BY a1 HAVING COUNT(*) > 0"; Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -280,7 +280,7 @@ void testJoinWithHaving() { Alias sumA2 = new Alias(new ExprId(6), new Sum(a2), "sum(a2)"); Alias sumB1 = new Alias(new ExprId(7), new Sum(b1), "sum(b1)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -347,7 +347,7 @@ void testComplexQueryWithHaving() { Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -388,7 +388,7 @@ public void testSortAggregateFunction() { ); Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -400,7 +400,7 @@ public void testSortAggregateFunction() { sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 ORDER BY SUM(a2)"; sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalSort( logicalAggregate( logicalOlapScan() @@ -418,7 +418,7 @@ public void testSortAggregateFunction() { ); Alias value = new Alias(new ExprId(3), new Sum(a2), "value"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalSort( logicalAggregate( logicalOlapScan() @@ -441,7 +441,7 @@ public void testSortAggregateFunction() { ); Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -453,7 +453,7 @@ public void testSortAggregateFunction() { sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 ORDER BY SUM(a1 + a2)"; Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalSort( logicalAggregate( logicalOlapScan() @@ -464,7 +464,7 @@ public void testSortAggregateFunction() { Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new TinyIntLiteral((byte) 3))), "sum(((a1 + a2) + 3))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -476,7 +476,7 @@ public void testSortAggregateFunction() { sql = "SELECT a1 FROM t1 GROUP BY a1 ORDER BY COUNT(*)"; Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -511,7 +511,7 @@ void testComplexQueryWithOrderBy() { Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java index c7e32bed8194c2..9758a6c10b0d97 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java @@ -54,7 +54,7 @@ public void testDefaultFunctionNameIsClassName() { // and default class name should be year. PlanChecker.from(connectContext) .analyze("select year('2021-01-01')") - .matchesFromRoot( + .matches( logicalOneRowRelation().when(r -> { Year year = (Year) r.getProjects().get(0).child(0); Assertions.assertEquals("2021-01-01", @@ -71,7 +71,7 @@ public void testMultiName() { // 2. substr PlanChecker.from(connectContext) .analyze("select substring('abc', 1, 2), substr(substring('abcdefg', 4, 3), 1, 2)") - .matchesFromRoot( + .matches( logicalOneRowRelation().when(r -> { Substring firstSubstring = (Substring) r.getProjects().get(0).child(0); Assertions.assertEquals("abc", ((Literal) firstSubstring.getSource()).getValue()); @@ -94,7 +94,7 @@ public void testOverrideArity() { // 2. substring(string, position, length) PlanChecker.from(connectContext) .analyze("select substr('abc', 1), substring('def', 2, 3)") - .matchesFromRoot( + .matches( logicalOneRowRelation().when(r -> { Substring firstSubstring = (Substring) r.getProjects().get(0).child(0); Assertions.assertEquals("abc", ((Literal) firstSubstring.getSource()).getValue()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java index cad39fe234a54c..04e84ab8e89d86 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java @@ -62,7 +62,7 @@ public void testPruneColumns1() { .analyze("select id,name,grade from student left join score on student.id = score.sid" + " where score.grade > 60") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject( @@ -94,7 +94,7 @@ public void testPruneColumns2() { + "from student left join score on student.id = score.sid " + "where score.grade > 60") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject( @@ -124,7 +124,7 @@ public void testPruneColumns3() { PlanChecker.from(connectContext) .analyze("select id,name from student where age > 18") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject().when(p -> getOutputQualifiedNames(p) @@ -146,7 +146,7 @@ public void testPruneColumns4() { + "on score.cid = course.cid " + "where score.grade > 60") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject( @@ -184,7 +184,7 @@ public void pruneCountStarStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -199,7 +199,7 @@ public void pruneCountConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(1) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -214,7 +214,7 @@ public void pruneCountConstantAndSumConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(1), SUM(2) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -229,7 +229,7 @@ public void pruneCountStarAndSumConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*), SUM(2) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -244,7 +244,7 @@ public void pruneCountStarAndSumColumnStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*), SUM(grade) FROM test.score") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -259,7 +259,7 @@ public void pruneCountStarAndSumColumnAndSumConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*), SUM(grade) + SUM(2) FROM test.score") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -274,7 +274,7 @@ public void pruneColumnForOneSideOnCrossJoin() { PlanChecker.from(connectContext) .analyze("select id,name from student cross join score") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject(logicalRelation()) @@ -296,7 +296,7 @@ public void pruneAggregateOutput() { PlanChecker.from(connectContext) .analyze("select id from (select id, sum(age) from student group by id)a") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalSubQueryAlias( logicalAggregate( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java index 1203f542f4f121..04613f7e75e7f0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java @@ -81,7 +81,7 @@ public void inferPredicatesTest01() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -102,7 +102,7 @@ public void inferPredicatesTest02() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -119,7 +119,7 @@ public void inferPredicatesTest03() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -138,7 +138,7 @@ public void inferPredicatesTest04() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -157,7 +157,7 @@ public void inferPredicatesTest05() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalJoin( @@ -183,7 +183,7 @@ public void inferPredicatesTest06() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalJoin( @@ -209,7 +209,7 @@ public void inferPredicatesTest07() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -230,7 +230,7 @@ public void inferPredicatesTest08() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -250,7 +250,7 @@ public void inferPredicatesTest09() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -271,7 +271,7 @@ public void inferPredicatesTest10() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -294,7 +294,7 @@ public void inferPredicatesTest11() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -315,7 +315,7 @@ public void inferPredicatesTest12() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -339,7 +339,7 @@ public void inferPredicatesTest13() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -362,7 +362,7 @@ public void inferPredicatesTest14() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -385,7 +385,7 @@ public void inferPredicatesTest15() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -408,7 +408,7 @@ public void inferPredicatesTest16() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -429,7 +429,7 @@ public void inferPredicatesTest17() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -450,7 +450,7 @@ public void inferPredicatesTest18() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -495,7 +495,7 @@ public void inferPredicatesTest19() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -534,7 +534,7 @@ public void inferPredicatesTest20() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( innerLogicalJoin( @@ -560,7 +560,7 @@ public void inferPredicatesTest21() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalJoin( @@ -589,7 +589,7 @@ public void inferPredicatesTest22() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java index 7d9114cef3279b..29cc509d954988 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java @@ -105,7 +105,7 @@ public void testSubQueryCase() { "SELECT * FROM (SELECT * FROM T1) X JOIN (SELECT * FROM T2) Y ON X.ID + 1 = Y.ID + 2 AND X.ID + 1 > 2") .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -134,7 +134,7 @@ public void testAggNodeCase() { "SELECT * FROM T1 JOIN (SELECT ID, SUM(SCORE) SCORE FROM T2 GROUP BY ID) T ON T1.ID + 1 = T.ID AND T.SCORE = T1.SCORE + 10") .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -159,7 +159,7 @@ public void testSortNodeCase() { "SELECT * FROM T1 JOIN (SELECT ID, SUM(SCORE) SCORE FROM T2 GROUP BY ID ORDER BY ID) T ON T1.ID + 1 = T.ID AND T.SCORE = T1.SCORE + 10") .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java index 2342d8f14a7337..a3bd46eb4f2f4e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java @@ -261,6 +261,7 @@ public void testKeyColumnInAggFunction() { })); } + @Disabled("reopen it if we fix rollup select bugs") @Test public void testMaxCanUseKeyColumn() { PlanChecker.from(connectContext) @@ -275,6 +276,7 @@ public void testMaxCanUseKeyColumn() { })); } + @Disabled("reopen it if we fix rollup select bugs") @Test public void testMinCanUseKeyColumn() { PlanChecker.from(connectContext) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java index adc5a993fdaf66..43c8c4b9dc42e7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java @@ -29,7 +29,7 @@ void testInferNotNullAndInferPredicates() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalFilter().when(f -> f.getPredicate().toString().equals("(id#0 = 4)")), @@ -47,7 +47,7 @@ void testInferNotNullFromFilterAndEliminateOuter2() { .analyze(sql) .rewrite() .printlnTree() - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalFilter().when( @@ -65,7 +65,7 @@ void testInferNotNullFromFilterAndEliminateOuter3() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalFilter( leftOuterLogicalJoin( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java index aa5514d304c5b7..72f8ec0879758d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.sqltest; +import org.apache.doris.nereids.properties.DistributionSpecGather; import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.rules.rewrite.ReorderJoin; @@ -49,7 +50,12 @@ void testColocatedJoin() { .getBestPlanTree(); // generate colocate join plan without physicalDistribute System.out.println(plan.treeString()); - Assertions.assertFalse(plan.anyMatch(PhysicalDistribute.class::isInstance)); + Assertions.assertFalse(plan.anyMatch(p -> { + if (p instanceof PhysicalDistribute) { + return !(((PhysicalDistribute) p).getDistributionSpec() instanceof DistributionSpecGather); + } + return false; + })); sql = "select * from T1 join T0 on T1.score = T0.score and T1.id = T0.id;"; plan = PlanChecker.from(connectContext) .analyze(sql) @@ -57,7 +63,12 @@ void testColocatedJoin() { .optimize() .getBestPlanTree(); // generate colocate join plan without physicalDistribute - Assertions.assertFalse(plan.anyMatch(PhysicalDistribute.class::isInstance)); + Assertions.assertFalse(plan.anyMatch(p -> { + if (p instanceof PhysicalDistribute) { + return !(((PhysicalDistribute) p).getDistributionSpec() instanceof DistributionSpecGather); + } + return false; + })); } @Test @@ -91,7 +102,9 @@ void testBucketJoinWithAgg() { .optimize() .getBestPlanTree(); Assertions.assertEquals( - ((DistributionSpecHash) plan.getPhysicalProperties().getDistributionSpec()).getShuffleType(), - ShuffleType.NATURAL); + ShuffleType.NATURAL, + ((DistributionSpecHash) ((PhysicalPlan) (plan.child(0).child(0))) + .getPhysicalProperties().getDistributionSpec()).getShuffleType() + ); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java index 8bb4dc81afaafa..4fe217b0e6bff8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java @@ -115,7 +115,7 @@ public void testSimpleViewMergeProjects() { .analyze("SELECT * FROM V1") .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) .applyTopDown(new MergeProjects()) - .matchesFromRoot( + .matches( logicalProject( logicalOlapScan() ) @@ -142,7 +142,7 @@ public void testNestedView() { ) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) .applyTopDown(new MergeProjects()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out index 0a15a57002d99f..55ff9ac886d73d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out @@ -13,29 +13,30 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year = 2000)) --------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) -------------PhysicalDistribute ---------------PhysicalProject -----------------PhysicalOlapScan[customer] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) -------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) ---------------------PhysicalDistribute -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((cast(s_state as VARCHAR(*)) = 'SD')) ---------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +--------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) +----------------------PhysicalDistribute +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalProject +--------------------------filter((cast(s_state as VARCHAR(*)) = 'SD')) +----------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 1f82006aec7845..2ec618b7da99f8 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,56 +1,57 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(($c$1 OR $c$2)) -------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ---------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) -----------------------------PhysicalOlapScan[customer_demographics] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(($c$1 OR $c$2)) +--------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) +----------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalOlapScan[customer] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter(ca_county IN ('Storey County', 'Marquette County', 'Warren County', 'Cochran County', 'Kandiyohi County')) ---------------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------------filter(ca_county IN ('Storey County', 'Marquette County', 'Warren County', 'Cochran County', 'Kandiyohi County')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) +--------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2001)(date_dim.d_moy <= 4)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2001)(date_dim.d_moy <= 4)) ---------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out index b97d80540ed59b..e773a1bf5fc02b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out @@ -40,27 +40,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter('w' IN ('s', 'w')) ------------------------PhysicalOlapScan[customer] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END) -------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) ---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END) +--------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001)) +--------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002)) +------------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out index 5750cb0006f408..a48c3cceb8f30a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_12 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= 1998-05-06)(date_dim.d_date >= 1998-04-06)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(i_category IN ('Books', 'Sports', 'Men')) -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_date <= 1998-05-06)(date_dim.d_date >= 1998-04-06)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Sports', 'Men')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out index 7e06af0ebffb69..e1ae820d98e6c1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_13 -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------PhysicalProject -------------PhysicalOlapScan[store] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------PhysicalProject -------------------filter((date_dim.d_year = 2001)) ---------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute +--------------PhysicalOlapScan[store] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)((((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1))) -----------------------PhysicalProject -------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')))) ---------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------PhysicalProject -----------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_dep_count = 1))) -------------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('SD', 'KS', 'MI') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('MO', 'ND', 'CO') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))) -------------------------------PhysicalProject ---------------------------------filter(((((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) -----------------------------------PhysicalOlapScan[store_sales] -------------------------------PhysicalDistribute +--------------------filter((date_dim.d_year = 2001)) +----------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)((((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1))) +------------------------PhysicalProject +--------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')))) +----------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------PhysicalProject +------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_dep_count = 1))) +--------------------------------PhysicalOlapScan[household_demographics] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('SD', 'KS', 'MI') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('MO', 'ND', 'CO') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))) --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_country = 'United States')((ca_state IN ('SD', 'KS', 'MI') OR ca_state IN ('MO', 'ND', 'CO')) OR ca_state IN ('NH', 'OH', 'TX'))) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter(((((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +------------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_country = 'United States')((ca_state IN ('SD', 'KS', 'MI') OR ca_state IN ('MO', 'ND', 'CO')) OR ca_state IN ('NH', 'OH', 'TX'))) +--------------------------------------PhysicalOlapScan[customer_address] 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 96dfb9508d3b7c..b271f329269f28 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 @@ -86,91 +86,92 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------filter((date_dim.d_year >= 2000)(date_dim.d_year <= 2002)) --------------------------PhysicalOlapScan[date_dim] -----PhysicalTopN -------PhysicalDistribute ---------PhysicalTopN -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalRepeat ---------------------PhysicalUnion -----------------------PhysicalProject -------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) ---------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----PhysicalResultSink +------PhysicalTopN +--------PhysicalDistribute +----------PhysicalTopN +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalRepeat +----------------------PhysicalUnion +------------------------PhysicalProject +--------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -----------------------PhysicalProject -------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) ---------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +----------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -----------------------PhysicalProject -------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) ---------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalAssertNumRows +--------------------------------PhysicalDistribute +----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------PhysicalProject +--------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------------PhysicalDistribute +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) ---------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +----------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalAssertNumRows +--------------------------------PhysicalDistribute +----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------PhysicalProject +--------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +----------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalAssertNumRows +--------------------------------PhysicalDistribute +----------------------------------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 3276beaec8b26d..abb2fe6156f5ae 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,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR ca_state IN ('CA', 'WA', 'GA')) OR (catalog_sales.cs_sales_price > 500.00)) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[catalog_sales] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR ca_state IN ('CA', 'WA', 'GA')) OR (catalog_sales.cs_sales_price > 500.00)) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] +------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] +--------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out index eefd61c93b4f6a..967e3b60636fcb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out @@ -1,38 +1,39 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](cs1.cs_call_center_sk = call_center.cc_call_center_sk) -----------------PhysicalProject -------------------filter(cc_county IN ('Ziebach County', 'Luce County', 'Richland County', 'Daviess County', 'Barrow County')) ---------------------PhysicalOlapScan[call_center] -----------------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](cs1.cs_call_center_sk = call_center.cc_call_center_sk) ------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](cs1.cs_order_number = cs2.cs_order_number)( not (cs_warehouse_sk = cs_warehouse_sk)) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](cs1.cs_ship_date_sk = date_dim.d_date_sk) +--------------------filter(cc_county IN ('Ziebach County', 'Luce County', 'Richland County', 'Daviess County', 'Barrow County')) +----------------------PhysicalOlapScan[call_center] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[RIGHT_SEMI_JOIN](cs1.cs_order_number = cs2.cs_order_number)( not (cs_warehouse_sk = cs_warehouse_sk)) +------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2002-4-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))(date_dim.d_date >= 2002-04-01)) -------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------hashJoin[RIGHT_ANTI_JOIN](cs1.cs_order_number = cr1.cr_order_number) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](cs1.cs_ship_addr_sk = customer_address.ca_address_sk) +----------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](cs1.cs_ship_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------filter((cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2002-4-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))(date_dim.d_date >= 2002-04-01)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------hashJoin[RIGHT_ANTI_JOIN](cs1.cs_order_number = cr1.cr_order_number) +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](cs1.cs_ship_addr_sk = customer_address.ca_address_sk) ------------------------------------PhysicalProject ---------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'WV')) -----------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'WV')) +------------------------------------------PhysicalOlapScan[customer_address] 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 3ea6acc6ee31d7..ab873f067427af 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,48 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalOlapScan[store_returns] +------------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((cast(d_quarter_name as VARCHAR(*)) = '2001Q1')) -------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((cast(d_quarter_name as VARCHAR(*)) = '2001Q1')) +--------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------PhysicalOlapScan[date_dim] 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 6ddc252783bb48..44c121660a3fe1 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,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_demographics] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_demographics] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) ---------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) -----------------------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +----------------------------------------PhysicalDistribute +------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter(c_birth_month IN (1, 2, 4, 7, 8, 10)) ---------------------------------------------PhysicalOlapScan[customer] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(ca_state IN ('WA', 'GA', 'NC', 'ME', 'WY', 'OK', 'IN')) ---------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) +------------------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 2, 4, 7, 8, 10)) +----------------------------------------------PhysicalOlapScan[customer] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter(ca_state IN ('WA', 'GA', 'NC', 'ME', 'WY', 'OK', 'IN')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out index 5a6299952a9ddf..b3b38025535b7a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out @@ -1,38 +1,39 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_19 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((item.i_manager_id = 2)) +--------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((item.i_manager_id = 2)) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +----------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out index 4d4ce2d58c8081..d05df91baef121 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out @@ -17,29 +17,30 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[date_dim] ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53)) -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq2) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year = 1999)) -------------------------PhysicalOlapScan[date_dim] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq1) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year = 1998)) -------------------------PhysicalOlapScan[date_dim] +--PhysicalResultSink +----PhysicalQuickSort +------PhysicalDistribute +--------PhysicalQuickSort +----------PhysicalProject +------------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53)) +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq2) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((date_dim.d_year = 1999)) +--------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq1) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((date_dim.d_year = 1998)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out index ef38ef4273ab70..8db8714815e2b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_20 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date >= 2002-01-26)(date_dim.d_date <= 2002-02-25)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date >= 2002-01-26)(date_dim.d_date <= 2002-02-25)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(i_category IN ('Shoes', 'Books', 'Women')) -------------------------------PhysicalOlapScan[item] +------------------------------filter(i_category IN ('Shoes', 'Books', 'Women')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out index 32df59522602e3..f950635ad66931 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_21 -- -PhysicalTopN ---PhysicalDistribute -----filter((CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END >= cast((2.000000 / 3.0) as DOUBLE))(CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END <= 1.5)) -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) -----------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) -------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) ---------------------PhysicalOlapScan[inventory] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------filter((CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END >= cast((2.000000 / 3.0) as DOUBLE))(CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END <= 1.5)) +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) +------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) +----------------------PhysicalOlapScan[inventory] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] +------------------------filter((date_dim.d_date >= 2002-01-28)(date_dim.d_date <= 2002-03-29)) +--------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((date_dim.d_date >= 2002-01-28)(date_dim.d_date <= 2002-03-29)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out index a5d6f602e671f8..ec560939d191a6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_22 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[inventory] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[inventory] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((date_dim.d_month_seq <= 1199)(date_dim.d_month_seq >= 1188)) ---------------------------PhysicalOlapScan[date_dim] +--------------------------filter((date_dim.d_month_seq <= 1199)(date_dim.d_month_seq >= 1188)) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out index 1deef474c9d053..9f1ec957297b6d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out @@ -55,48 +55,49 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[customer] -----PhysicalLimit +----PhysicalResultSink ------PhysicalLimit ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] -------------------------------PhysicalDistribute +--------PhysicalLimit +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) +--------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] -------------------------------PhysicalDistribute +--------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) +--------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) 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 eebe6647279d9d..a9e8df2aaef283 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 @@ -31,25 +31,26 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[item] ----------------PhysicalProject ------------------PhysicalOlapScan[store_returns] ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------NestedLoopJoin[INNER_JOIN](cast(paid as DOUBLE) > cast((0.05 * avg(netpaid)) as DOUBLE)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((cast(i_color as VARCHAR(*)) = 'beige')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalAssertNumRows -----------------PhysicalProject -------------------hashAgg[GLOBAL] +--PhysicalResultSink +----PhysicalQuickSort +------PhysicalDistribute +--------PhysicalQuickSort +----------PhysicalProject +------------NestedLoopJoin[INNER_JOIN](cast(paid as DOUBLE) > cast((0.05 * avg(netpaid)) as DOUBLE)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] --------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------filter((cast(i_color as VARCHAR(*)) = 'beige')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalAssertNumRows +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out index 52b9f72ff15868..35343e9a8c9146 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_25 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[catalog_sales] ---------------------PhysicalDistribute -----------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_returns] ---------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------PhysicalDistribute +------------------------------------------PhysicalOlapScan[store_returns] +----------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((d1.d_year = 2000)(d1.d_moy = 4)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((d2.d_moy <= 10)(d2.d_moy >= 4)(d2.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((d3.d_year = 2000)(d3.d_moy <= 10)(d3.d_moy >= 4)) -----------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((d1.d_year = 2000)(d1.d_moy = 4)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((d2.d_moy <= 10)(d2.d_moy >= 4)(d2.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((d3.d_year = 2000)(d3.d_moy <= 10)(d3.d_moy >= 4)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out index 9a85f030a23990..cdffd94d108e07 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_26 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((cast(cd_education_status as VARCHAR(*)) = 'Unknown')(cast(cd_gender as VARCHAR(*)) = 'M')(cast(cd_marital_status as VARCHAR(*)) = 'S')) +----------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((cast(cd_education_status as VARCHAR(*)) = 'Unknown')(cast(cd_gender as VARCHAR(*)) = 'M')(cast(cd_marital_status as VARCHAR(*)) = 'S')) ---------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) ---------------------------PhysicalOlapScan[promotion] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) +----------------------------PhysicalOlapScan[promotion] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[item] 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 e773eae3049644..81fad1aff1d21e 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,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalRepeat -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalRepeat +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Secondary')(cast(cd_marital_status as VARCHAR(*)) = 'D')) +----------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Secondary')(cast(cd_marital_status as VARCHAR(*)) = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 1999)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter(s_state IN ('MO', 'AL', 'MI', 'TN', 'LA', 'SC')) -----------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('MO', 'AL', 'MI', 'TN', 'LA', 'SC')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out index 9b4743ee67ecde..cf9431b2e02bf7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_28 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----PhysicalProject -------NestedLoopJoin[CROSS_JOIN] ---------PhysicalLimit -----------NestedLoopJoin[CROSS_JOIN] -------------PhysicalLimit ---------------NestedLoopJoin[CROSS_JOIN] -----------------PhysicalLimit -------------------NestedLoopJoin[CROSS_JOIN] ---------------------PhysicalLimit -----------------------NestedLoopJoin[CROSS_JOIN] -------------------------PhysicalLimit ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 5)((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00)))(store_sales.ss_quantity >= 0)) -------------------------------------PhysicalOlapScan[store_sales] -------------------------PhysicalDistribute +----PhysicalLimit +------PhysicalProject +--------NestedLoopJoin[CROSS_JOIN] +----------PhysicalLimit +------------NestedLoopJoin[CROSS_JOIN] +--------------PhysicalLimit +----------------NestedLoopJoin[CROSS_JOIN] +------------------PhysicalLimit +--------------------NestedLoopJoin[CROSS_JOIN] +----------------------PhysicalLimit +------------------------NestedLoopJoin[CROSS_JOIN] --------------------------PhysicalLimit ----------------------------hashAgg[GLOBAL] ------------------------------PhysicalDistribute --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------filter((store_sales.ss_quantity <= 10)((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00)))(store_sales.ss_quantity >= 6)) +------------------------------------filter((store_sales.ss_quantity <= 5)((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00)))(store_sales.ss_quantity >= 0)) --------------------------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute -----------------------PhysicalLimit -------------------------hashAgg[GLOBAL] --------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((((store_sales.ss_list_price >= 1.5E+2) AND (store_sales.ss_list_price <= 1.6E+2)) OR ((store_sales.ss_coupon_amt >= 6.6E+3) AND (store_sales.ss_coupon_amt <= 7.6E+3))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00)))(store_sales.ss_quantity >= 11)(store_sales.ss_quantity <= 15)) -----------------------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute -------------------PhysicalLimit ---------------------hashAgg[GLOBAL] +----------------------------PhysicalLimit +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------filter((store_sales.ss_quantity <= 10)((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00)))(store_sales.ss_quantity >= 6)) +----------------------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((store_sales.ss_quantity <= 20)((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00)))(store_sales.ss_quantity >= 16)) -------------------------------PhysicalOlapScan[store_sales] -------------PhysicalDistribute ---------------PhysicalLimit -----------------hashAgg[GLOBAL] +------------------------PhysicalLimit +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter(((((store_sales.ss_list_price >= 1.5E+2) AND (store_sales.ss_list_price <= 1.6E+2)) OR ((store_sales.ss_coupon_amt >= 6.6E+3) AND (store_sales.ss_coupon_amt <= 7.6E+3))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00)))(store_sales.ss_quantity >= 11)(store_sales.ss_quantity <= 15)) +------------------------------------PhysicalOlapScan[store_sales] ------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00)))(store_sales.ss_quantity <= 25)(store_sales.ss_quantity >= 21)) ---------------------------PhysicalOlapScan[store_sales] ---------PhysicalDistribute -----------PhysicalLimit -------------hashAgg[GLOBAL] +--------------------PhysicalLimit +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((store_sales.ss_quantity <= 20)((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00)))(store_sales.ss_quantity >= 16)) +--------------------------------PhysicalOlapScan[store_sales] --------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------filter((store_sales.ss_quantity >= 26)((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00)))(store_sales.ss_quantity <= 30)) -----------------------PhysicalOlapScan[store_sales] +----------------PhysicalLimit +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00)))(store_sales.ss_quantity <= 25)(store_sales.ss_quantity >= 21)) +----------------------------PhysicalOlapScan[store_sales] +----------PhysicalDistribute +------------PhysicalLimit +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------filter((store_sales.ss_quantity >= 26)((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00)))(store_sales.ss_quantity <= 30)) +------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out index b23eeda8fad031..5f93ae5559fe90 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out @@ -1,46 +1,47 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_29 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) ---------------------------PhysicalProject -----------------------------filter(d_year IN (1999, 2000, 2001)) -------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute +------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalDistribute +------------------------------filter(d_year IN (1999, 2000, 2001)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] ---------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((d2.d_moy <= 7)(d2.d_moy >= 4)(d2.d_year = 1999)) ---------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((d1.d_year = 1999)(d1.d_moy = 4)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------PhysicalOlapScan[store_returns] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((d2.d_moy <= 7)(d2.d_moy >= 4)(d2.d_year = 1999)) +----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((d1.d_year = 1999)(d1.d_moy = 4)) +----------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out index c11369d38f4287..6516eca4168b97 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_3 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manufact_id = 816)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manufact_id = 816)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((dt.d_moy = 11)) -------------------------PhysicalOlapScan[date_dim] +------------------------filter((dt.d_moy = 11)) +--------------------------PhysicalOlapScan[date_dim] 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 7741e0c45cd144..48239bce9e9f72 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 @@ -19,28 +19,29 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) -------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) ---------------PhysicalDistribute -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +--------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +----------------PhysicalDistribute +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------PhysicalProject -----------------------filter((cast(ca_state as VARCHAR(*)) = 'IN')) -------------------------PhysicalOlapScan[customer_address] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((cast(ca_state as VARCHAR(*)) = 'IN')) +--------------------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out index dd60ee2d41baf1..ed3787c35062ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out @@ -39,38 +39,39 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[customer_address] -----PhysicalQuickSort -------PhysicalDistribute ---------PhysicalQuickSort -----------PhysicalProject -------------hashJoin[INNER_JOIN](ws1.ca_county = ws3.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) ---------------PhysicalDistribute +----PhysicalResultSink +------PhysicalQuickSort +--------PhysicalDistribute +----------PhysicalQuickSort +------------PhysicalProject +--------------hashJoin[INNER_JOIN](ws1.ca_county = ws3.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3)) +----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------PhysicalProject -------------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3)) ---------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) ---------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county) -----------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) +----------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county) +------------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1)) +--------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000)) ----------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out index 1338d8dc1b1677..2f2c494bb25224 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +----PhysicalLimit +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date >= 1999-01-07)(date_dim.d_date <= 1999-04-07)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date >= 1999-01-07)(date_dim.d_date <= 1999-04-07)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out index e9a32797acea89..cea4ec9ccc22be 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out @@ -1,89 +1,57 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_33 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ---------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +----------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +----------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ---------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Home')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +--------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject --------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) @@ -92,8 +60,41 @@ PhysicalTopN ------------------------------------PhysicalProject --------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ----------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((item.i_category = 'Home')) +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------PhysicalProject --------------------------------PhysicalOlapScan[item] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +------------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_category = 'Home')) +----------------------------------PhysicalOlapScan[item] 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 952cc6ee639d49..6fd519ecc69bae 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,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) -----------PhysicalProject -------------PhysicalOlapScan[customer] -----------PhysicalDistribute -------------filter((dn.cnt <= 20)(dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) +------------PhysicalProject +--------------PhysicalOlapScan[customer] +------------PhysicalDistribute +--------------filter((dn.cnt <= 20)(dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28)))d_year IN (1998, 1999, 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28)))d_year IN (1998, 1999, 2000)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '1001-5000') OR (cast(hd_buy_potential as VARCHAR(*)) = '0-500'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.2)) +----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '1001-5000') OR (cast(hd_buy_potential as VARCHAR(*)) = '0-500'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.2)) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(s_county IN ('Ziebach County', 'Daviess County', 'Walker County', 'Richland County', 'Barrow County', 'Franklin Parish', 'Williamson County', 'Luce County')) -------------------------------PhysicalOlapScan[store] +------------------------------filter(s_county IN ('Ziebach County', 'Daviess County', 'Walker County', 'Richland County', 'Barrow County', 'Franklin Parish', 'Williamson County', 'Luce 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 93dac990e56444..1a1d022d75b7b9 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,58 +1,59 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(($c$1 OR $c$2)) -------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ---------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) -----------------------------------PhysicalDistribute -------------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(($c$1 OR $c$2)) +--------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) +----------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute +------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out index c5a8b73dc1a685..eed6724c0d7879 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_36 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) -----------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((d1.d_year = 2002)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((d1.d_year = 2002)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter(s_state IN ('SD', 'TN', 'GA', 'SC', 'MO', 'AL', 'MI', 'OH')) -------------------------------------PhysicalOlapScan[store] +----------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(s_state IN ('SD', 'TN', 'GA', 'SC', 'MO', 'AL', 'MI', 'OH')) +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out index a2301a7eec0c38..f9c9f7ec3ea664 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_37 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[catalog_sales] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) -----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) -------------------------PhysicalProject ---------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) -----------------------------PhysicalOlapScan[inventory] +--------------------PhysicalOlapScan[catalog_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) +------------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) +------------------------------PhysicalOlapScan[inventory] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((item.i_current_price <= 75.00)i_manufact_id IN (707, 747, 856, 1000)(item.i_current_price >= 45.00)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((item.i_current_price <= 75.00)i_manufact_id IN (707, 747, 856, 1000)(item.i_current_price >= 45.00)) -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_date >= 1999-02-21)(date_dim.d_date <= 1999-04-22)) -----------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_date >= 1999-02-21)(date_dim.d_date <= 1999-04-22)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out index 02fcd034cecb7e..3b8e5596cc2be6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out @@ -1,64 +1,65 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute +----PhysicalLimit +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] ---------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out index 977a1e576903ab..0fe8176d6f271f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out @@ -22,17 +22,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashJoin[INNER_JOIN](inv1.w_warehouse_sk = inv2.w_warehouse_sk)(inv1.i_item_sk = inv2.i_item_sk) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((inv1.d_moy = 1)) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((inv2.d_moy = 2)) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalQuickSort +------PhysicalDistribute +--------PhysicalQuickSort +----------PhysicalProject +------------hashJoin[INNER_JOIN](inv1.w_warehouse_sk = inv2.w_warehouse_sk)(inv1.i_item_sk = inv2.i_item_sk) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((inv1.d_moy = 1)) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((inv2.d_moy = 2)) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index 1ea2eae2bcf004..540045f8f467fb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out @@ -60,40 +60,41 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter('w' IN ('c', 's', 'w')d_year IN (2000, 1999)) ------------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) -------------PhysicalProject ---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) -------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +--------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) +------------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2000)) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2000)) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 5b6553bb2404fa..6c1818fbfdf8e4 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 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) -----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) ---------------------PhysicalProject -----------------------PhysicalOlapScan[catalog_returns] ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_returns] +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((item.i_current_price >= 0.99)(item.i_current_price <= 1.49)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((item.i_current_price >= 0.99)(item.i_current_price <= 1.49)) -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_date >= 2001-03-03)(date_dim.d_date <= 2001-05-02)) -----------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +----------------------------filter((date_dim.d_date >= 2001-03-03)(date_dim.d_date <= 2001-05-02)) +------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out index 1d86178c4c2c55..9a5ec3b3811873 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_41 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact = i1.i_manufact) -----------------PhysicalProject -------------------filter((i1.i_manufact_id >= 748)(i1.i_manufact_id <= 788)) ---------------------PhysicalOlapScan[item] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact = i1.i_manufact) ------------------PhysicalProject ---------------------filter((item_cnt > 0)) -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((((((((((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'gainsboro') OR (cast(i_color as VARCHAR(*)) = 'aquamarine'))) AND ((cast(i_units as VARCHAR(*)) = 'Ounce') OR (cast(i_units as VARCHAR(*)) = 'Dozen'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'chiffon') OR (cast(i_color as VARCHAR(*)) = 'violet'))) AND ((cast(i_units as VARCHAR(*)) = 'Ton') OR (cast(i_units as VARCHAR(*)) = 'Pound'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'blanched') OR (cast(i_color as VARCHAR(*)) = 'tomato'))) AND ((cast(i_units as VARCHAR(*)) = 'Tbl') OR (cast(i_units as VARCHAR(*)) = 'Case'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'almond') OR (cast(i_color as VARCHAR(*)) = 'lime'))) AND ((cast(i_units as VARCHAR(*)) = 'Box') OR (cast(i_units as VARCHAR(*)) = 'Dram'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'chartreuse') OR (cast(i_color as VARCHAR(*)) = 'blue'))) AND ((cast(i_units as VARCHAR(*)) = 'Each') OR (cast(i_units as VARCHAR(*)) = 'Oz'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'tan') OR (cast(i_color as VARCHAR(*)) = 'dodger'))) AND ((cast(i_units as VARCHAR(*)) = 'Bunch') OR (cast(i_units as VARCHAR(*)) = 'Tsp'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'peru') OR (cast(i_color as VARCHAR(*)) = 'saddle'))) AND ((cast(i_units as VARCHAR(*)) = 'Pallet') OR (cast(i_units as VARCHAR(*)) = 'Gram'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'indian') OR (cast(i_color as VARCHAR(*)) = 'spring'))) AND ((cast(i_units as VARCHAR(*)) = 'Unknown') OR (cast(i_units as VARCHAR(*)) = 'Carton'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))))) ---------------------------------PhysicalOlapScan[item] +--------------------filter((i1.i_manufact_id >= 748)(i1.i_manufact_id <= 788)) +----------------------PhysicalOlapScan[item] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((item_cnt > 0)) +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((((((((((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'gainsboro') OR (cast(i_color as VARCHAR(*)) = 'aquamarine'))) AND ((cast(i_units as VARCHAR(*)) = 'Ounce') OR (cast(i_units as VARCHAR(*)) = 'Dozen'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'chiffon') OR (cast(i_color as VARCHAR(*)) = 'violet'))) AND ((cast(i_units as VARCHAR(*)) = 'Ton') OR (cast(i_units as VARCHAR(*)) = 'Pound'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'blanched') OR (cast(i_color as VARCHAR(*)) = 'tomato'))) AND ((cast(i_units as VARCHAR(*)) = 'Tbl') OR (cast(i_units as VARCHAR(*)) = 'Case'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'almond') OR (cast(i_color as VARCHAR(*)) = 'lime'))) AND ((cast(i_units as VARCHAR(*)) = 'Box') OR (cast(i_units as VARCHAR(*)) = 'Dram'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'chartreuse') OR (cast(i_color as VARCHAR(*)) = 'blue'))) AND ((cast(i_units as VARCHAR(*)) = 'Each') OR (cast(i_units as VARCHAR(*)) = 'Oz'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'tan') OR (cast(i_color as VARCHAR(*)) = 'dodger'))) AND ((cast(i_units as VARCHAR(*)) = 'Bunch') OR (cast(i_units as VARCHAR(*)) = 'Tsp'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'peru') OR (cast(i_color as VARCHAR(*)) = 'saddle'))) AND ((cast(i_units as VARCHAR(*)) = 'Pallet') OR (cast(i_units as VARCHAR(*)) = 'Gram'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'indian') OR (cast(i_color as VARCHAR(*)) = 'spring'))) AND ((cast(i_units as VARCHAR(*)) = 'Unknown') OR (cast(i_units as VARCHAR(*)) = 'Carton'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))))) +----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out index f717b0d5783e17..298efbd86325e4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out @@ -1,22 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_42 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_sales] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((item.i_manager_id = 1)) +--------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((item.i_manager_id = 1)) -------------------------PhysicalOlapScan[item] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((dt.d_year = 2002)(dt.d_moy = 11)) -----------------------PhysicalOlapScan[date_dim] +----------------------filter((dt.d_year = 2002)(dt.d_moy = 11)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out index f36cfba5ae002e..53978bf7a7e916 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out @@ -1,22 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_43 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_sales] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((store.s_gmt_offset = -5.00)) -----------------------PhysicalOlapScan[store] +----------------------filter((store.s_gmt_offset = -5.00)) +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out index 5a3c0b08d90a9c..3f3f830e5d3a77 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out @@ -1,17 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_44 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) -----------PhysicalProject -------------PhysicalOlapScan[item] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk) ---------------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[item] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) +------------PhysicalProject +--------------PhysicalOlapScan[item] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk) +----------------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((rnk < 11)) +------------------------PhysicalWindow +--------------------------PhysicalQuickSort +----------------------------PhysicalDistribute +------------------------------PhysicalQuickSort +--------------------------------PhysicalPartitionTopN +----------------------------------PhysicalProject +------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) +--------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[LOCAL] +--------------------------------------------PhysicalProject +----------------------------------------------filter((ss1.ss_store_sk = 146)) +------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------hashAgg[GLOBAL] +------------------------------------------------PhysicalDistribute +--------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) +--------------------------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((rnk < 11)) @@ -38,30 +65,4 @@ PhysicalTopN --------------------------------------------------PhysicalProject ----------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) ------------------------------------------------------PhysicalOlapScan[store_sales] ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((rnk < 11)) ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalPartitionTopN -------------------------------PhysicalProject ---------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------filter((ss1.ss_store_sk = 146)) ---------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalAssertNumRows ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) -----------------------------------------------------PhysicalOlapScan[store_sales] 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 e84dbdaabe0fb7..83d87e1582a246 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,41 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR $c$1)) -----------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_qoy = 2)(date_dim.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------filter((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR $c$1)) +------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) --------------------PhysicalProject -----------------------filter(i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)) -------------------------PhysicalOlapScan[item] +----------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_qoy = 2)(date_dim.d_year = 2000)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)) +--------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out index bd24b2de19c35c..a21333fa4286af 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out @@ -1,42 +1,43 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_46 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk)( not (ca_city = bought_city)) -----------PhysicalDistribute -------------PhysicalProject ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk)( not (ca_city = bought_city)) +------------PhysicalDistribute +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(d_dow IN (0, 6)d_year IN (1999, 2000, 2001)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter(d_dow IN (0, 6)d_year IN (1999, 2000, 2001)) -------------------------------------PhysicalOlapScan[date_dim] +----------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) -----------------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter(s_city IN ('Five Points', 'Centerville', 'Oak Grove', 'Fairview', 'Liberty')) ---------------------------------PhysicalOlapScan[store] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) ---------------PhysicalProject -----------------PhysicalOlapScan[customer_address] ---------------PhysicalDistribute +--------------------------------filter(s_city IN ('Five Points', 'Centerville', 'Oak Grove', 'Fairview', 'Liberty')) +----------------------------------PhysicalOlapScan[store] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) ----------------PhysicalProject -------------------PhysicalOlapScan[customer] +------------------PhysicalOlapScan[customer_address] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out index 7b27c69128a9b9..b705f7a2cbf388 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out @@ -29,23 +29,24 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store] ---PhysicalProject -----PhysicalTopN -------PhysicalDistribute ---------PhysicalTopN -----------PhysicalProject -------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +--PhysicalResultSink +----PhysicalProject +------PhysicalTopN +--------PhysicalDistribute +----------PhysicalTopN +------------PhysicalProject +--------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) +----------------------PhysicalDistribute +------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out index 20b2439b02ef22..00af1ea47653ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out @@ -1,31 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_48 -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------PhysicalProject -------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('MD', 'MN', 'IA') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('VA', 'IL', 'TX') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('MI', 'WI', 'IN') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) ---------------------PhysicalProject -----------------------filter(((((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) -------------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------PhysicalProject +--------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('MD', 'MN', 'IA') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('VA', 'IL', 'TX') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('MI', 'WI', 'IN') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) ----------------------PhysicalProject -------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')))) ---------------------------PhysicalOlapScan[customer_demographics] +------------------------filter(((((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +--------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')))) +----------------------------PhysicalOlapScan[customer_demographics] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter(((ca_state IN ('MD', 'MN', 'IA') OR ca_state IN ('VA', 'IL', 'TX')) OR ca_state IN ('MI', 'WI', 'IN'))(customer_address.ca_country = 'United States')) +------------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter(((ca_state IN ('MD', 'MN', 'IA') OR ca_state IN ('VA', 'IL', 'TX')) OR ca_state IN ('MI', 'WI', 'IN'))(customer_address.ca_country = 'United States')) -----------------------PhysicalOlapScan[customer_address] ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((date_dim.d_year = 1999)) ---------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------PhysicalProject ---------------PhysicalOlapScan[store] +--------------------filter((date_dim.d_year = 1999)) +----------------------PhysicalOlapScan[date_dim] +------------PhysicalDistribute +--------------PhysicalProject +----------------PhysicalOlapScan[store] 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 188d6d3c7b282b..b5dc9c3f7050a1 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,86 +1,87 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[LOCAL] ---------PhysicalUnion -----------PhysicalDistribute -------------PhysicalProject ---------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) -----------------------------------------PhysicalProject -------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute +--------------PhysicalProject +----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------------PhysicalWindow +--------------------PhysicalQuickSort +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------PhysicalQuickSort +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) -----------------------------------------------PhysicalOlapScan[web_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------PhysicalOlapScan[web_returns] +------------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------PhysicalProject ---------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) -----------------------------------------PhysicalProject -------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) +------------------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------------PhysicalWindow +--------------------PhysicalQuickSort +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------PhysicalQuickSort +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) -----------------------------------------------PhysicalOlapScan[catalog_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------PhysicalProject ---------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) -----------------------------------------PhysicalProject -------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) +------------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------------PhysicalWindow +--------------------PhysicalQuickSort +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------PhysicalQuickSort +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) -----------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------PhysicalOlapScan[store_returns] +------------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) +------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out index eb45d441220307..49a99dc3af856f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out @@ -1,79 +1,80 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_5 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](salesreturns.store_sk = store.s_store_sk) -------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------PhysicalUnion +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](salesreturns.store_sk = store.s_store_sk) +--------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +----------------------------------PhysicalUnion +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------filter((date_dim.d_date <= 2000-09-02)(date_dim.d_date >= 2000-08-19)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date <= 2000-09-02)(date_dim.d_date >= 2000-08-19)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) -------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------PhysicalUnion -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) +--------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +----------------------------------PhysicalUnion +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_returns] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_page] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](salesreturns.wsr_web_site_sk = web_site.web_site_sk) -------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------PhysicalUnion -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalOlapScan[catalog_page] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](salesreturns.wsr_web_site_sk = web_site.web_site_sk) +--------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +----------------------------------PhysicalUnion +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_returns] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_site] +------------------------------------PhysicalOlapScan[web_site] 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 921f84d72cda7b..e5ee3833007169 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,31 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +--------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] -------------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[store_sales] +------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter((d2.d_year = 2001)(d2.d_moy = 8)) -------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((d2.d_year = 2001)(d2.d_moy = 8)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out index 66c68bed08495c..8ba49dc8d60bdd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out @@ -1,42 +1,43 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_51 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------filter((web_cumulative > store_cumulative)) ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) -------------------PhysicalProject ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------filter((web_cumulative > store_cumulative)) +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) +--------------------PhysicalProject +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) -------------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] -------------------------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) -------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) +--------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out index bf191b847cbd5c..4bf2ceed3dcbef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_52 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((dt.d_moy = 12)(dt.d_year = 2002)) -------------------------PhysicalOlapScan[date_dim] +------------------------filter((dt.d_moy = 12)(dt.d_year = 2002)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out index 0e916de7170f7e..87e4abe7797004 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_53 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((CASE WHEN (avg_quarterly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_quarterly_sales as DOUBLE))) / cast(avg_quarterly_sales as DOUBLE)) ELSE NULL END > 0.1)) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((CASE WHEN (avg_quarterly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_quarterly_sales as DOUBLE))) / cast(avg_quarterly_sales as DOUBLE)) ELSE NULL END > 0.1)) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) -----------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +--------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out index e6002e5a2d6c48..70a48fdb34e108 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out @@ -1,82 +1,83 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_54 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer_address.ca_county = store.s_county)(customer_address.ca_state = store.s_state) ---------------------------PhysicalProject -----------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) -------------------------------PhysicalProject ---------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](my_customers.c_customer_sk = store_sales.ss_customer_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer_address] ---------------------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer_address.ca_county = store.s_county)(customer_address.ca_state = store.s_state) +----------------------------PhysicalProject +------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +--------------------------------PhysicalProject +----------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](my_customers.c_customer_sk = store_sales.ss_customer_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute -----------------------------------------------------hashAgg[LOCAL] -------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[customer] -----------------------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) ---------------------------------------------------------------------PhysicalUnion -----------------------------------------------------------------------PhysicalDistribute -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------------------------------PhysicalOlapScan[customer] +------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) +------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) +----------------------------------------------------------------------PhysicalUnion +------------------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------PhysicalOlapScan[web_sales] ----------------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------------------------------------PhysicalDistribute -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------filter((cast(i_class as VARCHAR(*)) = 'maternity')(cast(i_category as VARCHAR(*)) = 'Women')) ---------------------------------------------------------------------------PhysicalOlapScan[item] -----------------------------------------------------------------PhysicalDistribute -------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) -----------------------------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalAssertNumRows ---------------------------------------PhysicalDistribute -----------------------------------------hashAgg[GLOBAL] -------------------------------------------PhysicalDistribute ---------------------------------------------hashAgg[LOCAL] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalAssertNumRows -----------------------------------PhysicalDistribute -------------------------------------hashAgg[GLOBAL] +--------------------------------------------------------------------------filter((cast(i_class as VARCHAR(*)) = 'maternity')(cast(i_category as VARCHAR(*)) = 'Women')) +----------------------------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +------------------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------PhysicalDistribute -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) -----------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalAssertNumRows +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[GLOBAL] +--------------------------------------------PhysicalDistribute +----------------------------------------------hashAgg[LOCAL] +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +----------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute +--------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[LOCAL] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out index f4e966c695017e..90db0b7ca05f11 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_55 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 100)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 100)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out index a7efe6f1d67a0c..b2821d91e97bb3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out @@ -1,93 +1,94 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_56 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_gmt_offset = -6.00)) ---------------------------------PhysicalOlapScan[customer_address] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute +--------------------------------filter((customer_address.ca_gmt_offset = -6.00)) +----------------------------------PhysicalOlapScan[customer_address] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_gmt_offset = -6.00)) +----------------------------------PhysicalOlapScan[customer_address] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalProject ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) --------------------------------PhysicalOlapScan[customer_address] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalProject -----------------------------filter((customer_address.ca_gmt_offset = -6.00)) -------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out index 0e13df083b4ceb..d3d96f0926e446 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out @@ -29,23 +29,24 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[call_center] ---PhysicalProject -----PhysicalTopN -------PhysicalDistribute ---------PhysicalTopN -----------PhysicalProject -------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 1999)(v2.avg_monthly_sales > 0.0000)) +--PhysicalResultSink +----PhysicalProject +------PhysicalTopN +--------PhysicalDistribute +----------PhysicalTopN +------------PhysicalProject +--------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) +----------------------PhysicalDistribute +------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 1999)(v2.avg_monthly_sales > 0.0000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out index 7ffaac876a79d8..e5fd32f39b6626 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out @@ -1,39 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_58 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](ss_items.item_id = cs_items.item_id)(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE)) -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date = 2001-03-24)) ---------------------------------------PhysicalOlapScan[date_dim] -----------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](ss_items.item_id = cs_items.item_id)(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE)) ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute @@ -41,10 +13,10 @@ PhysicalTopN --------------------PhysicalProject ----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] @@ -62,32 +34,61 @@ PhysicalTopN ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_date = 2001-03-24)) ----------------------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] +------------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_date = 2001-03-24)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute ---------------------------------PhysicalAssertNumRows -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_date = 2001-03-24)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_date = 2001-03-24)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out index 363cf914e02226..5b0559251ee462 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out @@ -13,37 +13,38 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) -------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) -------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +--------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalOlapScan[store] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +--------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +------------------------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 853909e2c94430..fff91e211ebb57 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,52 +1,53 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------filter((cnt >= 10)) ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](j.i_category = i.i_category)(cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(i_current_price))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](c.c_customer_sk = s.ss_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](s.ss_item_sk = i.i_item_sk) -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](s.ss_sold_date_sk = d.d_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------filter((cnt >= 10)) +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[LEFT_SEMI_JOIN](j.i_category = i.i_category)(cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(i_current_price))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](c.c_customer_sk = s.ss_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](s.ss_item_sk = i.i_item_sk) +------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](s.ss_sold_date_sk = d.d_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------hashJoin[INNER_JOIN](d.d_month_seq = date_dim.d_month_seq) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 3)) +------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](d.d_month_seq = date_dim.d_month_seq) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 3)) -----------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](a.ca_address_sk = c.c_current_addr_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] +----------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] +----------------------------hashJoin[INNER_JOIN](a.ca_address_sk = c.c_current_addr_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out index 8538ed1f7b5aef..c09adeeea2fcb1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out @@ -1,94 +1,95 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_60 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +----------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute +------------------------------------filter((item.i_category = 'Children')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +----------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute +------------------------------------filter((item.i_category = 'Children')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] +----------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Children')) +--------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out index beaf2eb582bb37..bc1155555ed9b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out @@ -1,48 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_61 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------NestedLoopJoin[CROSS_JOIN] ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter((cast(i_category as VARCHAR(*)) = 'Jewelry')) ---------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -----------------------------------PhysicalProject -------------------------------------filter((((cast(p_channel_dmail as VARCHAR(*)) = 'Y') OR (cast(p_channel_email as VARCHAR(*)) = 'Y')) OR (cast(p_channel_tv as VARCHAR(*)) = 'Y'))) ---------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) -------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((store.s_gmt_offset = -7.00)) ---------------------------------------------------PhysicalOlapScan[store] ---------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------NestedLoopJoin[CROSS_JOIN] ----------hashAgg[GLOBAL] ------------PhysicalDistribute --------------hashAgg[LOCAL] @@ -58,21 +20,60 @@ PhysicalTopN ----------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer] ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------------filter((((cast(p_channel_dmail as VARCHAR(*)) = 'Y') OR (cast(p_channel_email as VARCHAR(*)) = 'Y')) OR (cast(p_channel_tv as VARCHAR(*)) = 'Y'))) +----------------------------------------PhysicalOlapScan[promotion] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +--------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((store.s_gmt_offset = -7.00)) -----------------------------------------------PhysicalOlapScan[store] +--------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((store.s_gmt_offset = -7.00)) +----------------------------------------------------PhysicalOlapScan[store] +----------PhysicalDistribute +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +----------------------PhysicalProject +------------------------filter((customer_address.ca_gmt_offset = -7.00)) +--------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------filter((cast(i_category as VARCHAR(*)) = 'Jewelry')) +------------------------------PhysicalOlapScan[item] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((store.s_gmt_offset = -7.00)) +------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out index f9551baabdad88..ec78b77b89e5f3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_62 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) ---------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) -----------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalOlapScan[web_site] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_site] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[ship_mode] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------PhysicalOlapScan[ship_mode] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out index 906693680efaad..eea2cefd61a2fa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_63 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) -----------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +--------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out index 0cf4b87d09e1f1..2d74160159d40d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out @@ -1,45 +1,46 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_65 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](item.i_item_sk = sc.ss_item_sk) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store.s_store_sk = sc.ss_store_sk) -----------------hashJoin[INNER_JOIN](sb.ss_store_sk = sc.ss_store_sk)(cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE)) -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1232)(date_dim.d_month_seq >= 1221)) -----------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](item.i_item_sk = sc.ss_item_sk) +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store.s_store_sk = sc.ss_store_sk) +------------------hashJoin[INNER_JOIN](sb.ss_store_sk = sc.ss_store_sk)(cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE)) --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq <= 1232)(date_dim.d_month_seq >= 1221)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq >= 1221)(date_dim.d_month_seq <= 1232)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[store] -----------PhysicalDistribute -------------PhysicalProject ---------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_month_seq >= 1221)(date_dim.d_month_seq <= 1232)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] +------------PhysicalDistribute +--------------PhysicalProject +----------------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 ec829b426c9dfa..d83d9cc0f0da0c 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,64 +1,65 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +--------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------PhysicalOlapScan[ship_mode] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------PhysicalOlapScan[ship_mode] +--------------------------------------filter((date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) -------------------------------------PhysicalOlapScan[time_dim] ---------------------------PhysicalDistribute +------------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) +--------------------------------------PhysicalOlapScan[time_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[warehouse] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[warehouse] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) +--------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------PhysicalOlapScan[ship_mode] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------PhysicalOlapScan[ship_mode] +--------------------------------------filter((date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) -------------------------------------PhysicalOlapScan[time_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[warehouse] +------------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) +--------------------------------------PhysicalOlapScan[time_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out index b2c9bc02f78faa..b9707e1b5a25e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------filter((rk <= 100)) ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------PhysicalPartitionTopN -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------filter((rk <= 100)) +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalPartitionTopN +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq >= 1206)(date_dim.d_month_seq <= 1217)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_month_seq >= 1206)(date_dim.d_month_seq <= 1217)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out index 94796392312528..9a98918d915aa5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out @@ -1,43 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_68 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk)( not (ca_city = bought_city)) -----------PhysicalProject -------------PhysicalOlapScan[customer_address] -----------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk)( not (ca_city = bought_city)) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute +--------------PhysicalOlapScan[customer_address] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_dom >= 1)d_year IN (1998, 1999, 2000)(date_dim.d_dom <= 2)) +----------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_dom >= 1)d_year IN (1998, 1999, 2000)(date_dim.d_dom <= 2)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter(s_city IN ('Pleasant Hill', 'Five Points')) -------------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) ---------------------------------------PhysicalOlapScan[household_demographics] +------------------------------------------filter(s_city IN ('Pleasant Hill', 'Five Points')) +--------------------------------------------PhysicalOlapScan[store] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +----------------------------------------PhysicalOlapScan[household_demographics] 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 e01ec61085cb5c..2f5465dccc0ef0 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,55 +1,56 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] +----------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2000)(date_dim.d_moy <= 3)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute -----------------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +--------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2000)(date_dim.d_moy <= 3)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute +------------------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalOlapScan[customer] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) -------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(ca_state IN ('TX', 'VA', 'MI')) -----------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------filter(ca_state IN ('TX', 'VA', 'MI')) +------------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out index 69cc851c1d3eb9..574836334751d6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_7 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')(cast(cd_education_status as VARCHAR(*)) = 'College')(cast(cd_gender as VARCHAR(*)) = 'F')) +----------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')(cast(cd_education_status as VARCHAR(*)) = 'College')(cast(cd_gender as VARCHAR(*)) = 'F')) ---------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) ---------------------------PhysicalOlapScan[promotion] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) +----------------------------PhysicalOlapScan[promotion] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[item] 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 b5255185bd4f80..f85e1f6fbfb220 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 @@ -1,50 +1,51 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_70 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state) +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +--------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((ranking <= 5)) -----------------------------------------PhysicalWindow -------------------------------------------PhysicalQuickSort ---------------------------------------------PhysicalPartitionTopN -----------------------------------------------hashAgg[GLOBAL] -------------------------------------------------PhysicalDistribute ---------------------------------------------------hashAgg[LOCAL] -----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213)) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((ranking <= 5)) +------------------------------------------PhysicalWindow +--------------------------------------------PhysicalQuickSort +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------hashAgg[GLOBAL] +--------------------------------------------------PhysicalDistribute +----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------------PhysicalProject +--------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +----------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) +------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------PhysicalDistribute ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) -----------------------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------------------PhysicalDistribute -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[store] +--------------------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out index 650c95b66811dd..0347eb2b3737b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_71 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) -------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) ---------------------PhysicalUnion -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) +--------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) +----------------------PhysicalUnion +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter(((cast(t_meal_time as VARCHAR(*)) = 'breakfast') OR (cast(t_meal_time as VARCHAR(*)) = 'dinner'))) -------------------------PhysicalOlapScan[time_dim] +------------------------filter(((cast(t_meal_time as VARCHAR(*)) = 'breakfast') OR (cast(t_meal_time as VARCHAR(*)) = 'dinner'))) +--------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out index 0e04db4a1fe30f..b3747e4573d7bb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out @@ -1,61 +1,62 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_72 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_OUTER_JOIN](catalog_returns.cr_item_sk = catalog_sales.cs_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) -----------------PhysicalProject -------------------PhysicalOlapScan[catalog_returns] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[RIGHT_OUTER_JOIN](catalog_returns.cr_item_sk = catalog_sales.cs_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) ------------------PhysicalProject ---------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](warehouse.w_warehouse_sk = inventory.inv_warehouse_sk) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = d2.d_date_sk)(d1.d_week_seq = d2.d_week_seq) -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalOlapScan[inventory] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) ---------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) -----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) -------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------PhysicalOlapScan[catalog_returns] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](warehouse.w_warehouse_sk = inventory.inv_warehouse_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = d2.d_date_sk)(d1.d_week_seq = d2.d_week_seq) +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalOlapScan[inventory] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) +----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) +------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +--------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------------------PhysicalDistribute +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cast(hd_buy_potential as VARCHAR(*)) = '501-1000')) +----------------------------------------------------------PhysicalOlapScan[household_demographics] --------------------------------------------------PhysicalDistribute ----------------------------------------------------PhysicalProject -------------------------------------------------------filter((cast(hd_buy_potential as VARCHAR(*)) = '501-1000')) ---------------------------------------------------------PhysicalOlapScan[household_demographics] +------------------------------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')) +--------------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------------PhysicalDistribute --------------------------------------------------PhysicalProject -----------------------------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')) -------------------------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------------------filter((d1.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------PhysicalDistribute ------------------------------------------------PhysicalProject ---------------------------------------------------filter((d1.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[warehouse] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[promotion] +--------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[warehouse] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[promotion] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out index 422345128c4877..61342aa658f34b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_73 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](dj.ss_customer_sk = customer.c_customer_sk) -----------PhysicalProject -------------PhysicalOlapScan[customer] -----------PhysicalDistribute -------------filter((dj.cnt >= 1)(dj.cnt <= 5)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](dj.ss_customer_sk = customer.c_customer_sk) +------------PhysicalProject +--------------PhysicalOlapScan[customer] +------------PhysicalDistribute +--------------filter((dj.cnt >= 1)(dj.cnt <= 5)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_dom >= 1)d_year IN (2000, 2001, 2002)(date_dim.d_dom <= 2)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_dom >= 1)d_year IN (2000, 2001, 2002)(date_dim.d_dom <= 2)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '501-1000') OR (cast(hd_buy_potential as VARCHAR(*)) = 'Unknown'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.0)) +----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '501-1000') OR (cast(hd_buy_potential as VARCHAR(*)) = 'Unknown'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.0)) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(s_county IN ('Fairfield County', 'Walker County', 'Daviess County', 'Barrow County')) -------------------------------PhysicalOlapScan[store] +------------------------------filter(s_county IN ('Fairfield County', 'Walker County', 'Daviess County', 'Barrow County')) +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index dc32f6e970315c..7e915785b8bab2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -40,27 +40,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter('w' IN ('s', 'w')) ------------------------PhysicalOlapScan[customer] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END) -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id) ---------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END) +--------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id) +----------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) +--------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) +------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out index 09a72797851f63..50de18777c8ebd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out @@ -61,15 +61,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) ------------------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](curr_yr.i_brand_id = prev_yr.i_brand_id)(curr_yr.i_class_id = prev_yr.i_class_id)(curr_yr.i_category_id = prev_yr.i_category_id)(curr_yr.i_manufact_id = prev_yr.i_manufact_id)((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000) -------------PhysicalDistribute ---------------filter((curr_yr.d_year = 1999)) -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------filter((prev_yr.d_year = 1998)) -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](curr_yr.i_brand_id = prev_yr.i_brand_id)(curr_yr.i_class_id = prev_yr.i_class_id)(curr_yr.i_category_id = prev_yr.i_category_id)(curr_yr.i_manufact_id = prev_yr.i_manufact_id)((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000) +--------------PhysicalDistribute +----------------filter((curr_yr.d_year = 1999)) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------filter((prev_yr.d_year = 1998)) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 809d43e7b69bd5..45631094874884 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,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter(ss_hdemo_sk IS NULL) ---------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------filter(ss_hdemo_sk IS NULL) +----------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------PhysicalOlapScan[web_sales] +--------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter(cs_warehouse_sk IS NULL) ---------------------------PhysicalOlapScan[catalog_sales] +--------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------filter(cs_warehouse_sk IS NULL) +----------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out index 649b78703e9e23..6758459211b249 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out @@ -1,107 +1,108 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_77 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[LEFT_OUTER_JOIN](ss.s_store_sk = sr.s_store_sk) -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashJoin[LEFT_OUTER_JOIN](ss.s_store_sk = sr.s_store_sk) +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] +----------------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------NestedLoopJoin[CROSS_JOIN] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalProject --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------PhysicalOlapScan[catalog_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) ------------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[LEFT_OUTER_JOIN](ws.wp_web_page_sk = wr.wp_web_page_sk) -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[LEFT_OUTER_JOIN](ws.wp_web_page_sk = wr.wp_web_page_sk) +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_page] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) -----------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] +----------------------------------------PhysicalOlapScan[web_page] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) +------------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_page] +----------------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out index 855043acbed68b..812bbb057959b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_78 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) -----------hashJoin[LEFT_OUTER_JOIN](cs.cs_sold_year = ss.ss_sold_year)(cs.cs_item_sk = ss.ss_item_sk)(cs.cs_customer_sk = ss.ss_customer_sk) -------------PhysicalProject ---------------hashJoin[LEFT_OUTER_JOIN](ws.ws_sold_year = ss.ss_sold_year)(ws.ws_item_sk = ss.ss_item_sk)(ws.ws_customer_sk = ss.ss_customer_sk) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------filter(sr_ticket_number IS NULL) ---------------------------------hashJoin[LEFT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_returns] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------filter(wr_order_number IS NULL) ---------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_returns] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------filter(cr_order_number IS NULL) -----------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +------------hashJoin[LEFT_OUTER_JOIN](cs.cs_sold_year = ss.ss_sold_year)(cs.cs_item_sk = ss.ss_item_sk)(cs.cs_customer_sk = ss.ss_customer_sk) +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN](ws.ws_sold_year = ss.ss_sold_year)(ws.ws_item_sk = ss.ss_item_sk)(ws.ws_customer_sk = ss.ss_customer_sk) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------filter(sr_ticket_number IS NULL) +----------------------------------hashJoin[LEFT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_returns] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] -------------------------PhysicalDistribute +--------------------------------filter(wr_order_number IS NULL) +----------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_returns] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter(cr_order_number IS NULL) +------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out index d51a2de9696c87..05f8bd4c577563 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_79 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](ms.ss_customer_sk = customer.c_customer_sk) -----------PhysicalDistribute -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](ms.ss_customer_sk = customer.c_customer_sk) +------------PhysicalDistribute +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(d_year IN (1998, 1999, 2000)(date_dim.d_dow = 1)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(d_year IN (1998, 1999, 2000)(date_dim.d_dow = 1)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((store.s_number_employees >= 200)(store.s_number_employees <= 295)) -------------------------------PhysicalOlapScan[store] -----------PhysicalDistribute -------------PhysicalProject ---------------PhysicalOlapScan[customer] +------------------------------filter((store.s_number_employees >= 200)(store.s_number_employees <= 295)) +--------------------------------PhysicalOlapScan[store] +------------PhysicalDistribute +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out index b00cd15207f6fc..2c6d8aff03ad51 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_8 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](expr_substring(s_zip, 1, 2) = expr_substring(ca_zip, 1, 2)) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_year = 1998)(date_dim.d_qoy = 2)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[store] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](expr_substring(s_zip, 1, 2) = expr_substring(ca_zip, 1, 2)) ------------------PhysicalProject ---------------------PhysicalIntersect -----------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------PhysicalProject ---------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) -----------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((date_dim.d_year = 1998)(date_dim.d_qoy = 2)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((cnt > 10)) -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] ---------------------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[store] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalIntersect +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) +------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((cnt > 10)) +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((cast(c_preferred_cust_flag as VARCHAR(*)) = 'Y')) ---------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((cast(c_preferred_cust_flag as VARCHAR(*)) = 'Y')) +----------------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out index 29730c3a44aa09..da407a03dc5a24 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out @@ -1,102 +1,103 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_80 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) -----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------filter((item.i_current_price > 50.00)) +----------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------PhysicalOlapScan[item] +------------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) -------------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk) -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk) +------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) -----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------filter((item.i_current_price > 50.00)) +----------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------PhysicalOlapScan[item] +------------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) -------------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_page] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------PhysicalOlapScan[catalog_page] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) +--------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) -----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------filter((item.i_current_price > 50.00)) +----------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------PhysicalOlapScan[item] +------------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) -------------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_site] +----------------------------------------PhysicalOlapScan[web_site] 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 11524e0f10bfcc..15caa8024db395 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 @@ -19,28 +19,29 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) -------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) ---------------PhysicalDistribute -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +--------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +----------------PhysicalDistribute +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------PhysicalProject -----------------------filter((cast(ca_state as VARCHAR(*)) = 'CA')) -------------------------PhysicalOlapScan[customer_address] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((cast(ca_state as VARCHAR(*)) = 'CA')) +--------------------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out index 6a27e9eece414e..c9976cb1eb7975 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_82 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) -----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) -------------------------PhysicalProject ---------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) -----------------------------PhysicalOlapScan[inventory] +--------------------PhysicalOlapScan[store_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) +------------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) +------------------------------PhysicalOlapScan[inventory] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((item.i_current_price >= 17.00)i_manufact_id IN (138, 169, 339, 639)(item.i_current_price <= 47.00)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((item.i_current_price >= 17.00)i_manufact_id IN (138, 169, 339, 639)(item.i_current_price <= 47.00)) -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= 1999-09-07)(date_dim.d_date >= 1999-07-09)) -----------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_date <= 1999-09-07)(date_dim.d_date >= 1999-07-09)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out index 67d4702363e19b..f1f3cfaf941eab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out @@ -1,37 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_83 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](sr_items.item_id = cr_items.item_id) -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) ---------------------------hashJoin[INNER_JOIN](catalog_returns.cr_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) -----------------------------------PhysicalOlapScan[date_dim] -----------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](sr_items.item_id = cr_items.item_id) ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute @@ -39,10 +13,10 @@ PhysicalTopN --------------------PhysicalProject ----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](catalog_returns.cr_item_sk = item.i_item_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] @@ -58,30 +32,57 @@ PhysicalTopN --------------------------------PhysicalProject ----------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) ------------------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] +------------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] +----------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) -------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) +--------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out index 79cda5d28acb37..710652c2f1b65e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_84 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk) -----------PhysicalProject -------------PhysicalOlapScan[store_returns] -----------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[customer_demographics] -----------------PhysicalDistribute +--------------PhysicalOlapScan[store_returns] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] ---------------------------PhysicalDistribute +--------------------PhysicalOlapScan[customer_demographics] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalProject -------------------------------filter((customer_address.ca_city = 'Oakwood')) ---------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](income_band.ib_income_band_sk = household_demographics.hd_income_band_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_city = 'Oakwood')) +----------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](income_band.ib_income_band_sk = household_demographics.hd_income_band_sk) ----------------------------PhysicalProject -------------------------------filter((cast(ib_upper_bound as BIGINT) <= 55806)(income_band.ib_lower_bound >= 5806)) ---------------------------------PhysicalOlapScan[income_band] +------------------------------PhysicalOlapScan[household_demographics] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((cast(ib_upper_bound as BIGINT) <= 55806)(income_band.ib_lower_bound >= 5806)) +----------------------------------PhysicalOlapScan[income_band] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out index a078d9cfe1ccf6..77be6e85445279 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_85 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)(cd1.cd_marital_status = cd2.cd_marital_status)(cd1.cd_education_status = cd2.cd_education_status) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer_demographics] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)(cd1.cd_marital_status = cd2.cd_marital_status)(cd1.cd_education_status = cd2.cd_education_status) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_page] -------------------------PhysicalDistribute +----------------------PhysicalOlapScan[customer_demographics] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) -------------------------------PhysicalProject ---------------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) -----------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](reason.r_reason_sk = web_returns.wr_reason_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[reason] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)(((ca_state IN ('FL', 'TX', 'DE') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('IN', 'ND', 'ID') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('MT', 'IL', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))) ---------------------------------------PhysicalProject -----------------------------------------filter(((ca_state IN ('FL', 'TX', 'DE') OR ca_state IN ('IN', 'ND', 'ID')) OR ca_state IN ('MT', 'IL', 'OH'))(customer_address.ca_country = 'United States')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------------------PhysicalDistribute +----------------------------PhysicalOlapScan[web_page] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) +--------------------------------PhysicalProject +----------------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) +------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](reason.r_reason_sk = web_returns.wr_reason_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[reason] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)(((ca_state IN ('FL', 'TX', 'DE') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('IN', 'ND', 'ID') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('MT', 'IL', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))) ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[web_returns] ---------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------filter(((ca_state IN ('FL', 'TX', 'DE') OR ca_state IN ('IN', 'ND', 'ID')) OR ca_state IN ('MT', 'IL', 'OH'))(customer_address.ca_country = 'United States')) +--------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) ----------------------------------------------PhysicalProject -------------------------------------------------filter(((((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00)) OR ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00))) OR ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))((((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00)) OR ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00))) OR ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) ---------------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalOlapScan[web_returns] +----------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_year = 2000)) -----------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------filter(((((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00)) OR ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00))) OR ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))((((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00)) OR ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00))) OR ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) +----------------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out index 6bb40cfe302c87..63e6ce5ba71dab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_86 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) -------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = web_sales.ws_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +--------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = web_sales.ws_sold_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((d1.d_month_seq <= 1235)(d1.d_month_seq >= 1224)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((d1.d_month_seq <= 1235)(d1.d_month_seq >= 1224)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out index 4b5af9e2a8ce95..c42693e374ca2f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_87 -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------PhysicalExcept -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------PhysicalExcept +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1195)(date_dim.d_month_seq >= 1184)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq <= 1195)(date_dim.d_month_seq >= 1184)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out index 5242db935c1e2e..e8b4b57dc82f4b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out @@ -1,34 +1,57 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_88 -- -PhysicalProject ---NestedLoopJoin[CROSS_JOIN] +PhysicalResultSink +--PhysicalProject ----NestedLoopJoin[CROSS_JOIN] ------NestedLoopJoin[CROSS_JOIN] --------NestedLoopJoin[CROSS_JOIN] ----------NestedLoopJoin[CROSS_JOIN] ------------NestedLoopJoin[CROSS_JOIN] --------------NestedLoopJoin[CROSS_JOIN] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] +----------------NestedLoopJoin[CROSS_JOIN] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +--------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) -------------------------------------PhysicalOlapScan[time_dim] +----------------------------------filter((store.s_store_name = 'ese')) +------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((store.s_store_name = 'ese')) -----------------------------------PhysicalOlapScan[store] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) ---------------------------------PhysicalOlapScan[household_demographics] +--------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) +----------------------------------PhysicalOlapScan[household_demographics] +------------------PhysicalDistribute +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((time_dim.t_hour = 9)(time_dim.t_minute < 30)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((store.s_store_name = 'ese')) +--------------------------------------PhysicalOlapScan[store] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) +------------------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalDistribute ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute @@ -41,7 +64,7 @@ PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_hour = 9)(time_dim.t_minute < 30)) +------------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 9)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject @@ -63,7 +86,7 @@ PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 9)) +----------------------------------filter((time_dim.t_hour = 10)(time_dim.t_minute < 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject @@ -85,7 +108,7 @@ PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((time_dim.t_hour = 10)(time_dim.t_minute < 30)) +--------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 10)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject @@ -107,7 +130,7 @@ PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 10)) +------------------------------filter((time_dim.t_minute < 30)(time_dim.t_hour = 11)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -129,7 +152,7 @@ PhysicalProject --------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((time_dim.t_minute < 30)(time_dim.t_hour = 11)) +----------------------------filter((time_dim.t_hour = 11)(time_dim.t_minute >= 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -151,7 +174,7 @@ PhysicalProject ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((time_dim.t_hour = 11)(time_dim.t_minute >= 30)) +--------------------------filter((time_dim.t_hour = 12)(time_dim.t_minute < 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject @@ -161,26 +184,4 @@ PhysicalProject --------------------PhysicalProject ----------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) ------------------------PhysicalOlapScan[household_demographics] -----PhysicalDistribute -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((time_dim.t_hour = 12)(time_dim.t_minute < 30)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((store.s_store_name = 'ese')) -------------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) -----------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out index 52df2ddd7bdb49..fff718f5872332 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_89 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------filter((CASE WHEN ( not (avg_monthly_sales = 0.0000)) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------filter((CASE WHEN ( not (avg_monthly_sales = 0.0000)) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter(((i_category IN ('Jewelry', 'Shoes', 'Electronics') AND i_class IN ('semi-precious', 'athletic', 'portable')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'rock', 'maternity')))) +--------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter(((i_category IN ('Jewelry', 'Shoes', 'Electronics') AND i_class IN ('semi-precious', 'athletic', 'portable')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'rock', 'maternity')))) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +----------------------------------------filter((date_dim.d_year = 1999)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out index 551b4cb7cfd07d..b245af7262a6eb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_9 -- -PhysicalProject ---NestedLoopJoin[CROSS_JOIN] +PhysicalResultSink +--PhysicalProject ----NestedLoopJoin[CROSS_JOIN] ------NestedLoopJoin[CROSS_JOIN] --------NestedLoopJoin[CROSS_JOIN] @@ -15,8 +15,21 @@ PhysicalProject ------------------------NestedLoopJoin[CROSS_JOIN] --------------------------NestedLoopJoin[CROSS_JOIN] ----------------------------NestedLoopJoin[CROSS_JOIN] -------------------------------PhysicalProject ---------------------------------NestedLoopJoin[CROSS_JOIN] +------------------------------NestedLoopJoin[CROSS_JOIN] +--------------------------------PhysicalProject +----------------------------------NestedLoopJoin[CROSS_JOIN] +------------------------------------PhysicalAssertNumRows +--------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[LOCAL] +--------------------------------------------PhysicalProject +----------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) +------------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((reason.r_reason_sk = 1)) +------------------------------------------PhysicalOlapScan[reason] +--------------------------------PhysicalDistribute ----------------------------------PhysicalAssertNumRows ------------------------------------hashAgg[GLOBAL] --------------------------------------PhysicalDistribute @@ -24,10 +37,6 @@ PhysicalProject ------------------------------------------PhysicalProject --------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) ----------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((reason.r_reason_sk = 1)) -----------------------------------------PhysicalOlapScan[reason] ------------------------------PhysicalDistribute --------------------------------PhysicalAssertNumRows ----------------------------------hashAgg[GLOBAL] @@ -42,7 +51,7 @@ PhysicalProject ----------------------------------PhysicalDistribute ------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) +----------------------------------------filter((store_sales.ss_quantity >= 21)(store_sales.ss_quantity <= 40)) ------------------------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows @@ -50,7 +59,7 @@ PhysicalProject --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------filter((store_sales.ss_quantity >= 21)(store_sales.ss_quantity <= 40)) +--------------------------------------filter((store_sales.ss_quantity <= 40)(store_sales.ss_quantity >= 21)) ----------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalAssertNumRows @@ -58,7 +67,7 @@ PhysicalProject ------------------------------PhysicalDistribute --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------filter((store_sales.ss_quantity <= 40)(store_sales.ss_quantity >= 21)) +------------------------------------filter((store_sales.ss_quantity >= 21)(store_sales.ss_quantity <= 40)) --------------------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalAssertNumRows @@ -66,7 +75,7 @@ PhysicalProject ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity >= 21)(store_sales.ss_quantity <= 40)) +----------------------------------filter((store_sales.ss_quantity <= 60)(store_sales.ss_quantity >= 41)) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute ----------------------PhysicalAssertNumRows @@ -90,7 +99,7 @@ PhysicalProject ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------filter((store_sales.ss_quantity <= 60)(store_sales.ss_quantity >= 41)) +----------------------------filter((store_sales.ss_quantity <= 80)(store_sales.ss_quantity >= 61)) ------------------------------PhysicalOlapScan[store_sales] --------------PhysicalDistribute ----------------PhysicalAssertNumRows @@ -98,7 +107,7 @@ PhysicalProject --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity <= 80)(store_sales.ss_quantity >= 61)) +--------------------------filter((store_sales.ss_quantity >= 61)(store_sales.ss_quantity <= 80)) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalDistribute --------------PhysicalAssertNumRows @@ -106,7 +115,7 @@ PhysicalProject ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------filter((store_sales.ss_quantity >= 61)(store_sales.ss_quantity <= 80)) +------------------------filter((store_sales.ss_quantity <= 80)(store_sales.ss_quantity >= 61)) --------------------------PhysicalOlapScan[store_sales] ----------PhysicalDistribute ------------PhysicalAssertNumRows @@ -114,7 +123,7 @@ PhysicalProject ----------------PhysicalDistribute ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter((store_sales.ss_quantity <= 80)(store_sales.ss_quantity >= 61)) +----------------------filter((store_sales.ss_quantity >= 81)(store_sales.ss_quantity <= 100)) ------------------------PhysicalOlapScan[store_sales] --------PhysicalDistribute ----------PhysicalAssertNumRows @@ -122,7 +131,7 @@ PhysicalProject --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------filter((store_sales.ss_quantity >= 81)(store_sales.ss_quantity <= 100)) +--------------------filter((store_sales.ss_quantity <= 100)(store_sales.ss_quantity >= 81)) ----------------------PhysicalOlapScan[store_sales] ------PhysicalDistribute --------PhysicalAssertNumRows @@ -130,14 +139,6 @@ PhysicalProject ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((store_sales.ss_quantity <= 100)(store_sales.ss_quantity >= 81)) +------------------filter((store_sales.ss_quantity >= 81)(store_sales.ss_quantity <= 100)) --------------------PhysicalOlapScan[store_sales] -----PhysicalDistribute -------PhysicalAssertNumRows ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((store_sales.ss_quantity >= 81)(store_sales.ss_quantity <= 100)) -------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out index 309483f6448848..a2d4fe07ad7a1c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out @@ -1,31 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_90 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------NestedLoopJoin[CROSS_JOIN] ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) -------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) ---------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) -----------------------------PhysicalOlapScan[web_page] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((household_demographics.hd_dep_count = 2)) -------------------------PhysicalOlapScan[household_demographics] ---------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------NestedLoopJoin[CROSS_JOIN] ----------hashAgg[GLOBAL] ------------PhysicalDistribute --------------hashAgg[LOCAL] @@ -41,10 +20,32 @@ PhysicalTopN ------------------------------PhysicalOlapScan[web_page] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((time_dim.t_hour >= 16)(time_dim.t_hour <= 17)) +--------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((household_demographics.hd_dep_count = 2)) --------------------------PhysicalOlapScan[household_demographics] +----------PhysicalDistribute +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) +--------------------------------PhysicalOlapScan[web_page] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((time_dim.t_hour >= 16)(time_dim.t_hour <= 17)) +------------------------------PhysicalOlapScan[time_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((household_demographics.hd_dep_count = 2)) +----------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out index f856ec074871f7..1e3674e05365b9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_91 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------filter((customer_address.ca_gmt_offset = -6.00)) -----------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) -------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +----------------------filter((customer_address.ca_gmt_offset = -6.00)) +------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) +--------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_returns] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[call_center] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute +------------------------------filter((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) ------------------------------PhysicalProject ---------------------------------filter((hd_buy_potential like '1001-5000%')) -----------------------------------PhysicalOlapScan[household_demographics] +--------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[call_center] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((hd_buy_potential like '1001-5000%')) +------------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out index a37b9b6b369669..af107f55b49ec4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_92 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(ws_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(ws_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] +----PhysicalTopN +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(ws_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(ws_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 320)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 320)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= 2002-05-27)(date_dim.d_date >= 2002-02-26)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= 2002-05-27)(date_dim.d_date >= 2002-02-26)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out index 4dca61b5d01296..0ead46c8d739e0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_93 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = store_sales.ss_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) -----------------PhysicalProject -------------------PhysicalOlapScan[store_sales] -----------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = store_sales.ss_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) ------------------PhysicalProject ---------------------PhysicalOlapScan[store_returns] -------------------PhysicalDistribute +--------------------PhysicalOlapScan[store_sales] +------------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) --------------------PhysicalProject -----------------------filter((cast(r_reason_desc as VARCHAR(*)) = 'duplicate purchase')) -------------------------PhysicalOlapScan[reason] +----------------------PhysicalOlapScan[store_returns] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((cast(r_reason_desc as VARCHAR(*)) = 'duplicate purchase')) +--------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out index c26035693e5596..b613247de922e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out @@ -1,37 +1,38 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) -----------------PhysicalProject -------------------filter((date_dim.d_date >= 2000-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2000-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) ---------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) ------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] -----------------------hashJoin[RIGHT_ANTI_JOIN](ws1.ws_order_number = wr1.wr_order_number) +--------------------filter((date_dim.d_date >= 2000-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2000-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) +----------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_returns] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) -----------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalOlapScan[web_sales] +------------------------hashJoin[RIGHT_ANTI_JOIN](ws1.ws_order_number = wr1.wr_order_number) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_returns] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) +------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((cast(ca_state as VARCHAR(*)) = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) -----------------------------------PhysicalOlapScan[web_site] +----------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) +------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out index 410beed37167a1..f13ff715af4682 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out @@ -10,43 +10,44 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------PhysicalProject ------------PhysicalOlapScan[web_sales] ---PhysicalTopN +--PhysicalResultSink ----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) -------------------PhysicalProject ---------------------filter((date_dim.d_date >= 1999-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('1999-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) -----------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_returns] -----------------------PhysicalDistribute -------------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) -------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) +--------------------PhysicalProject +----------------------filter((date_dim.d_date >= 1999-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('1999-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) +------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] +------------------------PhysicalDistribute +--------------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) +--------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'NC')) +----------------------------------------PhysicalOlapScan[customer_address] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'NC')) ---------------------------------------PhysicalOlapScan[customer_address] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) -------------------------------------PhysicalOlapScan[web_site] +------------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) +--------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out index d935628ed6dabb..6d7df70b717438 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +----PhysicalTopN +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((household_demographics.hd_dep_count = 3)) -------------------------PhysicalOlapScan[household_demographics] ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((store.s_store_name = 'ese')) ---------------------PhysicalOlapScan[store] +------------------------filter((household_demographics.hd_dep_count = 3)) +--------------------------PhysicalOlapScan[household_demographics] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((store.s_store_name = 'ese')) +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out index fee94a11417d07..8b918462450d96 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN](ssci.customer_sk = csci.customer_sk)(ssci.item_sk = csci.item_sk) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] ---------------------------PhysicalDistribute +----PhysicalLimit +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN](ssci.customer_sk = csci.customer_sk)(ssci.item_sk = csci.item_sk) +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) ---------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +----------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out index 0e7c482e070ade..3a1c89d6a5fa6a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_98 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= 2002-06-19)(date_dim.d_date >= 2002-05-20)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= 2002-06-19)(date_dim.d_date >= 2002-05-20)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(i_category IN ('Sports', 'Music', 'Shoes')) -------------------------------PhysicalOlapScan[item] +------------------------------filter(i_category IN ('Sports', 'Music', 'Shoes')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out index 67c328915c830e..4f20f7bf82aa1b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_99 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalOlapScan[call_center] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[call_center] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[ship_mode] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------PhysicalOlapScan[ship_mode] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] 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 00a7cf50f83f52..7740edd76cc3a5 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,12 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((lineitem.l_shipdate <= 1998-09-02)) -----------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------filter((lineitem.l_shipdate <= 1998-09-02)) +------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out index 404026aa64a74a..bbf108f0aedc48 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -------------------PhysicalProject ---------------------filter((lineitem.l_returnflag = 'R')) -----------------------PhysicalOlapScan[lineitem] -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) ---------------------------------PhysicalOlapScan[orders] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +--------------------PhysicalProject +----------------------filter((lineitem.l_returnflag = 'R')) +------------------------PhysicalOlapScan[lineitem] +--------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) ------------------------PhysicalProject ---------------------------PhysicalOlapScan[nation] +--------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) +----------------------------------PhysicalOlapScan[orders] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out index cf27cdd93f4d44..914f6976ee3744 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out @@ -1,39 +1,40 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------PhysicalProject -------------------PhysicalOlapScan[partsupp] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[supplier] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((nation.n_name = 'GERMANY')) ---------------------------PhysicalOlapScan[nation] -----------PhysicalDistribute -------------PhysicalAssertNumRows +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[GLOBAL] +----------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[partsupp] ------------------PhysicalDistribute ---------------------hashAgg[LOCAL] +--------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute +------------------------PhysicalOlapScan[supplier] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((nation.n_name = 'GERMANY')) +----------------------------PhysicalOlapScan[nation] +------------PhysicalDistribute +--------------PhysicalAssertNumRows +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject -----------------------------------filter((nation.n_name = 'GERMANY')) -------------------------------------PhysicalOlapScan[nation] +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((nation.n_name = 'GERMANY')) +--------------------------------------PhysicalOlapScan[nation] 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 cf83a4373e7a7d..1fec7c6597fd30 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,16 +1,17 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------PhysicalProject -------------------PhysicalOlapScan[orders] -----------------PhysicalProject -------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[orders] +------------------PhysicalProject +--------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out index ae50570ecf54f2..ab3e4ce172b9db 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(( not (o_comment like '%special%requests%'))) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(( not (o_comment like '%special%requests%'))) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out index 9ec9d4f3fa9854..747b102ab8889a 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) -------------PhysicalProject ---------------PhysicalOlapScan[part] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) --------------PhysicalProject -----------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) -------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[part] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) +--------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out index da9dd396bebe3d..4106594748a355 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) -----------PhysicalProject -------------PhysicalOlapScan[supplier] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) ---------------hashAgg[GLOBAL] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) +------------PhysicalProject +--------------PhysicalOlapScan[supplier] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------PhysicalOlapScan[lineitem] ----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------PhysicalAssertNumRows -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalAssertNumRows +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------------------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 d72afcf57daec7..21332b6f99dc89 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,22 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] -------------------PhysicalProject ---------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) -----------------------PhysicalOlapScan[part] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((s_comment like '%Customer%Complaints%')) -----------------------PhysicalOlapScan[supplier] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] +--------------------PhysicalProject +----------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +------------------------PhysicalOlapScan[part] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((s_comment like '%Customer%Complaints%')) +------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out index 90b6d8a1903660..65bad6d59809a4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ------------------------PhysicalProject ---------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) -----------------------------PhysicalOlapScan[part] +--------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) +------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out index e91c302aaa36cb..ae401ef1cd429f 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------PhysicalProject ---------------PhysicalOlapScan[lineitem] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------filter((sum(l_quantity) > 300.00)) ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[lineitem] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------filter((sum(l_quantity) > 300.00)) +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out index 150f569cdfc749..44f7c223f11375 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) -----------PhysicalProject -------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) ---------------PhysicalOlapScan[lineitem] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) -----------------PhysicalOlapScan[part] +--------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) +----------------PhysicalOlapScan[lineitem] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) +------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out index ec4c6370b694a4..e4513c0ef82bb4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------PhysicalOlapScan[supplier] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'EUROPE')) -----------------------------------PhysicalOlapScan[region] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalProject -----------------------------filter((part.p_size = 15)(p_type like '%BRASS')) -------------------------------PhysicalOlapScan[part] +------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------PhysicalOlapScan[supplier] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'EUROPE')) +------------------------------------PhysicalOlapScan[region] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalProject +------------------------------filter((part.p_size = 15)(p_type like '%BRASS')) +--------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out index 2e6728910560ae..300ef0d9d617ea 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = t3.ps_suppkey) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[INNER_JOIN](t2.l_partkey = t1.ps_partkey)(t2.l_suppkey = t1.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > t2.l_q) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] ---------------------PhysicalProject -----------------------filter((p_name like 'forest%')) -------------------------PhysicalOlapScan[part] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = t3.ps_suppkey) +------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[supplier] ---------------PhysicalDistribute +----------------hashJoin[INNER_JOIN](t2.l_partkey = t1.ps_partkey)(t2.l_suppkey = t1.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > t2.l_q) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] +----------------------PhysicalProject +------------------------filter((p_name like 'forest%')) +--------------------------PhysicalOlapScan[part] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------PhysicalProject -------------------filter((nation.n_name = 'CANADA')) ---------------------PhysicalOlapScan[nation] +------------------PhysicalOlapScan[supplier] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((nation.n_name = 'CANADA')) +----------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out index af742120ca219e..d3901ee9c271af 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] ---------------------PhysicalProject -----------------------filter((p_name like 'forest%')) -------------------------PhysicalOlapScan[part] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[supplier] ---------------PhysicalDistribute +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] +----------------------PhysicalProject +------------------------filter((p_name like 'forest%')) +--------------------------PhysicalOlapScan[part] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------PhysicalProject -------------------filter((nation.n_name = 'CANADA')) ---------------------PhysicalOlapScan[nation] +------------------PhysicalOlapScan[supplier] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((nation.n_name = 'CANADA')) +----------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out index 722b813a7d96f9..2f4348619b2373 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) -----------------PhysicalProject -------------------filter((orders.o_orderstatus = 'F')) ---------------------PhysicalOlapScan[orders] -----------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) ------------------PhysicalProject ---------------------PhysicalOlapScan[lineitem] -------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +--------------------filter((orders.o_orderstatus = 'F')) +----------------------PhysicalOlapScan[orders] +------------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) --------------------PhysicalProject -----------------------filter((l3.l_receiptdate > l3.l_commitdate)) -------------------------PhysicalOlapScan[lineitem] ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) ----------------------PhysicalProject -------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------filter((l3.l_receiptdate > l3.l_commitdate)) --------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[supplier] ---------------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +------------------------PhysicalProject +--------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +----------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------------PhysicalProject -------------------------------filter((nation.n_name = 'SAUDI ARABIA')) ---------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalOlapScan[supplier] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +----------------------------------PhysicalOlapScan[nation] 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 b2399ebd132d2b..3525bb716889f9 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,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) -----------------PhysicalProject -------------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalAssertNumRows ---------------------hashAgg[GLOBAL] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) +------------------PhysicalProject +--------------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) ----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -------------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalAssertNumRows +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out index 23fc521663607a..da651e12cda6bc 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out @@ -1,24 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ---------------PhysicalProject -----------------filter((lineitem.l_shipdate > 1995-03-15)) -------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-03-15)) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((customer.c_mktsegment = 'BUILDING')) ---------------------------PhysicalOlapScan[customer] +------------------filter((lineitem.l_shipdate > 1995-03-15)) +--------------------PhysicalOlapScan[lineitem] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-03-15)) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((customer.c_mktsegment = 'BUILDING')) +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out index 05c67b9b65c80b..4d9d91d96323ae 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out @@ -1,17 +1,18 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------PhysicalProject -------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] -----------------PhysicalProject -------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) ---------------------PhysicalOlapScan[orders] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------PhysicalProject +--------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] +------------------PhysicalProject +--------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) +----------------------PhysicalOlapScan[orders] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out index 6d45aeda6ba21a..bf8fb1842de788 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ------------------------------------PhysicalProject ---------------------------------------filter((region.r_name = 'ASIA')) -----------------------------------------PhysicalOlapScan[region] -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) ---------------------------PhysicalOlapScan[orders] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] +--------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((region.r_name = 'ASIA')) +------------------------------------------PhysicalOlapScan[region] +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) +----------------------------PhysicalOlapScan[orders] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out index da938bd9de6bab..0cd7c1f4b2b411 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out @@ -1,9 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) -----------PhysicalOlapScan[lineitem] +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) +------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out index 8969786cfaac9f..2eefbd562eb216 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) -------------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) -----------------------------------PhysicalProject -------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ---------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +--------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------PhysicalProject +------------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) +--------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) ------------------------------------PhysicalProject ---------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +--------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ----------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +------------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out index cd339810e07e96..07a9cd9ca20154 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out @@ -1,48 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer] -------------------------------------PhysicalDistribute ---------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------------------------PhysicalProject -------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) ---------------------------------------------PhysicalOlapScan[orders] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[lineitem] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) ---------------------------------------------------PhysicalOlapScan[part] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[supplier] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) ------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'AMERICA')) -----------------------------------PhysicalOlapScan[region] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[nation] +--------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer] +--------------------------------------PhysicalDistribute +----------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------------------------PhysicalProject +--------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) +----------------------------------------------PhysicalOlapScan[orders] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[lineitem] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) +----------------------------------------------------PhysicalOlapScan[part] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'AMERICA')) +------------------------------------PhysicalOlapScan[region] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out index e20942535d3417..25705c06d641d2 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out @@ -1,38 +1,39 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[lineitem] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((p_name like '%green%')) +----------------------------------------PhysicalOlapScan[part] +------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[lineitem] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((p_name like '%green%')) ---------------------------------------PhysicalOlapScan[part] -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[supplier] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] +------------------------------------PhysicalOlapScan[supplier] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out index 00a7cf50f83f52..7740edd76cc3a5 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out @@ -1,12 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((lineitem.l_shipdate <= 1998-09-02)) -----------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------filter((lineitem.l_shipdate <= 1998-09-02)) +------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out index 404026aa64a74a..bbf108f0aedc48 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -------------------PhysicalProject ---------------------filter((lineitem.l_returnflag = 'R')) -----------------------PhysicalOlapScan[lineitem] -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) ---------------------------------PhysicalOlapScan[orders] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +--------------------PhysicalProject +----------------------filter((lineitem.l_returnflag = 'R')) +------------------------PhysicalOlapScan[lineitem] +--------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) ------------------------PhysicalProject ---------------------------PhysicalOlapScan[nation] +--------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) +----------------------------------PhysicalOlapScan[orders] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out index cf27cdd93f4d44..914f6976ee3744 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out @@ -1,39 +1,40 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------PhysicalProject -------------------PhysicalOlapScan[partsupp] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[supplier] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((nation.n_name = 'GERMANY')) ---------------------------PhysicalOlapScan[nation] -----------PhysicalDistribute -------------PhysicalAssertNumRows +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[GLOBAL] +----------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[partsupp] ------------------PhysicalDistribute ---------------------hashAgg[LOCAL] +--------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute +------------------------PhysicalOlapScan[supplier] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((nation.n_name = 'GERMANY')) +----------------------------PhysicalOlapScan[nation] +------------PhysicalDistribute +--------------PhysicalAssertNumRows +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject -----------------------------------filter((nation.n_name = 'GERMANY')) -------------------------------------PhysicalOlapScan[nation] +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((nation.n_name = 'GERMANY')) +--------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out index cf83a4373e7a7d..1fec7c6597fd30 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out @@ -1,16 +1,17 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------PhysicalProject -------------------PhysicalOlapScan[orders] -----------------PhysicalProject -------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[orders] +------------------PhysicalProject +--------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out index ae50570ecf54f2..ab3e4ce172b9db 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(( not (o_comment like '%special%requests%'))) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(( not (o_comment like '%special%requests%'))) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out index 9ec9d4f3fa9854..747b102ab8889a 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) -------------PhysicalProject ---------------PhysicalOlapScan[part] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) --------------PhysicalProject -----------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) -------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[part] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) +--------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out index da9dd396bebe3d..4106594748a355 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) -----------PhysicalProject -------------PhysicalOlapScan[supplier] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) ---------------hashAgg[GLOBAL] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) +------------PhysicalProject +--------------PhysicalOlapScan[supplier] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------PhysicalOlapScan[lineitem] ----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------PhysicalAssertNumRows -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalAssertNumRows +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out index d72afcf57daec7..21332b6f99dc89 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out @@ -1,21 +1,22 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] -------------------PhysicalProject ---------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) -----------------------PhysicalOlapScan[part] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((s_comment like '%Customer%Complaints%')) -----------------------PhysicalOlapScan[supplier] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] +--------------------PhysicalProject +----------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +------------------------PhysicalOlapScan[part] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((s_comment like '%Customer%Complaints%')) +------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out index 90b6d8a1903660..65bad6d59809a4 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ------------------------PhysicalProject ---------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) -----------------------------PhysicalOlapScan[part] +--------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) +------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out index e91c302aaa36cb..ae401ef1cd429f 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------PhysicalProject ---------------PhysicalOlapScan[lineitem] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------filter((sum(l_quantity) > 300.00)) ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[lineitem] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------filter((sum(l_quantity) > 300.00)) +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out index 150f569cdfc749..44f7c223f11375 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) -----------PhysicalProject -------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) ---------------PhysicalOlapScan[lineitem] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) -----------------PhysicalOlapScan[part] +--------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) +----------------PhysicalOlapScan[lineitem] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) +------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out index ec4c6370b694a4..e4513c0ef82bb4 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------PhysicalOlapScan[supplier] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'EUROPE')) -----------------------------------PhysicalOlapScan[region] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalProject -----------------------------filter((part.p_size = 15)(p_type like '%BRASS')) -------------------------------PhysicalOlapScan[part] +------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------PhysicalOlapScan[supplier] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'EUROPE')) +------------------------------------PhysicalOlapScan[region] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalProject +------------------------------filter((part.p_size = 15)(p_type like '%BRASS')) +--------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out index af742120ca219e..d3901ee9c271af 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] ---------------------PhysicalProject -----------------------filter((p_name like 'forest%')) -------------------------PhysicalOlapScan[part] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[supplier] ---------------PhysicalDistribute +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] +----------------------PhysicalProject +------------------------filter((p_name like 'forest%')) +--------------------------PhysicalOlapScan[part] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------PhysicalProject -------------------filter((nation.n_name = 'CANADA')) ---------------------PhysicalOlapScan[nation] +------------------PhysicalOlapScan[supplier] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((nation.n_name = 'CANADA')) +----------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out index 722b813a7d96f9..2f4348619b2373 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) -----------------PhysicalProject -------------------filter((orders.o_orderstatus = 'F')) ---------------------PhysicalOlapScan[orders] -----------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) ------------------PhysicalProject ---------------------PhysicalOlapScan[lineitem] -------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +--------------------filter((orders.o_orderstatus = 'F')) +----------------------PhysicalOlapScan[orders] +------------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) --------------------PhysicalProject -----------------------filter((l3.l_receiptdate > l3.l_commitdate)) -------------------------PhysicalOlapScan[lineitem] ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) ----------------------PhysicalProject -------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------filter((l3.l_receiptdate > l3.l_commitdate)) --------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[supplier] ---------------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +------------------------PhysicalProject +--------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +----------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------------PhysicalProject -------------------------------filter((nation.n_name = 'SAUDI ARABIA')) ---------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalOlapScan[supplier] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +----------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out index b2399ebd132d2b..3525bb716889f9 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) -----------------PhysicalProject -------------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalAssertNumRows ---------------------hashAgg[GLOBAL] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) +------------------PhysicalProject +--------------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) ----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -------------------------------PhysicalOlapScan[customer] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalAssertNumRows +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out index 23fc521663607a..da651e12cda6bc 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out @@ -1,24 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ---------------PhysicalProject -----------------filter((lineitem.l_shipdate > 1995-03-15)) -------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-03-15)) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((customer.c_mktsegment = 'BUILDING')) ---------------------------PhysicalOlapScan[customer] +------------------filter((lineitem.l_shipdate > 1995-03-15)) +--------------------PhysicalOlapScan[lineitem] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-03-15)) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((customer.c_mktsegment = 'BUILDING')) +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out index 05c67b9b65c80b..4d9d91d96323ae 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out @@ -1,17 +1,18 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------PhysicalProject -------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] -----------------PhysicalProject -------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) ---------------------PhysicalOlapScan[orders] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------PhysicalProject +--------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] +------------------PhysicalProject +--------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) +----------------------PhysicalOlapScan[orders] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out index 6d45aeda6ba21a..bf8fb1842de788 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ------------------------------------PhysicalProject ---------------------------------------filter((region.r_name = 'ASIA')) -----------------------------------------PhysicalOlapScan[region] -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) ---------------------------PhysicalOlapScan[orders] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] +--------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((region.r_name = 'ASIA')) +------------------------------------------PhysicalOlapScan[region] +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) +----------------------------PhysicalOlapScan[orders] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out index da938bd9de6bab..0cd7c1f4b2b411 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out @@ -1,9 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) -----------PhysicalOlapScan[lineitem] +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) +------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out index 8969786cfaac9f..2eefbd562eb216 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) -------------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) -----------------------------------PhysicalProject -------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ---------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +--------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------PhysicalProject +------------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) +--------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) ------------------------------------PhysicalProject ---------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +--------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ----------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +------------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out index cd339810e07e96..07a9cd9ca20154 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out @@ -1,48 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer] -------------------------------------PhysicalDistribute ---------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------------------------PhysicalProject -------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) ---------------------------------------------PhysicalOlapScan[orders] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[lineitem] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) ---------------------------------------------------PhysicalOlapScan[part] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[supplier] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) ------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'AMERICA')) -----------------------------------PhysicalOlapScan[region] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[nation] +--------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer] +--------------------------------------PhysicalDistribute +----------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------------------------PhysicalProject +--------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) +----------------------------------------------PhysicalOlapScan[orders] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[lineitem] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) +----------------------------------------------------PhysicalOlapScan[part] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'AMERICA')) +------------------------------------PhysicalOlapScan[region] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out index 6a73f8f1f00996..a46d07b12562dc 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineitem] -------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) --------------------------------PhysicalProject -----------------------------------filter((p_name like '%green%')) -------------------------------------PhysicalOlapScan[part] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[lineitem] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((p_name like '%green%')) +--------------------------------------PhysicalOlapScan[part] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[nation] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[nation] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp]